You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by "mobuchowski (via GitHub)" <gi...@apache.org> on 2023/03/06 18:34:54 UTC

[GitHub] [airflow] mobuchowski opened a new pull request, #29940: Add base OpenLineage provider implementation

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

   This PR consistent mostly of code that was created in OpenLineage project. It consists of
   
   - Provider wiring
   - `OpenLineageListener` that uses Listener API to get notification about changes to TaskInstance and Dag states
   - `Extractor` framework, which is used to extract lineage information from particular operators. It's ment to be replaced by direct implementation of lineage features in later phase. This PR does not include actual extractors, but code around using and registering them.
   - `OpenLineageAdapter` that translates extracted information to OpenLineage events.
   - Utils around specific Airflow OL facets and features
   
   This is a base implementation that's not ment to be released yet, but to add code modified to be consistent with Airflow standards, get early feedback and provide canvas to add later features, docs, tests on.
   
   Closes: #29669
   
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1156118477


##########
airflow/providers/openlineage/utils/utils.py:
##########
@@ -0,0 +1,482 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from contextlib import suppress
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attrs
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def redacted_connection_uri(conn: Connection, filtered_params=None, filtered_prefixes=None):
+    """
+    Return the connection URI for the given Connection.
+    This method additionally filters URI by removing query parameters that are known to carry sensitive data
+    like username, password, access key.
+    """
+    if filtered_prefixes is None:
+        filtered_prefixes = []
+    if filtered_params is None:
+        filtered_params = []
+
+    def filter_key_params(k: str):
+        return k not in filtered_params and any(substr in k for substr in filtered_prefixes)
+
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not filter_key_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def get_normalized_postgres_connection_uri(conn):
+    """
+    URIs starting with postgresql:// and postgres:// are both valid
+    PostgreSQL connection strings. This function normalizes it to
+    postgres:// as canonical name according to OpenLineage spec.
+    """
+    uri = redacted_connection_uri(conn)
+    if uri.startswith("postgresql"):
+        uri = uri.replace("postgresql", "postgres", 1)
+    return uri
+
+
+def get_connection(conn_id) -> Connection | None:
+    from airflow.hooks.base import BaseHook
+
+    with suppress(Exception):
+        return BaseHook.get_connection(conn_id=conn_id)
+    return None
+
+
+def get_job_name(task):
+    return f"{task.dag_id}.{task.task_id}"
+
+
+def get_custom_facets(
+    dagrun, task, is_external_trigger: bool, task_instance: TaskInstance | None = None
+) -> dict[str, Any]:
+    custom_facets = {
+        "airflow_runArgs": AirflowRunArgsRunFacet(is_external_trigger),
+        "airflow_version": AirflowVersionRunFacet.from_dagrun_and_task(dagrun, task),
+    }
+    # check for -1 comes from SmartSensor compatibility with dynamic task mapping
+    # this comes from Airflow code
+    if hasattr(task_instance, "map_index") and getattr(task_instance, "map_index") != -1:
+        custom_facets["airflow_mappedTask"] = AirflowMappedTaskRunFacet.from_task_instance(task_instance)
+    return custom_facets
+
+
+class InfoJsonEncodable(dict):
+    """
+    Airflow objects might not be json-encodable overall.
+
+    The class provides additional attributes to control
+    what and how is encoded:
+    * renames: a dictionary of attribute name changes
+    * casts: a dictionary consisting of attribute names
+             and corresponding methods that should change
+             object value
+    * includes: list of attributes to be included in encoding
+    * excludes: list of attributes to be excluded from encoding
+
+    Don't use both includes and excludes.
+    """
+
+    renames: dict[str, str] = {}
+    casts: dict[str, Any] = {}
+    includes: list[str] = []
+    excludes: list[str] = []
+
+    def __init__(self, obj):
+        self.obj = obj
+        self._fields = []
+
+        self._cast_fields()
+        self._rename_fields()
+        self._include_fields()
+        dict.__init__(
+            self,
+            **{field: InfoJsonEncodable._cast_basic_types(getattr(self, field)) for field in self._fields},
+        )
+
+    @staticmethod
+    def _cast_basic_types(value):
+        if isinstance(value, datetime.datetime):
+            return value.isoformat()
+        if isinstance(value, (set, list, tuple)):
+            return str(list(value))
+        return value
+
+    def _rename_fields(self):
+        for field, renamed in self.renames.items():
+            if hasattr(self.obj, field):
+                setattr(self, renamed, getattr(self.obj, field))
+                self._fields.append(renamed)
+
+    def _cast_fields(self):
+        for field, func in self.casts.items():
+            setattr(self, field, func(self.obj))
+            self._fields.append(field)
+
+    def _include_fields(self):
+        if self.includes and self.excludes:
+            raise Exception("Don't use both includes and excludes.")
+        if self.includes:
+            for field in self.includes:
+                if field in self._fields or not hasattr(self.obj, field):
+                    continue
+                setattr(self, field, getattr(self.obj, field))
+                self._fields.append(field)
+        else:
+            for field, val in self.obj.__dict__.items():
+                if field in self._fields or field in self.excludes or field in self.renames:
+                    continue
+                setattr(self, field, val)
+                self._fields.append(field)
+
+
+class DagInfo(InfoJsonEncodable):
+    """Defines encoding DAG object to JSON."""
+
+    includes = ["dag_id", "schedule_interval", "tags", "start_date"]
+    casts = {"timetable": lambda dag: dag.timetable.serialize() if getattr(dag, "timetable", None) else None}
+    renames = {"_dag_id": "dag_id"}
+
+
+class DagRunInfo(InfoJsonEncodable):
+    """Defines encoding DagRun object to JSON."""
+
+    includes = [
+        "conf",
+        "dag_id",
+        "data_interval_start",
+        "data_interval_end",
+        "external_trigger",
+        "run_id",
+        "run_type",
+        "start_date",
+    ]
+
+
+class TaskInstanceInfo(InfoJsonEncodable):
+    """Defines encoding TaskInstance object to JSON."""
+
+    includes = ["duration", "try_number", "pool"]
+    casts = {
+        "map_index": lambda ti: ti.map_index
+        if hasattr(ti, "map_index") and getattr(ti, "map_index") != -1
+        else None
+    }
+
+
+class TaskInfo(InfoJsonEncodable):
+    """Defines encoding BaseOperator/AbstractOperator object to JSON."""
+
+    renames = {
+        "_BaseOperator__init_kwargs": "args",
+        "_BaseOperator__from_mapped": "mapped",
+        "_downstream_task_ids": "downstream_task_ids",
+        "_upstream_task_ids": "upstream_task_ids",
+    }
+    excludes = [
+        "_BaseOperator__instantiated",
+        "_dag",
+        "_hook",
+        "_log",
+        "_outlets",
+        "_inlets",
+        "_lock_for_execution",
+        "handler",
+        "params",
+        "python_callable",
+        "retry_delay",
+    ]
+    casts = {
+        "operator_class": lambda task: f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",  # noqa
+        "task_group": lambda task: TaskGroupInfo(task.task_group)
+        if hasattr(task, "task_group") and getattr(task.task_group, "_group_id", None)
+        else None,
+    }
+
+
+class TaskGroupInfo(InfoJsonEncodable):
+    """Defines encoding TaskGroup object to JSON."""
+
+    renames = {
+        "_group_id": "group_id",
+    }
+    includes = [
+        "downstream_group_ids",
+        "downstream_task_ids",
+        "prefix_group_id",
+        "tooltip",
+        "upstream_group_ids",
+        "upstream_task_ids",
+    ]
+
+
+def get_airflow_run_facet(
+    dag_run: DagRun,
+    dag: DAG,
+    task_instance: TaskInstance,
+    task: BaseOperator,
+    task_uuid: str,
+):
+    return {
+        "airflow": json.loads(
+            json.dumps(
+                asdict(
+                    AirflowRunFacet(
+                        dag=DagInfo(dag),
+                        dagRun=DagRunInfo(dag_run),
+                        taskInstance=TaskInstanceInfo(task_instance),
+                        task=TaskInfo(task),
+                        taskUuid=task_uuid,
+                    )
+                ),
+                default=str,
+            )
+        )
+    }
+
+
+def import_from_string(path: str):
+    try:
+        module_path, target = path.rsplit(".", 1)
+        module = importlib.import_module(module_path)
+        return getattr(module, target)
+    except Exception as e:
+        log.warning(e)
+        raise ImportError(f"Failed to import {path}") from e

Review Comment:
   I'll remove it and just use 
   ```
       with suppress(ImportError):
           return import_string(path)
   ```
   if we need it in following PRs
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] eladkal commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "eladkal (via GitHub)" <gi...@apache.org>.
eladkal commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1132818244


##########
airflow/providers/openlineage/provider.yaml:
##########
@@ -0,0 +1,42 @@
+# 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.
+
+---
+package-name: apache-airflow-providers-openlineage
+name: OpenLineage Airflow
+description: |
+  `OpenLineage <https://openlineage.io/>`__
+
+versions:
+  - 1.0.0
+
+dependencies:
+  - apache-airflow>=2.5.1

Review Comment:
   Not something we should focus on now. I will not release the provider before the AIP is complete so we can update this one later.
   OL support is considered a feature thus core changes related to it will be in 2.6 or 2.7



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1156130110


##########
airflow/providers/openlineage/utils/utils.py:
##########
@@ -0,0 +1,482 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from contextlib import suppress
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attrs
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def redacted_connection_uri(conn: Connection, filtered_params=None, filtered_prefixes=None):
+    """
+    Return the connection URI for the given Connection.
+    This method additionally filters URI by removing query parameters that are known to carry sensitive data
+    like username, password, access key.
+    """
+    if filtered_prefixes is None:
+        filtered_prefixes = []
+    if filtered_params is None:
+        filtered_params = []
+
+    def filter_key_params(k: str):
+        return k not in filtered_params and any(substr in k for substr in filtered_prefixes)
+
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not filter_key_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def get_normalized_postgres_connection_uri(conn):
+    """
+    URIs starting with postgresql:// and postgres:// are both valid
+    PostgreSQL connection strings. This function normalizes it to
+    postgres:// as canonical name according to OpenLineage spec.
+    """
+    uri = redacted_connection_uri(conn)
+    if uri.startswith("postgresql"):
+        uri = uri.replace("postgresql", "postgres", 1)
+    return uri
+
+
+def get_connection(conn_id) -> Connection | None:
+    from airflow.hooks.base import BaseHook
+
+    with suppress(Exception):
+        return BaseHook.get_connection(conn_id=conn_id)
+    return None
+
+
+def get_job_name(task):
+    return f"{task.dag_id}.{task.task_id}"
+
+
+def get_custom_facets(
+    dagrun, task, is_external_trigger: bool, task_instance: TaskInstance | None = None
+) -> dict[str, Any]:
+    custom_facets = {
+        "airflow_runArgs": AirflowRunArgsRunFacet(is_external_trigger),
+        "airflow_version": AirflowVersionRunFacet.from_dagrun_and_task(dagrun, task),
+    }
+    # check for -1 comes from SmartSensor compatibility with dynamic task mapping
+    # this comes from Airflow code
+    if hasattr(task_instance, "map_index") and getattr(task_instance, "map_index") != -1:
+        custom_facets["airflow_mappedTask"] = AirflowMappedTaskRunFacet.from_task_instance(task_instance)
+    return custom_facets
+
+
+class InfoJsonEncodable(dict):
+    """
+    Airflow objects might not be json-encodable overall.
+
+    The class provides additional attributes to control
+    what and how is encoded:
+    * renames: a dictionary of attribute name changes
+    * casts: a dictionary consisting of attribute names
+             and corresponding methods that should change
+             object value
+    * includes: list of attributes to be included in encoding
+    * excludes: list of attributes to be excluded from encoding
+
+    Don't use both includes and excludes.
+    """
+
+    renames: dict[str, str] = {}
+    casts: dict[str, Any] = {}
+    includes: list[str] = []
+    excludes: list[str] = []
+
+    def __init__(self, obj):
+        self.obj = obj
+        self._fields = []
+
+        self._cast_fields()
+        self._rename_fields()
+        self._include_fields()
+        dict.__init__(
+            self,
+            **{field: InfoJsonEncodable._cast_basic_types(getattr(self, field)) for field in self._fields},
+        )
+
+    @staticmethod
+    def _cast_basic_types(value):
+        if isinstance(value, datetime.datetime):
+            return value.isoformat()
+        if isinstance(value, (set, list, tuple)):
+            return str(list(value))
+        return value
+
+    def _rename_fields(self):
+        for field, renamed in self.renames.items():
+            if hasattr(self.obj, field):
+                setattr(self, renamed, getattr(self.obj, field))
+                self._fields.append(renamed)
+
+    def _cast_fields(self):
+        for field, func in self.casts.items():
+            setattr(self, field, func(self.obj))
+            self._fields.append(field)
+
+    def _include_fields(self):
+        if self.includes and self.excludes:
+            raise Exception("Don't use both includes and excludes.")
+        if self.includes:
+            for field in self.includes:
+                if field in self._fields or not hasattr(self.obj, field):
+                    continue
+                setattr(self, field, getattr(self.obj, field))
+                self._fields.append(field)
+        else:
+            for field, val in self.obj.__dict__.items():
+                if field in self._fields or field in self.excludes or field in self.renames:
+                    continue
+                setattr(self, field, val)
+                self._fields.append(field)
+
+
+class DagInfo(InfoJsonEncodable):
+    """Defines encoding DAG object to JSON."""
+
+    includes = ["dag_id", "schedule_interval", "tags", "start_date"]
+    casts = {"timetable": lambda dag: dag.timetable.serialize() if getattr(dag, "timetable", None) else None}
+    renames = {"_dag_id": "dag_id"}
+
+
+class DagRunInfo(InfoJsonEncodable):
+    """Defines encoding DagRun object to JSON."""
+
+    includes = [
+        "conf",
+        "dag_id",
+        "data_interval_start",
+        "data_interval_end",
+        "external_trigger",
+        "run_id",
+        "run_type",
+        "start_date",
+    ]
+
+
+class TaskInstanceInfo(InfoJsonEncodable):
+    """Defines encoding TaskInstance object to JSON."""
+
+    includes = ["duration", "try_number", "pool"]
+    casts = {
+        "map_index": lambda ti: ti.map_index
+        if hasattr(ti, "map_index") and getattr(ti, "map_index") != -1
+        else None
+    }
+
+
+class TaskInfo(InfoJsonEncodable):
+    """Defines encoding BaseOperator/AbstractOperator object to JSON."""
+
+    renames = {
+        "_BaseOperator__init_kwargs": "args",
+        "_BaseOperator__from_mapped": "mapped",
+        "_downstream_task_ids": "downstream_task_ids",
+        "_upstream_task_ids": "upstream_task_ids",
+    }
+    excludes = [
+        "_BaseOperator__instantiated",
+        "_dag",
+        "_hook",
+        "_log",
+        "_outlets",
+        "_inlets",
+        "_lock_for_execution",
+        "handler",
+        "params",
+        "python_callable",
+        "retry_delay",
+    ]
+    casts = {
+        "operator_class": lambda task: f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",  # noqa
+        "task_group": lambda task: TaskGroupInfo(task.task_group)
+        if hasattr(task, "task_group") and getattr(task.task_group, "_group_id", None)
+        else None,
+    }
+
+
+class TaskGroupInfo(InfoJsonEncodable):
+    """Defines encoding TaskGroup object to JSON."""
+
+    renames = {
+        "_group_id": "group_id",
+    }
+    includes = [
+        "downstream_group_ids",
+        "downstream_task_ids",
+        "prefix_group_id",
+        "tooltip",
+        "upstream_group_ids",
+        "upstream_task_ids",
+    ]
+
+
+def get_airflow_run_facet(
+    dag_run: DagRun,
+    dag: DAG,
+    task_instance: TaskInstance,
+    task: BaseOperator,
+    task_uuid: str,
+):
+    return {
+        "airflow": json.loads(
+            json.dumps(
+                asdict(
+                    AirflowRunFacet(
+                        dag=DagInfo(dag),
+                        dagRun=DagRunInfo(dag_run),
+                        taskInstance=TaskInstanceInfo(task_instance),
+                        task=TaskInfo(task),
+                        taskUuid=task_uuid,
+                    )
+                ),
+                default=str,
+            )
+        )
+    }
+
+
+def import_from_string(path: str):
+    try:
+        module_path, target = path.rsplit(".", 1)
+        module = importlib.import_module(module_path)
+        return getattr(module, target)
+    except Exception as e:
+        log.warning(e)
+        raise ImportError(f"Failed to import {path}") from e
+
+
+def try_import_from_string(path: str):
+    with suppress(ImportError):
+        return import_from_string(path)
+
+
+def redact_with_exclusions(source: Any):
+    """This function redacts sensitive data similar to SecretsMasker in Airflow logs.
+    The difference is that MAX_RECURSION_DEPTH is way higher - due to the structure of OL events
+    we need more depth.
+    Also, we allow data structures to specify data that needs not to be redacted by specifying
+    _skip_redact list.
+    """
+    try:
+        from airflow.utils.log.secrets_masker import (
+            _secrets_masker,
+            should_hide_value_for_key,
+        )
+    except ImportError:
+        return source
+
+    sm = _secrets_masker()
+    MAX_RECURSION_DEPTH = 20
+
+    def _redact(item, name: str | None, depth: int):
+        if depth > MAX_RECURSION_DEPTH:
+            return item

Review Comment:
   @ashb will it be okay to add exclusion (in similar way) and increase depth level then? If yes, we can just move this to Airflow



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] JDarDagran commented on pull request #29940: Add base OpenLineage provider implementation

Posted by "JDarDagran (via GitHub)" <gi...@apache.org>.
JDarDagran commented on PR #29940:
URL: https://github.com/apache/airflow/pull/29940#issuecomment-1464307325

   Also, `Build docs` step fails when trying to fetch inventory for OpenLineage provider. I'm not sure if that's something that should be uploaded manually at first?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] ashb commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "ashb (via GitHub)" <gi...@apache.org>.
ashb commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1138369962


##########
airflow/providers/openlineage/plugins/macros.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.
+from __future__ import annotations
+
+import os
+import typing
+
+from airflow.providers.openlineage.plugins.adapter import OpenLineageAdapter
+
+if typing.TYPE_CHECKING:
+    from airflow.models import BaseOperator, TaskInstance
+
+_JOB_NAMESPACE = os.getenv("OPENLINEAGE_NAMESPACE", "default")
+
+
+def lineage_run_id(task: BaseOperator, task_instance: TaskInstance):
+    """
+    Macro function which returns the generated run id for a given task. This
+    can be used to forward the run id from a task to a child run so the job
+    hierarchy is preserved. Invoke as a jinja template, e.g.
+
+    PythonOperator(
+        task_id='render_template',
+        python_callable=my_task_function,
+        op_args=['{{ lineage_run_id(task, task_instance) }}'], # lineage_run_id macro invoked
+        provide_context=False,
+        dag=dag
+    )
+    """
+    return OpenLineageAdapter.build_task_instance_run_id(
+        task.task_id, task_instance.execution_date, task_instance.try_number
+    )
+
+
+def lineage_parent_id(run_id: str, task: BaseOperator, task_instance: TaskInstance):
+    """
+    Macro function which returns the generated job and run id for a given task. This
+    can be used to forward the ids from a task to a child run so the job
+    hierarchy is preserved. Child run can create ParentRunFacet from those ids.
+    Invoke as a jinja template, e.g.
+
+    PythonOperator(
+        task_id='render_template',
+        python_callable=my_task_function,
+        op_args=['{{ lineage_parent_id(run_id, task, task_instance) }}'], # macro invoked

Review Comment:
   Huh so it is!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1135361660


##########
airflow/providers/openlineage/plugins/adapter.py:
##########
@@ -0,0 +1,302 @@
+# 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
+import uuid
+from typing import TYPE_CHECKING
+
+import requests.exceptions
+
+from airflow.providers.openlineage import version as OPENLINEAGE_PROVIDER_VERSION
+from airflow.providers.openlineage.extractors import OperatorLineage
+from airflow.providers.openlineage.utils import redact_with_exclusions
+from airflow.utils.log.logging_mixin import LoggingMixin
+from openlineage.client import OpenLineageClient, set_producer
+from openlineage.client.facet import (
+    BaseFacet,
+    DocumentationJobFacet,
+    ErrorMessageRunFacet,
+    NominalTimeRunFacet,
+    OwnershipJobFacet,
+    OwnershipJobFacetOwners,
+    ParentRunFacet,
+    ProcessingEngineRunFacet,
+    SourceCodeLocationJobFacet,
+)
+from openlineage.client.run import Job, Run, RunEvent, RunState
+
+if TYPE_CHECKING:
+    from airflow.models.dagrun import DagRun
+
+
+_DAG_DEFAULT_NAMESPACE = "default"
+
+_DAG_NAMESPACE = os.getenv("OPENLINEAGE_NAMESPACE", _DAG_DEFAULT_NAMESPACE)
+
+_PRODUCER = f"https://github.com/apache/airflow/tree/providers-openlineage/" f"{OPENLINEAGE_PROVIDER_VERSION}"
+
+set_producer(_PRODUCER)
+
+
+class OpenLineageAdapter(LoggingMixin):
+    """
+    Adapter for translating Airflow metadata to OpenLineage events,
+    instead of directly creating them from Airflow code.
+    """
+
+    def __init__(self, client=None):
+        super().__init__()
+        self._client = client
+
+    def get_or_create_openlineage_client(self) -> OpenLineageClient:
+        if not self._client:
+            self._client = OpenLineageClient.from_environment()
+        return self._client
+
+    def build_dag_run_id(self, dag_id, dag_run_id):
+        return str(uuid.uuid3(uuid.NAMESPACE_URL, f"{_DAG_NAMESPACE}.{dag_id}.{dag_run_id}"))
+
+    @staticmethod
+    def build_task_instance_run_id(task_id, execution_date, try_number):
+        return str(
+            uuid.uuid3(
+                uuid.NAMESPACE_URL,
+                f"{_DAG_NAMESPACE}.{task_id}.{execution_date}.{try_number}",
+            )
+        )
+
+    def emit(self, event: RunEvent):
+        event = redact_with_exclusions(event)
+        try:
+            return self.get_or_create_openlineage_client().emit(event)
+        except requests.exceptions.RequestException:
+            self.log.exception(f"Failed to emit OpenLineage event of id {event.run.runId}")
+
+    def start_task(
+        self,
+        run_id: str,
+        job_name: str,
+        job_description: str,
+        event_time: str,
+        parent_job_name: str | None,
+        parent_run_id: str | None,
+        code_location: str | None,
+        nominal_start_time: str,
+        nominal_end_time: str,
+        owners: list[str],
+        task: OperatorLineage | None,
+        run_facets: dict[str, type[BaseFacet]] | None = None,  # Custom run facets
+    ) -> str:
+        """
+        Emits openlineage event of type START
+        :param run_id: globally unique identifier of task in dag run
+        :param job_name: globally unique identifier of task in dag
+        :param job_description: user provided description of job
+        :param event_time:
+        :param parent_job_name: the name of the parent job (typically the DAG,
+                but possibly a task group)
+        :param parent_run_id: identifier of job spawning this task
+        :param code_location: file path or URL of DAG file
+        :param nominal_start_time: scheduled time of dag run
+        :param nominal_end_time: following schedule of dag run
+        :param owners: list of owners of DAG
+        :param task: metadata container with information extracted from operator
+        :param run_facets: custom run facets
+        :return:

Review Comment:
   Actually, removed return value - we don't need this now.



##########
airflow/providers/openlineage/plugins/facets.py:
##########
@@ -0,0 +1,116 @@
+# 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
+
+from attrs import define, field
+
+from airflow.providers.openlineage import version as OPENLINEAGE_AIRFLOW_VERSION
+from airflow.version import version as AIRFLOW_VERSION
+from openlineage.client.facet import BaseFacet
+from openlineage.client.utils import RedactMixin
+
+
+@define(slots=False)
+class AirflowVersionRunFacet(BaseFacet):
+    """Run facet containing task and DAG info"""
+
+    operator: str = field()
+    taskInfo: dict[str, object] = field()
+    airflowVersion: str = field()
+    openlineageAirflowVersion: str = field()
+
+    _additional_skip_redact: list[str] = [
+        "operator",
+        "airflowVersion",
+        "openlineageAirflowVersion",
+    ]
+
+    @classmethod
+    def from_dagrun_and_task(cls, dagrun, task):
+        # task.__dict__ may contain values uncastable to str
+        from airflow.providers.openlineage.utils import get_operator_class, to_json_encodable
+
+        task_info = to_json_encodable(task)
+        task_info["dag_run"] = to_json_encodable(dagrun)
+
+        return cls(
+            operator=f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",
+            taskInfo=task_info,
+            airflowVersion=AIRFLOW_VERSION,
+            openlineageAirflowVersion=OPENLINEAGE_AIRFLOW_VERSION,
+        )
+
+
+@define(slots=False)
+class AirflowRunArgsRunFacet(BaseFacet):
+    """Run facet pointing if DAG was triggered manually"""
+
+    externalTrigger: bool = field(default=False)
+
+    _additional_skip_redact: list[str] = ["externalTrigger"]
+
+
+@define(slots=False)
+class AirflowMappedTaskRunFacet(BaseFacet):
+    """Run facet containing information about mapped tasks"""
+
+    mapIndex: int = field()
+    operatorClass: str = field()
+
+    _additional_skip_redact: list[str] = ["operatorClass"]
+
+    @classmethod
+    def from_task_instance(cls, task_instance):
+        task = task_instance.task
+        from airflow.providers.openlineage.utils import get_operator_class
+
+        return cls(
+            mapIndex=task_instance.map_index,
+            operatorClass=f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",
+        )
+
+
+@define(slots=False)
+class AirflowRunFacet(BaseFacet):
+    """Composite Airflow run facet."""
+
+    dag: dict = field()
+    dagRun: dict = field()
+    task: dict = field()
+    taskInstance: dict = field()
+    taskUuid: str = field()

Review Comment:
   Removed. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] JDarDagran commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "JDarDagran (via GitHub)" <gi...@apache.org>.
JDarDagran commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1134027178


##########
airflow/providers/openlineage/plugins/macros.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.
+from __future__ import annotations
+
+import os
+import typing
+
+from airflow.providers.openlineage.plugins.adapter import OpenLineageAdapter
+
+if typing.TYPE_CHECKING:
+    from airflow.models import BaseOperator, TaskInstance
+
+_JOB_NAMESPACE = os.getenv("OPENLINEAGE_NAMESPACE", "default")
+
+
+def lineage_run_id(task: BaseOperator, task_instance: TaskInstance):
+    """
+    Macro function which returns the generated run id for a given task. This
+    can be used to forward the run id from a task to a child run so the job
+    hierarchy is preserved. Invoke as a jinja template, e.g.
+
+    PythonOperator(
+        task_id='render_template',
+        python_callable=my_task_function,
+        op_args=['{{ lineage_run_id(task, task_instance) }}'], # lineage_run_id macro invoked
+        provide_context=False,
+        dag=dag
+    )
+    """
+    return OpenLineageAdapter.build_task_instance_run_id(
+        task.task_id, task_instance.execution_date, task_instance.try_number
+    )
+
+
+def lineage_parent_id(run_id: str, task: BaseOperator, task_instance: TaskInstance):
+    """
+    Macro function which returns the generated job and run id for a given task. This
+    can be used to forward the ids from a task to a child run so the job
+    hierarchy is preserved. Child run can create ParentRunFacet from those ids.
+    Invoke as a jinja template, e.g.
+
+    PythonOperator(
+        task_id='render_template',
+        python_callable=my_task_function,
+        op_args=['{{ lineage_parent_id(run_id, task, task_instance) }}'], # macro invoked

Review Comment:
   That's in [Templates reference](https://airflow.apache.org/docs/apache-airflow/stable/templates-ref.html#variables)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] ashb commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "ashb (via GitHub)" <gi...@apache.org>.
ashb commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1133955748


##########
airflow/providers/openlineage/extractors/base.py:
##########
@@ -0,0 +1,133 @@
+# 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 json
+from abc import ABC, abstractmethod
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+from attrs import Factory, define
+
+from airflow.utils.log.logging_mixin import LoggingMixin
+from openlineage.client.facet import BaseFacet
+from openlineage.client.run import Dataset
+
+
+@define
+class OperatorLineage:
+    """Structure returned from lineage extraction."""
+
+    inputs: list[Dataset] = Factory(list)
+    outputs: list[Dataset] = Factory(list)
+    run_facets: dict[str, BaseFacet] = Factory(dict)
+    job_facets: dict[str, BaseFacet] = Factory(dict)
+
+
+class BaseExtractor(ABC, LoggingMixin):
+    """
+    Abstract base extractor class.
+
+    This is used mostly to maintain support for custom extractors.
+    """
+
+    _allowed_query_params: list[str] = []
+
+    def __init__(self, operator):
+        super().__init__()
+        self.operator = operator
+        self.patch()
+
+    def patch(self):
+        # Extractor should register extension methods or patches to operator here
+        pass
+
+    @classmethod
+    def get_operator_classnames(cls) -> list[str]:
+        """
+        Implement this method returning list of operators that extractor works for.
+        Particularly, in Airflow 2 some operators are deprecated and simply subclass the new
+        implementation, for example BigQueryOperator:
+        https://github.com/apache/airflow/blob/main/airflow/contrib/operators/bigquery_operator.py
+        The BigQueryExtractor needs to work with both of them.
+        :return:
+        """
+        raise NotImplementedError()
+
+    def validate(self):
+        assert self.operator.__class__.__name__ in self.get_operator_classnames()
+
+    @abstractmethod
+    def extract(self) -> OperatorLineage | None:
+        pass
+
+    def extract_on_complete(self, task_instance) -> OperatorLineage | None:
+        return self.extract()
+
+    @classmethod
+    def get_connection_uri(cls, conn):
+        """
+        Return the connection URI for the given ID. We first attempt to lookup
+        the connection URI via AIRFLOW_CONN_<conn_id>, else fallback on querying
+        the Airflow's connection table.

Review Comment:
   Firstly: This doc string doesn't really match the implementation.
   
   Secondly: How does this differ from just `conn.get_uri()` It's not clear from reading the function alone. Please add some comments saying why that isn't suitable (or is it?)



##########
airflow/providers/openlineage/extractors/base.py:
##########
@@ -0,0 +1,133 @@
+# 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 json
+from abc import ABC, abstractmethod
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+from attrs import Factory, define
+
+from airflow.utils.log.logging_mixin import LoggingMixin
+from openlineage.client.facet import BaseFacet
+from openlineage.client.run import Dataset
+
+
+@define
+class OperatorLineage:
+    """Structure returned from lineage extraction."""
+
+    inputs: list[Dataset] = Factory(list)
+    outputs: list[Dataset] = Factory(list)
+    run_facets: dict[str, BaseFacet] = Factory(dict)
+    job_facets: dict[str, BaseFacet] = Factory(dict)
+
+
+class BaseExtractor(ABC, LoggingMixin):
+    """
+    Abstract base extractor class.
+
+    This is used mostly to maintain support for custom extractors.
+    """
+
+    _allowed_query_params: list[str] = []
+
+    def __init__(self, operator):
+        super().__init__()
+        self.operator = operator
+        self.patch()
+
+    def patch(self):
+        # Extractor should register extension methods or patches to operator here
+        pass
+
+    @classmethod
+    def get_operator_classnames(cls) -> list[str]:
+        """
+        Implement this method returning list of operators that extractor works for.
+        Particularly, in Airflow 2 some operators are deprecated and simply subclass the new
+        implementation, for example BigQueryOperator:
+        https://github.com/apache/airflow/blob/main/airflow/contrib/operators/bigquery_operator.py
+        The BigQueryExtractor needs to work with both of them.
+        :return:
+        """
+        raise NotImplementedError()
+
+    def validate(self):
+        assert self.operator.__class__.__name__ in self.get_operator_classnames()

Review Comment:
   This would mess up with mapped operator I think
   
   ```suggestion
           assert self.operator.task_type in self.get_operator_classnames()
   ```



##########
airflow/providers/openlineage/extractors/extractors.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.
+
+from __future__ import annotations
+
+import os
+
+from airflow.providers.openlineage.extractors.base import BaseExtractor, DefaultExtractor
+
+
+class Extractors:
+    """
+    This exposes implemented extractors, while hiding ones that require additional, unmet
+    dependency. Patchers are a category of extractor that needs to hook up to operator's
+    internals during DAG creation.
+    """
+
+    def __init__(self):
+        # Do not expose extractors relying on external dependencies that are not installed
+        self.extractors = {}
+        self.default_extractor = DefaultExtractor
+
+        # Comma-separated extractors in OPENLINEAGE_EXTRACTORS variable.
+        # Extractors should implement BaseExtractor
+        from airflow.providers.openlineage.utils import import_from_string
+
+        env_extractors = os.getenv("OPENLINEAGE_EXTRACTORS")

Review Comment:
   Possibly this should now be `conf.get("openlineage", "extractors")` or similar now.



##########
airflow/providers/openlineage/plugins/adapter.py:
##########
@@ -0,0 +1,302 @@
+# 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
+import uuid
+from typing import TYPE_CHECKING
+
+import requests.exceptions
+
+from airflow.providers.openlineage import version as OPENLINEAGE_PROVIDER_VERSION
+from airflow.providers.openlineage.extractors import OperatorLineage
+from airflow.providers.openlineage.utils import redact_with_exclusions
+from airflow.utils.log.logging_mixin import LoggingMixin
+from openlineage.client import OpenLineageClient, set_producer
+from openlineage.client.facet import (
+    BaseFacet,
+    DocumentationJobFacet,
+    ErrorMessageRunFacet,
+    NominalTimeRunFacet,
+    OwnershipJobFacet,
+    OwnershipJobFacetOwners,
+    ParentRunFacet,
+    ProcessingEngineRunFacet,
+    SourceCodeLocationJobFacet,
+)
+from openlineage.client.run import Job, Run, RunEvent, RunState
+
+if TYPE_CHECKING:
+    from airflow.models.dagrun import DagRun
+
+
+_DAG_DEFAULT_NAMESPACE = "default"
+
+_DAG_NAMESPACE = os.getenv("OPENLINEAGE_NAMESPACE", _DAG_DEFAULT_NAMESPACE)
+
+_PRODUCER = f"https://github.com/apache/airflow/tree/providers-openlineage/" f"{OPENLINEAGE_PROVIDER_VERSION}"
+
+set_producer(_PRODUCER)
+
+
+class OpenLineageAdapter(LoggingMixin):
+    """
+    Adapter for translating Airflow metadata to OpenLineage events,
+    instead of directly creating them from Airflow code.
+    """
+
+    def __init__(self, client=None):
+        super().__init__()
+        self._client = client
+
+    def get_or_create_openlineage_client(self) -> OpenLineageClient:
+        if not self._client:
+            self._client = OpenLineageClient.from_environment()
+        return self._client
+
+    def build_dag_run_id(self, dag_id, dag_run_id):
+        return str(uuid.uuid3(uuid.NAMESPACE_URL, f"{_DAG_NAMESPACE}.{dag_id}.{dag_run_id}"))
+
+    @staticmethod
+    def build_task_instance_run_id(task_id, execution_date, try_number):
+        return str(
+            uuid.uuid3(
+                uuid.NAMESPACE_URL,
+                f"{_DAG_NAMESPACE}.{task_id}.{execution_date}.{try_number}",
+            )
+        )
+
+    def emit(self, event: RunEvent):
+        event = redact_with_exclusions(event)
+        try:
+            return self.get_or_create_openlineage_client().emit(event)
+        except requests.exceptions.RequestException:
+            self.log.exception(f"Failed to emit OpenLineage event of id {event.run.runId}")
+
+    def start_task(
+        self,
+        run_id: str,
+        job_name: str,
+        job_description: str,
+        event_time: str,
+        parent_job_name: str | None,
+        parent_run_id: str | None,
+        code_location: str | None,
+        nominal_start_time: str,
+        nominal_end_time: str,
+        owners: list[str],
+        task: OperatorLineage | None,
+        run_facets: dict[str, type[BaseFacet]] | None = None,  # Custom run facets
+    ) -> str:
+        """
+        Emits openlineage event of type START

Review Comment:
   ```suggestion
           Emits openlineage event of type START
           
   ```



##########
airflow/providers/openlineage/plugins/facets.py:
##########
@@ -0,0 +1,116 @@
+# 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
+
+from attrs import define, field
+
+from airflow.providers.openlineage import version as OPENLINEAGE_AIRFLOW_VERSION
+from airflow.version import version as AIRFLOW_VERSION
+from openlineage.client.facet import BaseFacet
+from openlineage.client.utils import RedactMixin
+
+
+@define(slots=False)
+class AirflowVersionRunFacet(BaseFacet):
+    """Run facet containing task and DAG info"""
+
+    operator: str = field()
+    taskInfo: dict[str, object] = field()
+    airflowVersion: str = field()
+    openlineageAirflowVersion: str = field()
+
+    _additional_skip_redact: list[str] = [
+        "operator",
+        "airflowVersion",
+        "openlineageAirflowVersion",
+    ]
+
+    @classmethod
+    def from_dagrun_and_task(cls, dagrun, task):
+        # task.__dict__ may contain values uncastable to str
+        from airflow.providers.openlineage.utils import get_operator_class, to_json_encodable
+
+        task_info = to_json_encodable(task)
+        task_info["dag_run"] = to_json_encodable(dagrun)
+
+        return cls(
+            operator=f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",
+            taskInfo=task_info,
+            airflowVersion=AIRFLOW_VERSION,
+            openlineageAirflowVersion=OPENLINEAGE_AIRFLOW_VERSION,
+        )
+
+
+@define(slots=False)
+class AirflowRunArgsRunFacet(BaseFacet):
+    """Run facet pointing if DAG was triggered manually"""
+
+    externalTrigger: bool = field(default=False)
+
+    _additional_skip_redact: list[str] = ["externalTrigger"]
+
+
+@define(slots=False)
+class AirflowMappedTaskRunFacet(BaseFacet):
+    """Run facet containing information about mapped tasks"""
+
+    mapIndex: int = field()
+    operatorClass: str = field()
+
+    _additional_skip_redact: list[str] = ["operatorClass"]
+
+    @classmethod
+    def from_task_instance(cls, task_instance):
+        task = task_instance.task
+        from airflow.providers.openlineage.utils import get_operator_class
+
+        return cls(
+            mapIndex=task_instance.map_index,
+            operatorClass=f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",
+        )
+
+
+@define(slots=False)
+class AirflowRunFacet(BaseFacet):
+    """Composite Airflow run facet."""
+
+    dag: dict = field()
+    dagRun: dict = field()
+    task: dict = field()
+    taskInstance: dict = field()
+    taskUuid: str = field()

Review Comment:
   `= field()` shouldn't be needed here
   
   ```suggestion
       dag: dict
       dagRun: dict
       task: dict
       taskInstance: dict
       taskUuid: str
   ```



##########
airflow/providers/openlineage/plugins/listener.py:
##########
@@ -0,0 +1,189 @@
+# 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 logging
+from concurrent.futures import Executor, ThreadPoolExecutor
+from typing import TYPE_CHECKING
+
+from airflow.listeners import hookimpl
+from airflow.providers.openlineage.extractors import ExtractorManager
+from airflow.providers.openlineage.plugins.adapter import OpenLineageAdapter
+from airflow.providers.openlineage.utils import (
+    get_airflow_run_facet,
+    get_custom_facets,
+    get_job_name,
+    get_task_location,
+    print_exception,
+)
+
+if TYPE_CHECKING:
+    from sqlalchemy.orm import Session
+
+    from airflow.models import DagRun, TaskInstance
+
+
+class OpenLineageListener:
+    """
+    OpenLineage listener
+    Sends events on task instance and dag run starts, completes and failures.
+    """
+
+    def __init__(self):
+        self.log = logging.getLogger(__name__)
+        self.executor: Executor = None  # type: ignore
+        self.extractor_manager = ExtractorManager()
+        self.adapter = OpenLineageAdapter()
+
+    @hookimpl
+    def on_task_instance_running(
+        self, previous_state, task_instance: TaskInstance, session: Session  # This will always be QUEUED
+    ):
+        if not hasattr(task_instance, "task"):
+            self.log.warning(
+                f"No task set for TI object task_id: {task_instance.task_id} - "
+                f"dag_id: {task_instance.dag_id} - run_id {task_instance.run_id}"
+            )
+            return
+
+        self.log.debug("OpenLineage listener got notification about task instance start")
+        dagrun = task_instance.dag_run
+        task = task_instance.task
+        dag = task.dag
+
+        @print_exception
+        def on_running():
+            # that's a workaround to detect task running from deferred state
+            # we return here because Airflow 2.3 needs task from deferred state
+            if task_instance.next_method is not None:
+                return
+            parent_run_id = self.adapter.build_dag_run_id(dag.dag_id, dagrun.run_id)
+
+            task_uuid = self.adapter.build_task_instance_run_id(
+                task.task_id, task_instance.execution_date, task_instance.try_number
+            )
+
+            task_metadata = self.extractor_manager.extract_metadata(dagrun, task)
+
+            self.adapter.start_task(
+                run_id=task_uuid,
+                job_name=get_job_name(task),
+                job_description=dag.description,
+                event_time=task_instance.start_date.isoformat(),
+                parent_job_name=dag.dag_id,
+                parent_run_id=parent_run_id,
+                code_location=get_task_location(task),
+                nominal_start_time=dagrun.data_interval_start.isoformat(),
+                nominal_end_time=dagrun.data_interval_end.isoformat(),
+                owners=dag.owner.split(", "),
+                task=task_metadata,
+                run_facets={
+                    **task_metadata.run_facets,
+                    **get_custom_facets(dagrun, task, dagrun.external_trigger, task_instance),
+                    **get_airflow_run_facet(dagrun, dag, task_instance, task, task_uuid),
+                },
+            )
+
+        self.executor.submit(on_running)
+
+    @hookimpl
+    def on_task_instance_success(self, previous_state, task_instance: TaskInstance, session):
+        self.log.debug("OpenLineage listener got notification about task instance success")
+
+        dagrun = task_instance.dag_run
+        task = task_instance.task
+
+        task_uuid = OpenLineageAdapter.build_task_instance_run_id(
+            task.task_id, task_instance.execution_date, task_instance.try_number - 1
+        )
+
+        @print_exception
+        def on_success():
+            task_metadata = self.extractor_manager.extract_metadata(
+                dagrun, task, complete=True, task_instance=task_instance
+            )
+            self.adapter.complete_task(
+                run_id=task_uuid,
+                job_name=get_job_name(task),
+                end_time=task_instance.end_date.isoformat(),
+                task=task_metadata,
+            )
+
+        self.executor.submit(on_success)
+
+    @hookimpl
+    def on_task_instance_failed(self, previous_state, task_instance: TaskInstance, session):
+        self.log.debug("OpenLineage listener got notification about task instance failure")
+
+        dagrun = task_instance.dag_run
+        task = task_instance.task
+
+        task_uuid = OpenLineageAdapter.build_task_instance_run_id(
+            task.task_id, task_instance.execution_date, task_instance.try_number - 1
+        )
+
+        @print_exception
+        def on_failure():
+            task_metadata = self.extractor_manager.extract_metadata(
+                dagrun, task, complete=True, task_instance=task_instance
+            )
+
+            self.adapter.fail_task(
+                run_id=task_uuid,
+                job_name=get_job_name(task),
+                end_time=task_instance.end_date.isoformat(),
+                task=task_metadata,
+            )
+
+        self.executor.submit(on_failure)
+
+    @hookimpl
+    def on_starting(self, component):
+        self.log.debug("on_starting: %s", component.__class__.__name__)
+        self.executor = ThreadPoolExecutor(max_workers=8, thread_name_prefix="openlineage_")

Review Comment:
   That's a lot of max workers. Does it ever need more than 1?



##########
airflow/providers/openlineage/plugins/facets.py:
##########
@@ -0,0 +1,116 @@
+# 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
+
+from attrs import define, field
+
+from airflow.providers.openlineage import version as OPENLINEAGE_AIRFLOW_VERSION
+from airflow.version import version as AIRFLOW_VERSION
+from openlineage.client.facet import BaseFacet
+from openlineage.client.utils import RedactMixin
+
+
+@define(slots=False)
+class AirflowVersionRunFacet(BaseFacet):
+    """Run facet containing task and DAG info"""
+
+    operator: str = field()
+    taskInfo: dict[str, object] = field()
+    airflowVersion: str = field()
+    openlineageAirflowVersion: str = field()
+
+    _additional_skip_redact: list[str] = [
+        "operator",
+        "airflowVersion",
+        "openlineageAirflowVersion",
+    ]
+
+    @classmethod
+    def from_dagrun_and_task(cls, dagrun, task):
+        # task.__dict__ may contain values uncastable to str
+        from airflow.providers.openlineage.utils import get_operator_class, to_json_encodable
+
+        task_info = to_json_encodable(task)
+        task_info["dag_run"] = to_json_encodable(dagrun)
+
+        return cls(
+            operator=f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",
+            taskInfo=task_info,
+            airflowVersion=AIRFLOW_VERSION,
+            openlineageAirflowVersion=OPENLINEAGE_AIRFLOW_VERSION,
+        )
+
+
+@define(slots=False)
+class AirflowRunArgsRunFacet(BaseFacet):
+    """Run facet pointing if DAG was triggered manually"""
+
+    externalTrigger: bool = field(default=False)
+
+    _additional_skip_redact: list[str] = ["externalTrigger"]
+
+
+@define(slots=False)
+class AirflowMappedTaskRunFacet(BaseFacet):
+    """Run facet containing information about mapped tasks"""
+
+    mapIndex: int = field()
+    operatorClass: str = field()

Review Comment:
   ```suggestion
       mapIndex: int
       operatorClass: str



##########
airflow/providers/openlineage/plugins/macros.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.
+from __future__ import annotations
+
+import os
+import typing
+
+from airflow.providers.openlineage.plugins.adapter import OpenLineageAdapter
+
+if typing.TYPE_CHECKING:
+    from airflow.models import BaseOperator, TaskInstance
+
+_JOB_NAMESPACE = os.getenv("OPENLINEAGE_NAMESPACE", "default")
+
+
+def lineage_run_id(task: BaseOperator, task_instance: TaskInstance):
+    """
+    Macro function which returns the generated run id for a given task. This
+    can be used to forward the run id from a task to a child run so the job
+    hierarchy is preserved. Invoke as a jinja template, e.g.
+
+    PythonOperator(
+        task_id='render_template',
+        python_callable=my_task_function,
+        op_args=['{{ lineage_run_id(task, task_instance) }}'], # lineage_run_id macro invoked

Review Comment:
   Do you need to pass task and ti? Couldn't you look at `ti.task`?



##########
airflow/providers/openlineage/utils/__init__.py:
##########
@@ -0,0 +1,499 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin

Review Comment:
   Yes please!



##########
airflow/providers/openlineage/utils/__init__.py:
##########
@@ -0,0 +1,499 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def get_connection_uri(conn):

Review Comment:
   Second copy of this function in this PR :)



##########
airflow/providers/openlineage/plugins/adapter.py:
##########
@@ -0,0 +1,302 @@
+# 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
+import uuid
+from typing import TYPE_CHECKING
+
+import requests.exceptions
+
+from airflow.providers.openlineage import version as OPENLINEAGE_PROVIDER_VERSION
+from airflow.providers.openlineage.extractors import OperatorLineage
+from airflow.providers.openlineage.utils import redact_with_exclusions
+from airflow.utils.log.logging_mixin import LoggingMixin
+from openlineage.client import OpenLineageClient, set_producer
+from openlineage.client.facet import (
+    BaseFacet,
+    DocumentationJobFacet,
+    ErrorMessageRunFacet,
+    NominalTimeRunFacet,
+    OwnershipJobFacet,
+    OwnershipJobFacetOwners,
+    ParentRunFacet,
+    ProcessingEngineRunFacet,
+    SourceCodeLocationJobFacet,
+)
+from openlineage.client.run import Job, Run, RunEvent, RunState
+
+if TYPE_CHECKING:
+    from airflow.models.dagrun import DagRun
+
+
+_DAG_DEFAULT_NAMESPACE = "default"
+
+_DAG_NAMESPACE = os.getenv("OPENLINEAGE_NAMESPACE", _DAG_DEFAULT_NAMESPACE)
+
+_PRODUCER = f"https://github.com/apache/airflow/tree/providers-openlineage/" f"{OPENLINEAGE_PROVIDER_VERSION}"
+
+set_producer(_PRODUCER)
+
+
+class OpenLineageAdapter(LoggingMixin):
+    """
+    Adapter for translating Airflow metadata to OpenLineage events,
+    instead of directly creating them from Airflow code.
+    """
+
+    def __init__(self, client=None):
+        super().__init__()
+        self._client = client
+
+    def get_or_create_openlineage_client(self) -> OpenLineageClient:
+        if not self._client:
+            self._client = OpenLineageClient.from_environment()
+        return self._client
+
+    def build_dag_run_id(self, dag_id, dag_run_id):
+        return str(uuid.uuid3(uuid.NAMESPACE_URL, f"{_DAG_NAMESPACE}.{dag_id}.{dag_run_id}"))
+
+    @staticmethod
+    def build_task_instance_run_id(task_id, execution_date, try_number):
+        return str(
+            uuid.uuid3(
+                uuid.NAMESPACE_URL,
+                f"{_DAG_NAMESPACE}.{task_id}.{execution_date}.{try_number}",
+            )
+        )
+
+    def emit(self, event: RunEvent):
+        event = redact_with_exclusions(event)
+        try:
+            return self.get_or_create_openlineage_client().emit(event)
+        except requests.exceptions.RequestException:
+            self.log.exception(f"Failed to emit OpenLineage event of id {event.run.runId}")
+
+    def start_task(
+        self,
+        run_id: str,
+        job_name: str,
+        job_description: str,
+        event_time: str,
+        parent_job_name: str | None,
+        parent_run_id: str | None,
+        code_location: str | None,
+        nominal_start_time: str,
+        nominal_end_time: str,
+        owners: list[str],
+        task: OperatorLineage | None,
+        run_facets: dict[str, type[BaseFacet]] | None = None,  # Custom run facets
+    ) -> str:
+        """
+        Emits openlineage event of type START
+        :param run_id: globally unique identifier of task in dag run
+        :param job_name: globally unique identifier of task in dag
+        :param job_description: user provided description of job
+        :param event_time:
+        :param parent_job_name: the name of the parent job (typically the DAG,
+                but possibly a task group)
+        :param parent_run_id: identifier of job spawning this task
+        :param code_location: file path or URL of DAG file
+        :param nominal_start_time: scheduled time of dag run
+        :param nominal_end_time: following schedule of dag run
+        :param owners: list of owners of DAG
+        :param task: metadata container with information extracted from operator
+        :param run_facets: custom run facets
+        :return:

Review Comment:
   ```suggestion
           :return: OpenLineage event run ID
   ```
   
   (I guessed)



##########
airflow/providers/openlineage/plugins/adapter.py:
##########
@@ -0,0 +1,302 @@
+# 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
+import uuid
+from typing import TYPE_CHECKING
+
+import requests.exceptions
+
+from airflow.providers.openlineage import version as OPENLINEAGE_PROVIDER_VERSION
+from airflow.providers.openlineage.extractors import OperatorLineage
+from airflow.providers.openlineage.utils import redact_with_exclusions
+from airflow.utils.log.logging_mixin import LoggingMixin
+from openlineage.client import OpenLineageClient, set_producer
+from openlineage.client.facet import (
+    BaseFacet,
+    DocumentationJobFacet,
+    ErrorMessageRunFacet,
+    NominalTimeRunFacet,
+    OwnershipJobFacet,
+    OwnershipJobFacetOwners,
+    ParentRunFacet,
+    ProcessingEngineRunFacet,
+    SourceCodeLocationJobFacet,
+)
+from openlineage.client.run import Job, Run, RunEvent, RunState
+
+if TYPE_CHECKING:
+    from airflow.models.dagrun import DagRun
+
+
+_DAG_DEFAULT_NAMESPACE = "default"
+
+_DAG_NAMESPACE = os.getenv("OPENLINEAGE_NAMESPACE", _DAG_DEFAULT_NAMESPACE)

Review Comment:
   If you want to make it easier to upgrade you can do `conf.get("openlineage", "namespace", fallback=os.getenv("OPENLINEAGE_NAMESPACE", _DAG_DEFAULT_NAMESPACE))`



##########
airflow/providers/openlineage/utils/__init__.py:
##########
@@ -0,0 +1,499 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def get_connection_uri(conn):
+    """
+    Return the connection URI for the given ID. We first attempt to lookup
+    the connection URI via AIRFLOW_CONN_<conn_id>, else fallback on querying
+    the Airflow's connection table.
+    """
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not _filtered_query_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def _filtered_query_params(k: str):
+    unfiltered_snowflake_keys = [
+        "extra__snowflake__warehouse",
+        "extra__snowflake__account",
+        "extra__snowflake__database",
+    ]
+    filtered_key_substrings = [
+        "aws_access_key_id",
+        "aws_secret_access_key",
+        "extra__snowflake__",
+    ]
+    return k not in unfiltered_snowflake_keys and any(substr in k for substr in filtered_key_substrings)
+
+
+def get_normalized_postgres_connection_uri(conn):
+    """
+    URIs starting with postgresql:// and postgres:// are both valid
+    PostgreSQL connection strings. This function normalizes it to
+    postgres:// as canonical name according to OpenLineage spec.
+    """
+    uri = get_connection_uri(conn)
+    if uri.startswith("postgresql"):
+        uri = uri.replace("postgresql", "postgres", 1)
+    return uri
+
+
+def get_connection(conn_id) -> Connection | None:
+    from airflow.hooks.base import BaseHook
+
+    try:
+        return BaseHook.get_connection(conn_id=conn_id)
+    except Exception:
+        return None
+
+
+def get_job_name(task):
+    return f"{task.dag_id}.{task.task_id}"
+
+
+def get_custom_facets(
+    dagrun, task, is_external_trigger: bool, task_instance: TaskInstance | None = None
+) -> dict[str, Any]:
+    custom_facets = {
+        "airflow_runArgs": AirflowRunArgsRunFacet(is_external_trigger),
+        "airflow_version": AirflowVersionRunFacet.from_dagrun_and_task(dagrun, task),
+    }
+    # check for -1 comes from SmartSensor compatibility with dynamic task mapping
+    # this comes from Airflow code
+    if hasattr(task_instance, "map_index") and getattr(task_instance, "map_index") != -1:
+        custom_facets["airflow_mappedTask"] = AirflowMappedTaskRunFacet.from_task_instance(task_instance)
+    return custom_facets
+
+
+class InfoJsonEncodable(dict):
+    """
+    Airflow objects might not be json-encodable overall.
+
+    The class provides additional attributes to control
+    what and how is encoded:
+    * renames: a dictionary of attribute name changes
+    * casts: a dictionary consisting of attribute names
+             and corresponding methods that should change
+             object value
+    * includes: list of attributes to be included in encoding
+    * excludes: list of attributes to be excluded from encoding
+
+    Don't use both includes and excludes.
+    """
+
+    renames: dict[str, str] = {}
+    casts: dict[str, Any] = {}
+    includes: list[str] = []
+    excludes: list[str] = []
+
+    def __init__(self, obj):
+        self.obj = obj
+        self._fields = []
+
+        self._cast_fields()
+        self._rename_fields()
+        self._include_fields()
+        dict.__init__(
+            self,
+            **{field: InfoJsonEncodable._cast_basic_types(getattr(self, field)) for field in self._fields},
+        )
+
+    @staticmethod
+    def _cast_basic_types(value):
+        if isinstance(value, datetime.datetime):
+            return value.isoformat()
+        if isinstance(value, (set, list, tuple)):
+            return str(list(value))
+        return value
+
+    def _rename_fields(self):
+        for field, renamed in self.renames.items():
+            if hasattr(self.obj, field):
+                setattr(self, renamed, getattr(self.obj, field))
+                self._fields.append(renamed)
+
+    def _cast_fields(self):
+        for field, func in self.casts.items():
+            setattr(self, field, func(self.obj))
+            self._fields.append(field)
+
+    def _include_fields(self):
+        if self.includes and self.excludes:
+            raise Exception("Don't use both includes and excludes.")
+        if self.includes:
+            for field in self.includes:
+                if field in self._fields or not hasattr(self.obj, field):
+                    continue
+                setattr(self, field, getattr(self.obj, field))
+                self._fields.append(field)
+        else:
+            for field, val in self.obj.__dict__.items():
+                if field in self._fields or field in self.excludes or field in self.renames:
+                    continue
+                setattr(self, field, val)
+                self._fields.append(field)
+
+
+class DagInfo(InfoJsonEncodable):
+    """Defines encoding DAG object to JSON."""
+
+    includes = ["dag_id", "schedule_interval", "tags", "start_date"]
+    casts = {"timetable": lambda dag: dag.timetable.serialize() if getattr(dag, "timetable", None) else None}
+    renames = {"_dag_id": "dag_id"}
+
+
+class DagRunInfo(InfoJsonEncodable):
+    """Defines encoding DagRun object to JSON."""
+
+    includes = [
+        "conf",
+        "dag_id",
+        "data_interval_start",
+        "data_interval_end",
+        "external_trigger",
+        "run_id",
+        "run_type",
+        "start_date",
+    ]
+
+
+class TaskInstanceInfo(InfoJsonEncodable):
+    """Defines encoding TaskInstance object to JSON."""
+
+    includes = ["duration", "try_number", "pool"]
+    casts = {
+        "map_index": lambda ti: ti.map_index
+        if hasattr(ti, "map_index") and getattr(ti, "map_index") != -1
+        else None
+    }
+
+
+class TaskInfo(InfoJsonEncodable):
+    """Defines encoding BaseOperator/AbstractOperator object to JSON."""
+
+    renames = {
+        "_BaseOperator__init_kwargs": "args",
+        "_BaseOperator__from_mapped": "mapped",
+        "_downstream_task_ids": "downstream_task_ids",
+        "_upstream_task_ids": "upstream_task_ids",
+    }
+    excludes = [
+        "_BaseOperator__instantiated",
+        "_dag",
+        "_hook",
+        "_log",
+        "_outlets",
+        "_inlets",
+        "_lock_for_execution",
+        "handler",
+        "params",
+        "python_callable",
+        "retry_delay",
+    ]
+    casts = {
+        "operator_class": lambda task: f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",  # noqa
+        "task_group": lambda task: TaskGroupInfo(task.task_group)
+        if hasattr(task, "task_group") and getattr(task.task_group, "_group_id", None)
+        else None,
+    }
+
+
+class TaskGroupInfo(InfoJsonEncodable):
+    """Defines encoding TaskGroup object to JSON."""
+
+    renames = {
+        "_group_id": "group_id",
+    }
+    includes = [
+        "downstream_group_ids",
+        "downstream_task_ids",
+        "prefix_group_id",
+        "tooltip",
+        "upstream_group_ids",
+        "upstream_task_ids",
+    ]
+
+
+def get_airflow_run_facet(
+    dag_run: DagRun,
+    dag: DAG,
+    task_instance: TaskInstance,
+    task: BaseOperator,
+    task_uuid: str,
+):
+    return {
+        "airflow": json.loads(
+            json.dumps(
+                asdict(
+                    AirflowRunFacet(
+                        dag=DagInfo(dag),
+                        dagRun=DagRunInfo(dag_run),
+                        taskInstance=TaskInstanceInfo(task_instance),
+                        task=TaskInfo(task),
+                        taskUuid=task_uuid,
+                    )
+                ),
+                default=str,
+            )
+        )
+    }
+
+
+def get_dagrun_start_end(dagrun: DagRun, dag: DAG):
+    try:
+        return dagrun.data_interval_start, dagrun.data_interval_end
+    except AttributeError:
+        # Airflow < 2.2 before adding data interval
+        pass
+    start = dagrun.execution_date
+    end = dag.following_schedule(start)
+    return start, end or start
+
+
+def import_from_string(path: str):
+    try:
+        module_path, target = path.rsplit(".", 1)
+        module = importlib.import_module(module_path)
+        return getattr(module, target)
+    except Exception as e:
+        log.warning(e)
+        raise ImportError(f"Failed to import {path}") from e
+
+
+def try_import_from_string(path: str):
+    try:
+        return import_from_string(path)
+    except ImportError:
+        return None
+
+
+def redact_with_exclusions(source: Any):

Review Comment:
   A lot of this looks like a reimplementation of the existing masking code. How is it different?



##########
airflow/providers/openlineage/utils/__init__.py:
##########
@@ -0,0 +1,499 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin

Review Comment:
   It looks like you already are in `redact_with_exclusions`?



##########
airflow/providers/openlineage/extractors/base.py:
##########
@@ -0,0 +1,133 @@
+# 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 json
+from abc import ABC, abstractmethod
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+from attrs import Factory, define
+
+from airflow.utils.log.logging_mixin import LoggingMixin
+from openlineage.client.facet import BaseFacet
+from openlineage.client.run import Dataset
+
+
+@define
+class OperatorLineage:
+    """Structure returned from lineage extraction."""
+
+    inputs: list[Dataset] = Factory(list)
+    outputs: list[Dataset] = Factory(list)
+    run_facets: dict[str, BaseFacet] = Factory(dict)
+    job_facets: dict[str, BaseFacet] = Factory(dict)
+
+
+class BaseExtractor(ABC, LoggingMixin):
+    """
+    Abstract base extractor class.
+
+    This is used mostly to maintain support for custom extractors.
+    """
+
+    _allowed_query_params: list[str] = []
+
+    def __init__(self, operator):
+        super().__init__()
+        self.operator = operator
+        self.patch()
+
+    def patch(self):
+        # Extractor should register extension methods or patches to operator here
+        pass
+
+    @classmethod

Review Comment:
   ```suggestion
       @abstractclassmethod
   ```
   
   I think?



##########
airflow/providers/openlineage/extractors/extractors.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.
+
+from __future__ import annotations
+
+import os
+
+from airflow.providers.openlineage.extractors.base import BaseExtractor, DefaultExtractor
+
+
+class Extractors:
+    """
+    This exposes implemented extractors, while hiding ones that require additional, unmet
+    dependency. Patchers are a category of extractor that needs to hook up to operator's
+    internals during DAG creation.
+    """
+
+    def __init__(self):
+        # Do not expose extractors relying on external dependencies that are not installed
+        self.extractors = {}
+        self.default_extractor = DefaultExtractor
+
+        # Comma-separated extractors in OPENLINEAGE_EXTRACTORS variable.
+        # Extractors should implement BaseExtractor
+        from airflow.providers.openlineage.utils import import_from_string
+
+        env_extractors = os.getenv("OPENLINEAGE_EXTRACTORS")
+        if env_extractors is not None:
+            for extractor in env_extractors.split(";"):

Review Comment:
   Comma more often used for this purpose in Airflow



##########
airflow/providers/openlineage/plugins/facets.py:
##########
@@ -0,0 +1,116 @@
+# 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
+
+from attrs import define, field
+
+from airflow.providers.openlineage import version as OPENLINEAGE_AIRFLOW_VERSION
+from airflow.version import version as AIRFLOW_VERSION
+from openlineage.client.facet import BaseFacet
+from openlineage.client.utils import RedactMixin
+
+
+@define(slots=False)
+class AirflowVersionRunFacet(BaseFacet):
+    """Run facet containing task and DAG info"""
+
+    operator: str = field()
+    taskInfo: dict[str, object] = field()
+    airflowVersion: str = field()
+    openlineageAirflowVersion: str = field()
+
+    _additional_skip_redact: list[str] = [
+        "operator",
+        "airflowVersion",
+        "openlineageAirflowVersion",
+    ]
+
+    @classmethod
+    def from_dagrun_and_task(cls, dagrun, task):
+        # task.__dict__ may contain values uncastable to str
+        from airflow.providers.openlineage.utils import get_operator_class, to_json_encodable
+
+        task_info = to_json_encodable(task)
+        task_info["dag_run"] = to_json_encodable(dagrun)
+
+        return cls(
+            operator=f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",
+            taskInfo=task_info,
+            airflowVersion=AIRFLOW_VERSION,
+            openlineageAirflowVersion=OPENLINEAGE_AIRFLOW_VERSION,
+        )
+
+
+@define(slots=False)
+class AirflowRunArgsRunFacet(BaseFacet):
+    """Run facet pointing if DAG was triggered manually"""
+
+    externalTrigger: bool = field(default=False)
+
+    _additional_skip_redact: list[str] = ["externalTrigger"]
+
+
+@define(slots=False)
+class AirflowMappedTaskRunFacet(BaseFacet):
+    """Run facet containing information about mapped tasks"""
+
+    mapIndex: int = field()
+    operatorClass: str = field()
+
+    _additional_skip_redact: list[str] = ["operatorClass"]
+
+    @classmethod
+    def from_task_instance(cls, task_instance):
+        task = task_instance.task
+        from airflow.providers.openlineage.utils import get_operator_class
+
+        return cls(
+            mapIndex=task_instance.map_index,
+            operatorClass=f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",
+        )
+
+
+@define(slots=False)
+class AirflowRunFacet(BaseFacet):
+    """Composite Airflow run facet."""
+
+    dag: dict = field()
+    dagRun: dict = field()
+    task: dict = field()
+    taskInstance: dict = field()
+    taskUuid: str = field()
+
+
+@define(slots=False)
+class UnknownOperatorInstance(RedactMixin):
+    """
+    Describes an unknown operator - specifies the (class) name of the operator
+    and its properties
+    """
+
+    name: str = field()
+    properties: dict[str, object] = field()
+    type: str = field(default="operator")

Review Comment:
   ```suggestion
       name: str
       properties: dict[str, object]
       type: str = "operator"
   ```
   
   Same effect



##########
airflow/providers/openlineage/plugins/facets.py:
##########
@@ -0,0 +1,116 @@
+# 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
+
+from attrs import define, field
+
+from airflow.providers.openlineage import version as OPENLINEAGE_AIRFLOW_VERSION
+from airflow.version import version as AIRFLOW_VERSION
+from openlineage.client.facet import BaseFacet
+from openlineage.client.utils import RedactMixin
+
+
+@define(slots=False)
+class AirflowVersionRunFacet(BaseFacet):
+    """Run facet containing task and DAG info"""
+
+    operator: str = field()
+    taskInfo: dict[str, object] = field()
+    airflowVersion: str = field()
+    openlineageAirflowVersion: str = field()
+
+    _additional_skip_redact: list[str] = [
+        "operator",
+        "airflowVersion",
+        "openlineageAirflowVersion",
+    ]
+
+    @classmethod
+    def from_dagrun_and_task(cls, dagrun, task):
+        # task.__dict__ may contain values uncastable to str
+        from airflow.providers.openlineage.utils import get_operator_class, to_json_encodable
+
+        task_info = to_json_encodable(task)
+        task_info["dag_run"] = to_json_encodable(dagrun)
+
+        return cls(
+            operator=f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",
+            taskInfo=task_info,
+            airflowVersion=AIRFLOW_VERSION,
+            openlineageAirflowVersion=OPENLINEAGE_AIRFLOW_VERSION,
+        )
+
+
+@define(slots=False)
+class AirflowRunArgsRunFacet(BaseFacet):
+    """Run facet pointing if DAG was triggered manually"""
+
+    externalTrigger: bool = field(default=False)

Review Comment:
   ```suggestion
       externalTrigger: bool = False
   ```



##########
airflow/providers/openlineage/plugins/macros.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.
+from __future__ import annotations
+
+import os
+import typing
+
+from airflow.providers.openlineage.plugins.adapter import OpenLineageAdapter
+
+if typing.TYPE_CHECKING:
+    from airflow.models import BaseOperator, TaskInstance
+
+_JOB_NAMESPACE = os.getenv("OPENLINEAGE_NAMESPACE", "default")
+
+
+def lineage_run_id(task: BaseOperator, task_instance: TaskInstance):
+    """
+    Macro function which returns the generated run id for a given task. This
+    can be used to forward the run id from a task to a child run so the job
+    hierarchy is preserved. Invoke as a jinja template, e.g.
+
+    PythonOperator(
+        task_id='render_template',
+        python_callable=my_task_function,
+        op_args=['{{ lineage_run_id(task, task_instance) }}'], # lineage_run_id macro invoked
+        provide_context=False,
+        dag=dag
+    )
+    """
+    return OpenLineageAdapter.build_task_instance_run_id(
+        task.task_id, task_instance.execution_date, task_instance.try_number
+    )
+
+
+def lineage_parent_id(run_id: str, task: BaseOperator, task_instance: TaskInstance):
+    """
+    Macro function which returns the generated job and run id for a given task. This
+    can be used to forward the ids from a task to a child run so the job
+    hierarchy is preserved. Child run can create ParentRunFacet from those ids.
+    Invoke as a jinja template, e.g.
+
+    PythonOperator(
+        task_id='render_template',
+        python_callable=my_task_function,
+        op_args=['{{ lineage_parent_id(run_id, task, task_instance) }}'], # macro invoked

Review Comment:
   Where does `run_id` come from? That isn't a key in the context (at least not to my memory)



##########
airflow/providers/openlineage/plugins/openlineage.py:
##########
@@ -0,0 +1,46 @@
+# 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 airflow.plugins_manager import AirflowPlugin
+from airflow.providers.openlineage.plugins.macros import lineage_parent_id, lineage_run_id
+
+
+def _is_disabled():
+    return os.getenv("OPENLINEAGE_DISABLED", None) in [True, "true", "True"]
+
+
+if _is_disabled():  # type: ignore
+    # Provide empty plugin when OL is disabled
+    class OpenLineageProviderPlugin(AirflowPlugin):
+        """OpenLineage plugin that provides macros only"""
+
+        name = "OpenLineageProviderPlugin"
+        macros = [lineage_run_id, lineage_parent_id]
+
+else:
+    from airflow.providers.openlineage.plugins.listener import OpenLineageListener
+
+    # Provide entrypoint airflow plugin that registers listener module
+    class OpenLineageProviderPlugin(AirflowPlugin):  # type: ignore
+        """OpenLineage plugin that provides listener module and macros"""
+
+        name = "OpenLineageProviderPlugin"
+        listeners = [OpenLineageListener()]
+        macros = [lineage_run_id, lineage_parent_id]

Review Comment:
   ```suggestion
   
   class OpenLineageProviderPlugin(AirflowPlugin):
   
       name = "OpenLineageProviderPlugin"
       macros = [lineage_run_id, lineage_parent_id]
       if _is_disabled():
           from airflow.providers.openlineage.plugins.listener import OpenLineageListener
           listeners = [OpenLineageListener()]
   ```



##########
airflow/providers/openlineage/plugins/openlineage.py:
##########
@@ -0,0 +1,46 @@
+# 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 airflow.plugins_manager import AirflowPlugin
+from airflow.providers.openlineage.plugins.macros import lineage_parent_id, lineage_run_id
+
+
+def _is_disabled():

Review Comment:
   ```suggestion
   def _is_disabled() -> bool:
   ```



##########
airflow/providers/openlineage/utils/__init__.py:
##########
@@ -0,0 +1,499 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def get_connection_uri(conn):
+    """
+    Return the connection URI for the given ID. We first attempt to lookup
+    the connection URI via AIRFLOW_CONN_<conn_id>, else fallback on querying
+    the Airflow's connection table.
+    """
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not _filtered_query_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def _filtered_query_params(k: str):
+    unfiltered_snowflake_keys = [
+        "extra__snowflake__warehouse",
+        "extra__snowflake__account",
+        "extra__snowflake__database",
+    ]

Review Comment:
   Do these need to live here? Shouldn't they be in the facet generation code inside the snowflake provider?



##########
airflow/providers/openlineage/utils/__init__.py:
##########
@@ -0,0 +1,499 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def get_connection_uri(conn):
+    """
+    Return the connection URI for the given ID. We first attempt to lookup
+    the connection URI via AIRFLOW_CONN_<conn_id>, else fallback on querying
+    the Airflow's connection table.
+    """
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not _filtered_query_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def _filtered_query_params(k: str):
+    unfiltered_snowflake_keys = [
+        "extra__snowflake__warehouse",
+        "extra__snowflake__account",
+        "extra__snowflake__database",
+    ]
+    filtered_key_substrings = [
+        "aws_access_key_id",
+        "aws_secret_access_key",
+        "extra__snowflake__",
+    ]
+    return k not in unfiltered_snowflake_keys and any(substr in k for substr in filtered_key_substrings)
+
+
+def get_normalized_postgres_connection_uri(conn):
+    """
+    URIs starting with postgresql:// and postgres:// are both valid
+    PostgreSQL connection strings. This function normalizes it to
+    postgres:// as canonical name according to OpenLineage spec.
+    """
+    uri = get_connection_uri(conn)
+    if uri.startswith("postgresql"):
+        uri = uri.replace("postgresql", "postgres", 1)
+    return uri
+
+
+def get_connection(conn_id) -> Connection | None:
+    from airflow.hooks.base import BaseHook
+
+    try:
+        return BaseHook.get_connection(conn_id=conn_id)
+    except Exception:
+        return None
+
+
+def get_job_name(task):
+    return f"{task.dag_id}.{task.task_id}"
+
+
+def get_custom_facets(
+    dagrun, task, is_external_trigger: bool, task_instance: TaskInstance | None = None
+) -> dict[str, Any]:
+    custom_facets = {
+        "airflow_runArgs": AirflowRunArgsRunFacet(is_external_trigger),
+        "airflow_version": AirflowVersionRunFacet.from_dagrun_and_task(dagrun, task),
+    }
+    # check for -1 comes from SmartSensor compatibility with dynamic task mapping
+    # this comes from Airflow code
+    if hasattr(task_instance, "map_index") and getattr(task_instance, "map_index") != -1:
+        custom_facets["airflow_mappedTask"] = AirflowMappedTaskRunFacet.from_task_instance(task_instance)
+    return custom_facets
+
+
+class InfoJsonEncodable(dict):
+    """
+    Airflow objects might not be json-encodable overall.
+
+    The class provides additional attributes to control
+    what and how is encoded:
+    * renames: a dictionary of attribute name changes
+    * casts: a dictionary consisting of attribute names
+             and corresponding methods that should change
+             object value
+    * includes: list of attributes to be included in encoding
+    * excludes: list of attributes to be excluded from encoding
+
+    Don't use both includes and excludes.
+    """
+
+    renames: dict[str, str] = {}
+    casts: dict[str, Any] = {}
+    includes: list[str] = []
+    excludes: list[str] = []
+
+    def __init__(self, obj):
+        self.obj = obj
+        self._fields = []
+
+        self._cast_fields()
+        self._rename_fields()
+        self._include_fields()
+        dict.__init__(
+            self,
+            **{field: InfoJsonEncodable._cast_basic_types(getattr(self, field)) for field in self._fields},
+        )
+
+    @staticmethod
+    def _cast_basic_types(value):
+        if isinstance(value, datetime.datetime):
+            return value.isoformat()
+        if isinstance(value, (set, list, tuple)):
+            return str(list(value))
+        return value
+
+    def _rename_fields(self):
+        for field, renamed in self.renames.items():
+            if hasattr(self.obj, field):
+                setattr(self, renamed, getattr(self.obj, field))
+                self._fields.append(renamed)
+
+    def _cast_fields(self):
+        for field, func in self.casts.items():
+            setattr(self, field, func(self.obj))
+            self._fields.append(field)
+
+    def _include_fields(self):
+        if self.includes and self.excludes:
+            raise Exception("Don't use both includes and excludes.")
+        if self.includes:
+            for field in self.includes:
+                if field in self._fields or not hasattr(self.obj, field):
+                    continue
+                setattr(self, field, getattr(self.obj, field))
+                self._fields.append(field)
+        else:
+            for field, val in self.obj.__dict__.items():
+                if field in self._fields or field in self.excludes or field in self.renames:
+                    continue
+                setattr(self, field, val)
+                self._fields.append(field)
+
+
+class DagInfo(InfoJsonEncodable):
+    """Defines encoding DAG object to JSON."""
+
+    includes = ["dag_id", "schedule_interval", "tags", "start_date"]
+    casts = {"timetable": lambda dag: dag.timetable.serialize() if getattr(dag, "timetable", None) else None}
+    renames = {"_dag_id": "dag_id"}
+
+
+class DagRunInfo(InfoJsonEncodable):
+    """Defines encoding DagRun object to JSON."""
+
+    includes = [
+        "conf",
+        "dag_id",
+        "data_interval_start",
+        "data_interval_end",
+        "external_trigger",
+        "run_id",
+        "run_type",
+        "start_date",
+    ]
+
+
+class TaskInstanceInfo(InfoJsonEncodable):
+    """Defines encoding TaskInstance object to JSON."""
+
+    includes = ["duration", "try_number", "pool"]
+    casts = {
+        "map_index": lambda ti: ti.map_index
+        if hasattr(ti, "map_index") and getattr(ti, "map_index") != -1
+        else None
+    }
+
+
+class TaskInfo(InfoJsonEncodable):
+    """Defines encoding BaseOperator/AbstractOperator object to JSON."""
+
+    renames = {
+        "_BaseOperator__init_kwargs": "args",
+        "_BaseOperator__from_mapped": "mapped",
+        "_downstream_task_ids": "downstream_task_ids",
+        "_upstream_task_ids": "upstream_task_ids",
+    }
+    excludes = [
+        "_BaseOperator__instantiated",
+        "_dag",
+        "_hook",
+        "_log",
+        "_outlets",
+        "_inlets",
+        "_lock_for_execution",
+        "handler",
+        "params",
+        "python_callable",
+        "retry_delay",
+    ]
+    casts = {
+        "operator_class": lambda task: f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",  # noqa
+        "task_group": lambda task: TaskGroupInfo(task.task_group)
+        if hasattr(task, "task_group") and getattr(task.task_group, "_group_id", None)
+        else None,
+    }
+
+
+class TaskGroupInfo(InfoJsonEncodable):
+    """Defines encoding TaskGroup object to JSON."""
+
+    renames = {
+        "_group_id": "group_id",
+    }
+    includes = [
+        "downstream_group_ids",
+        "downstream_task_ids",
+        "prefix_group_id",
+        "tooltip",
+        "upstream_group_ids",
+        "upstream_task_ids",
+    ]
+
+
+def get_airflow_run_facet(
+    dag_run: DagRun,
+    dag: DAG,
+    task_instance: TaskInstance,
+    task: BaseOperator,
+    task_uuid: str,
+):
+    return {
+        "airflow": json.loads(
+            json.dumps(
+                asdict(
+                    AirflowRunFacet(
+                        dag=DagInfo(dag),
+                        dagRun=DagRunInfo(dag_run),
+                        taskInstance=TaskInstanceInfo(task_instance),
+                        task=TaskInfo(task),
+                        taskUuid=task_uuid,
+                    )
+                ),
+                default=str,
+            )
+        )
+    }
+
+
+def get_dagrun_start_end(dagrun: DagRun, dag: DAG):
+    try:
+        return dagrun.data_interval_start, dagrun.data_interval_end
+    except AttributeError:
+        # Airflow < 2.2 before adding data interval
+        pass
+    start = dagrun.execution_date
+    end = dag.following_schedule(start)
+    return start, end or start
+
+
+def import_from_string(path: str):
+    try:
+        module_path, target = path.rsplit(".", 1)
+        module = importlib.import_module(module_path)
+        return getattr(module, target)
+    except Exception as e:
+        log.warning(e)
+        raise ImportError(f"Failed to import {path}") from e
+
+
+def try_import_from_string(path: str):
+    try:
+        return import_from_string(path)
+    except ImportError:
+        return None

Review Comment:
   `from contextlib import suppress` and then
   
   ```suggestion
       with suppress(ImportError):
           return import_from_string(path)
   ```



##########
airflow/providers/openlineage/utils/__init__.py:
##########
@@ -0,0 +1,499 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def get_connection_uri(conn):
+    """
+    Return the connection URI for the given ID. We first attempt to lookup
+    the connection URI via AIRFLOW_CONN_<conn_id>, else fallback on querying
+    the Airflow's connection table.
+    """
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not _filtered_query_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def _filtered_query_params(k: str):
+    unfiltered_snowflake_keys = [
+        "extra__snowflake__warehouse",
+        "extra__snowflake__account",
+        "extra__snowflake__database",
+    ]
+    filtered_key_substrings = [
+        "aws_access_key_id",
+        "aws_secret_access_key",
+        "extra__snowflake__",
+    ]
+    return k not in unfiltered_snowflake_keys and any(substr in k for substr in filtered_key_substrings)
+
+
+def get_normalized_postgres_connection_uri(conn):
+    """
+    URIs starting with postgresql:// and postgres:// are both valid
+    PostgreSQL connection strings. This function normalizes it to
+    postgres:// as canonical name according to OpenLineage spec.
+    """
+    uri = get_connection_uri(conn)
+    if uri.startswith("postgresql"):
+        uri = uri.replace("postgresql", "postgres", 1)
+    return uri
+
+
+def get_connection(conn_id) -> Connection | None:
+    from airflow.hooks.base import BaseHook
+
+    try:
+        return BaseHook.get_connection(conn_id=conn_id)
+    except Exception:
+        return None
+
+
+def get_job_name(task):
+    return f"{task.dag_id}.{task.task_id}"
+
+
+def get_custom_facets(
+    dagrun, task, is_external_trigger: bool, task_instance: TaskInstance | None = None
+) -> dict[str, Any]:
+    custom_facets = {
+        "airflow_runArgs": AirflowRunArgsRunFacet(is_external_trigger),
+        "airflow_version": AirflowVersionRunFacet.from_dagrun_and_task(dagrun, task),
+    }
+    # check for -1 comes from SmartSensor compatibility with dynamic task mapping
+    # this comes from Airflow code
+    if hasattr(task_instance, "map_index") and getattr(task_instance, "map_index") != -1:
+        custom_facets["airflow_mappedTask"] = AirflowMappedTaskRunFacet.from_task_instance(task_instance)
+    return custom_facets
+
+
+class InfoJsonEncodable(dict):
+    """
+    Airflow objects might not be json-encodable overall.
+
+    The class provides additional attributes to control
+    what and how is encoded:
+    * renames: a dictionary of attribute name changes
+    * casts: a dictionary consisting of attribute names
+             and corresponding methods that should change
+             object value
+    * includes: list of attributes to be included in encoding
+    * excludes: list of attributes to be excluded from encoding
+
+    Don't use both includes and excludes.
+    """
+
+    renames: dict[str, str] = {}
+    casts: dict[str, Any] = {}
+    includes: list[str] = []
+    excludes: list[str] = []
+
+    def __init__(self, obj):
+        self.obj = obj
+        self._fields = []
+
+        self._cast_fields()
+        self._rename_fields()
+        self._include_fields()
+        dict.__init__(
+            self,
+            **{field: InfoJsonEncodable._cast_basic_types(getattr(self, field)) for field in self._fields},
+        )
+
+    @staticmethod
+    def _cast_basic_types(value):
+        if isinstance(value, datetime.datetime):
+            return value.isoformat()
+        if isinstance(value, (set, list, tuple)):
+            return str(list(value))
+        return value
+
+    def _rename_fields(self):
+        for field, renamed in self.renames.items():
+            if hasattr(self.obj, field):
+                setattr(self, renamed, getattr(self.obj, field))
+                self._fields.append(renamed)
+
+    def _cast_fields(self):
+        for field, func in self.casts.items():
+            setattr(self, field, func(self.obj))
+            self._fields.append(field)
+
+    def _include_fields(self):
+        if self.includes and self.excludes:
+            raise Exception("Don't use both includes and excludes.")
+        if self.includes:
+            for field in self.includes:
+                if field in self._fields or not hasattr(self.obj, field):
+                    continue
+                setattr(self, field, getattr(self.obj, field))
+                self._fields.append(field)
+        else:
+            for field, val in self.obj.__dict__.items():
+                if field in self._fields or field in self.excludes or field in self.renames:
+                    continue
+                setattr(self, field, val)
+                self._fields.append(field)
+
+
+class DagInfo(InfoJsonEncodable):
+    """Defines encoding DAG object to JSON."""
+
+    includes = ["dag_id", "schedule_interval", "tags", "start_date"]
+    casts = {"timetable": lambda dag: dag.timetable.serialize() if getattr(dag, "timetable", None) else None}
+    renames = {"_dag_id": "dag_id"}
+
+
+class DagRunInfo(InfoJsonEncodable):
+    """Defines encoding DagRun object to JSON."""
+
+    includes = [
+        "conf",
+        "dag_id",
+        "data_interval_start",
+        "data_interval_end",
+        "external_trigger",
+        "run_id",
+        "run_type",
+        "start_date",
+    ]
+
+
+class TaskInstanceInfo(InfoJsonEncodable):
+    """Defines encoding TaskInstance object to JSON."""
+
+    includes = ["duration", "try_number", "pool"]
+    casts = {
+        "map_index": lambda ti: ti.map_index
+        if hasattr(ti, "map_index") and getattr(ti, "map_index") != -1
+        else None
+    }
+
+
+class TaskInfo(InfoJsonEncodable):
+    """Defines encoding BaseOperator/AbstractOperator object to JSON."""
+
+    renames = {
+        "_BaseOperator__init_kwargs": "args",
+        "_BaseOperator__from_mapped": "mapped",
+        "_downstream_task_ids": "downstream_task_ids",
+        "_upstream_task_ids": "upstream_task_ids",
+    }
+    excludes = [
+        "_BaseOperator__instantiated",
+        "_dag",
+        "_hook",
+        "_log",
+        "_outlets",
+        "_inlets",
+        "_lock_for_execution",
+        "handler",
+        "params",
+        "python_callable",
+        "retry_delay",
+    ]
+    casts = {
+        "operator_class": lambda task: f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",  # noqa
+        "task_group": lambda task: TaskGroupInfo(task.task_group)
+        if hasattr(task, "task_group") and getattr(task.task_group, "_group_id", None)
+        else None,
+    }
+
+
+class TaskGroupInfo(InfoJsonEncodable):
+    """Defines encoding TaskGroup object to JSON."""
+
+    renames = {
+        "_group_id": "group_id",
+    }
+    includes = [
+        "downstream_group_ids",
+        "downstream_task_ids",
+        "prefix_group_id",
+        "tooltip",
+        "upstream_group_ids",
+        "upstream_task_ids",
+    ]
+
+
+def get_airflow_run_facet(
+    dag_run: DagRun,
+    dag: DAG,
+    task_instance: TaskInstance,
+    task: BaseOperator,
+    task_uuid: str,
+):
+    return {
+        "airflow": json.loads(
+            json.dumps(
+                asdict(
+                    AirflowRunFacet(
+                        dag=DagInfo(dag),
+                        dagRun=DagRunInfo(dag_run),
+                        taskInstance=TaskInstanceInfo(task_instance),
+                        task=TaskInfo(task),
+                        taskUuid=task_uuid,
+                    )
+                ),
+                default=str,
+            )
+        )
+    }
+
+
+def get_dagrun_start_end(dagrun: DagRun, dag: DAG):
+    try:
+        return dagrun.data_interval_start, dagrun.data_interval_end
+    except AttributeError:
+        # Airflow < 2.2 before adding data interval
+        pass
+    start = dagrun.execution_date
+    end = dag.following_schedule(start)
+    return start, end or start
+
+
+def import_from_string(path: str):
+    try:
+        module_path, target = path.rsplit(".", 1)
+        module = importlib.import_module(module_path)
+        return getattr(module, target)
+    except Exception as e:
+        log.warning(e)
+        raise ImportError(f"Failed to import {path}") from e
+
+
+def try_import_from_string(path: str):
+    try:
+        return import_from_string(path)
+    except ImportError:
+        return None
+
+
+def redact_with_exclusions(source: Any):
+    try:
+        from airflow.utils.log.secrets_masker import (
+            _secrets_masker,
+            should_hide_value_for_key,
+        )
+    except ImportError:
+        return source
+    import copy
+
+    sm = copy.deepcopy(_secrets_masker())
+    MAX_RECURSION_DEPTH = 20
+
+    def _redact(item, name: str | None, depth: int):
+        if depth > MAX_RECURSION_DEPTH:
+            return item
+        try:
+            if name and should_hide_value_for_key(name):
+                return sm._redact_all(item, depth)
+            if isinstance(item, dict):
+                return {
+                    dict_key: _redact(subval, name=dict_key, depth=(depth + 1))
+                    for dict_key, subval in item.items()
+                }
+            elif is_dataclass(item) or (is_json_serializable(item) and hasattr(item, "__dict__")):
+                for dict_key, subval in item.__dict__.items():
+                    if _is_name_redactable(dict_key, item):
+                        setattr(
+                            item,
+                            dict_key,
+                            _redact(subval, name=dict_key, depth=(depth + 1)),
+                        )
+                return item
+            elif isinstance(item, str):
+                if sm.replacer:
+                    return sm.replacer.sub("***", item)
+                return item
+            elif isinstance(item, (tuple, set)):
+                return tuple(_redact(subval, name=None, depth=(depth + 1)) for subval in item)
+            elif isinstance(item, list):
+                return [_redact(subval, name=None, depth=(depth + 1)) for subval in item]
+            else:
+                return item
+        except Exception as e:
+            log.warning(
+                "Unable to redact %s" "Error was: %s: %s",
+                repr(item),
+                type(e).__name__,
+                str(e),
+            )
+            return item
+
+    return _redact(source, name=None, depth=0)
+
+
+def is_dataclass(item):
+    return getattr(item.__class__, "__attrs_attrs__", None) is not None

Review Comment:
   a) this is not dataclass, but attrs, b) https://www.attrs.org/en/stable/api.html#attrs.has
   
   ```suggestion
       return attrs.has(item.__class__)   
   ```



##########
airflow/providers/openlineage/utils/converters.py:
##########
@@ -0,0 +1,33 @@
+# 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
+
+from airflow.lineage.entities import Table
+from openlineage.client.run import Dataset
+
+
+def convert_to_dataset(obj):

Review Comment:
   Since Airflow has it's own concept of Dataset we should probably disambiguate them a bit.
   ```suggestion
   def convert_to_ol_dataset(obj):
   ```



##########
airflow/providers/openlineage/extractors/manager.py:
##########
@@ -0,0 +1,119 @@
+# 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
+
+from airflow.providers.openlineage.extractors import BaseExtractor, Extractors, OperatorLineage
+from airflow.providers.openlineage.plugins.facets import (
+    UnknownOperatorAttributeRunFacet,
+    UnknownOperatorInstance,
+)
+from airflow.providers.openlineage.utils import get_operator_class
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class ExtractorManager(LoggingMixin):
+    """Class abstracting management of custom extractors."""
+
+    def __init__(self):
+        self.extractors = {}
+        self.task_to_extractor = Extractors()
+
+    def add_extractor(self, operator, extractor: type[BaseExtractor]):
+        self.task_to_extractor.add_extractor(operator, extractor)
+
+    def extract_metadata(self, dagrun, task, complete: bool = False, task_instance=None) -> OperatorLineage:
+        extractor = self._get_extractor(task)
+        task_info = (
+            f"task_type={get_operator_class(task).__name__} "
+            f"airflow_dag_id={task.dag_id} "
+            f"task_id={task.task_id} "
+            f"airflow_run_id={dagrun.run_id} "
+        )
+
+        if extractor:
+            # Extracting advanced metadata is only possible when extractor for particular operator
+            # is defined. Without it, we can't extract any input or output data.
+            try:
+                self.log.debug("Using extractor %s %s", extractor.__class__.__name__, str(task_info))
+                if complete:
+                    task_metadata = extractor.extract_on_complete(task_instance)
+                else:
+                    task_metadata = extractor.extract()
+
+                self.log.debug("Found task metadata for operation %s: %s", task.task_id, str(task_metadata))
+                if task_metadata:
+                    if (not task_metadata.inputs) and (not task_metadata.outputs):
+                        inlets = task.get_inlet_defs()
+                        outlets = task.get_outlet_defs()
+                        self.extract_inlets_and_outlets(task_metadata, inlets, outlets)
+
+                    return task_metadata
+
+            except Exception as e:
+                self.log.exception("Failed to extract metadata %s %s", e, task_info)
+        else:
+            self.log.warning("Unable to find an extractor %s", task_info)
+
+            # Only include the unkonwnSourceAttribute facet if there is no extractor
+            task_metadata = OperatorLineage(
+                run_facets={
+                    "unknownSourceAttribute": UnknownOperatorAttributeRunFacet(
+                        unknownItems=[
+                            UnknownOperatorInstance(
+                                name=get_operator_class(task).__name__,
+                                properties={attr: value for attr, value in task.__dict__.items()},
+                            )
+                        ]
+                    )
+                },
+            )
+            inlets = task.get_inlet_defs()
+            outlets = task.get_outlet_defs()
+            self.extract_inlets_and_outlets(task_metadata, inlets, outlets)
+            return task_metadata
+
+        return OperatorLineage()
+
+    def _get_extractor(self, task) -> BaseExtractor | None:
+        # TODO: Re-enable in Extractor PR
+        # self.task_to_extractor.instantiate_abstract_extractors(task)
+        if task.task_id in self.extractors:
+            return self.extractors[task.task_id]

Review Comment:
   Why do we need to cache by task_id? What process is this called in that it makes sense to cache this at all?



##########
airflow/providers/openlineage/extractors/manager.py:
##########
@@ -0,0 +1,119 @@
+# 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
+
+from airflow.providers.openlineage.extractors import BaseExtractor, Extractors, OperatorLineage
+from airflow.providers.openlineage.plugins.facets import (
+    UnknownOperatorAttributeRunFacet,
+    UnknownOperatorInstance,
+)
+from airflow.providers.openlineage.utils import get_operator_class
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class ExtractorManager(LoggingMixin):
+    """Class abstracting management of custom extractors."""
+
+    def __init__(self):
+        self.extractors = {}
+        self.task_to_extractor = Extractors()
+
+    def add_extractor(self, operator, extractor: type[BaseExtractor]):
+        self.task_to_extractor.add_extractor(operator, extractor)
+
+    def extract_metadata(self, dagrun, task, complete: bool = False, task_instance=None) -> OperatorLineage:
+        extractor = self._get_extractor(task)
+        task_info = (
+            f"task_type={get_operator_class(task).__name__} "

Review Comment:
   This should use `task.task_type` etc, not a custom function.



##########
airflow/providers/openlineage/plugins/adapter.py:
##########
@@ -0,0 +1,302 @@
+# 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
+import uuid
+from typing import TYPE_CHECKING
+
+import requests.exceptions
+
+from airflow.providers.openlineage import version as OPENLINEAGE_PROVIDER_VERSION
+from airflow.providers.openlineage.extractors import OperatorLineage
+from airflow.providers.openlineage.utils import redact_with_exclusions
+from airflow.utils.log.logging_mixin import LoggingMixin
+from openlineage.client import OpenLineageClient, set_producer
+from openlineage.client.facet import (
+    BaseFacet,
+    DocumentationJobFacet,
+    ErrorMessageRunFacet,
+    NominalTimeRunFacet,
+    OwnershipJobFacet,
+    OwnershipJobFacetOwners,
+    ParentRunFacet,
+    ProcessingEngineRunFacet,
+    SourceCodeLocationJobFacet,
+)
+from openlineage.client.run import Job, Run, RunEvent, RunState
+
+if TYPE_CHECKING:
+    from airflow.models.dagrun import DagRun
+
+
+_DAG_DEFAULT_NAMESPACE = "default"
+
+_DAG_NAMESPACE = os.getenv("OPENLINEAGE_NAMESPACE", _DAG_DEFAULT_NAMESPACE)

Review Comment:
   Use Airflow config please.



##########
airflow/providers/openlineage/extractors/manager.py:
##########
@@ -0,0 +1,119 @@
+# 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
+
+from airflow.providers.openlineage.extractors import BaseExtractor, Extractors, OperatorLineage
+from airflow.providers.openlineage.plugins.facets import (
+    UnknownOperatorAttributeRunFacet,
+    UnknownOperatorInstance,
+)
+from airflow.providers.openlineage.utils import get_operator_class
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class ExtractorManager(LoggingMixin):
+    """Class abstracting management of custom extractors."""
+
+    def __init__(self):
+        self.extractors = {}
+        self.task_to_extractor = Extractors()
+
+    def add_extractor(self, operator, extractor: type[BaseExtractor]):
+        self.task_to_extractor.add_extractor(operator, extractor)
+
+    def extract_metadata(self, dagrun, task, complete: bool = False, task_instance=None) -> OperatorLineage:
+        extractor = self._get_extractor(task)
+        task_info = (
+            f"task_type={get_operator_class(task).__name__} "
+            f"airflow_dag_id={task.dag_id} "
+            f"task_id={task.task_id} "
+            f"airflow_run_id={dagrun.run_id} "
+        )
+
+        if extractor:
+            # Extracting advanced metadata is only possible when extractor for particular operator
+            # is defined. Without it, we can't extract any input or output data.
+            try:
+                self.log.debug("Using extractor %s %s", extractor.__class__.__name__, str(task_info))
+                if complete:
+                    task_metadata = extractor.extract_on_complete(task_instance)
+                else:
+                    task_metadata = extractor.extract()
+
+                self.log.debug("Found task metadata for operation %s: %s", task.task_id, str(task_metadata))
+                if task_metadata:
+                    if (not task_metadata.inputs) and (not task_metadata.outputs):
+                        inlets = task.get_inlet_defs()
+                        outlets = task.get_outlet_defs()
+                        self.extract_inlets_and_outlets(task_metadata, inlets, outlets)
+
+                    return task_metadata
+
+            except Exception as e:
+                self.log.exception("Failed to extract metadata %s %s", e, task_info)
+        else:
+            self.log.warning("Unable to find an extractor %s", task_info)

Review Comment:
   Not sure (yet as I'm reading this PR) where this code is called, but would this end up warning just because that task has no registered extractor? If so this should be at debug level, not warning



##########
airflow/providers/openlineage/plugins/openlineage.py:
##########
@@ -0,0 +1,46 @@
+# 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 airflow.plugins_manager import AirflowPlugin
+from airflow.providers.openlineage.plugins.macros import lineage_parent_id, lineage_run_id
+
+
+def _is_disabled():
+    return os.getenv("OPENLINEAGE_DISABLED", None) in [True, "true", "True"]

Review Comment:
   Airflow config here too (again, can fallback to OL env var) -- using `conf.getboolean`



##########
airflow/providers/openlineage/plugins/listener.py:
##########
@@ -0,0 +1,189 @@
+# 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 logging
+from concurrent.futures import Executor, ThreadPoolExecutor
+from typing import TYPE_CHECKING
+
+from airflow.listeners import hookimpl
+from airflow.providers.openlineage.extractors import ExtractorManager
+from airflow.providers.openlineage.plugins.adapter import OpenLineageAdapter
+from airflow.providers.openlineage.utils import (
+    get_airflow_run_facet,
+    get_custom_facets,
+    get_job_name,
+    get_task_location,
+    print_exception,
+)
+
+if TYPE_CHECKING:
+    from sqlalchemy.orm import Session
+
+    from airflow.models import DagRun, TaskInstance
+
+
+class OpenLineageListener:
+    """
+    OpenLineage listener
+    Sends events on task instance and dag run starts, completes and failures.
+    """
+
+    def __init__(self):
+        self.log = logging.getLogger(__name__)
+        self.executor: Executor = None  # type: ignore
+        self.extractor_manager = ExtractorManager()
+        self.adapter = OpenLineageAdapter()
+
+    @hookimpl
+    def on_task_instance_running(
+        self, previous_state, task_instance: TaskInstance, session: Session  # This will always be QUEUED
+    ):
+        if not hasattr(task_instance, "task"):
+            self.log.warning(
+                f"No task set for TI object task_id: {task_instance.task_id} - "
+                f"dag_id: {task_instance.dag_id} - run_id {task_instance.run_id}"
+            )
+            return
+
+        self.log.debug("OpenLineage listener got notification about task instance start")
+        dagrun = task_instance.dag_run
+        task = task_instance.task
+        dag = task.dag
+
+        @print_exception
+        def on_running():
+            # that's a workaround to detect task running from deferred state
+            # we return here because Airflow 2.3 needs task from deferred state
+            if task_instance.next_method is not None:
+                return
+            parent_run_id = self.adapter.build_dag_run_id(dag.dag_id, dagrun.run_id)
+
+            task_uuid = self.adapter.build_task_instance_run_id(
+                task.task_id, task_instance.execution_date, task_instance.try_number
+            )
+
+            task_metadata = self.extractor_manager.extract_metadata(dagrun, task)
+
+            self.adapter.start_task(
+                run_id=task_uuid,
+                job_name=get_job_name(task),
+                job_description=dag.description,
+                event_time=task_instance.start_date.isoformat(),
+                parent_job_name=dag.dag_id,
+                parent_run_id=parent_run_id,
+                code_location=get_task_location(task),
+                nominal_start_time=dagrun.data_interval_start.isoformat(),
+                nominal_end_time=dagrun.data_interval_end.isoformat(),
+                owners=dag.owner.split(", "),
+                task=task_metadata,
+                run_facets={
+                    **task_metadata.run_facets,
+                    **get_custom_facets(dagrun, task, dagrun.external_trigger, task_instance),
+                    **get_airflow_run_facet(dagrun, dag, task_instance, task, task_uuid),
+                },
+            )
+
+        self.executor.submit(on_running)
+
+    @hookimpl
+    def on_task_instance_success(self, previous_state, task_instance: TaskInstance, session):
+        self.log.debug("OpenLineage listener got notification about task instance success")
+
+        dagrun = task_instance.dag_run
+        task = task_instance.task
+
+        task_uuid = OpenLineageAdapter.build_task_instance_run_id(
+            task.task_id, task_instance.execution_date, task_instance.try_number - 1
+        )
+
+        @print_exception
+        def on_success():
+            task_metadata = self.extractor_manager.extract_metadata(
+                dagrun, task, complete=True, task_instance=task_instance
+            )
+            self.adapter.complete_task(
+                run_id=task_uuid,
+                job_name=get_job_name(task),
+                end_time=task_instance.end_date.isoformat(),
+                task=task_metadata,
+            )
+
+        self.executor.submit(on_success)
+
+    @hookimpl
+    def on_task_instance_failed(self, previous_state, task_instance: TaskInstance, session):
+        self.log.debug("OpenLineage listener got notification about task instance failure")
+
+        dagrun = task_instance.dag_run
+        task = task_instance.task
+
+        task_uuid = OpenLineageAdapter.build_task_instance_run_id(
+            task.task_id, task_instance.execution_date, task_instance.try_number - 1
+        )
+
+        @print_exception
+        def on_failure():
+            task_metadata = self.extractor_manager.extract_metadata(
+                dagrun, task, complete=True, task_instance=task_instance
+            )
+
+            self.adapter.fail_task(
+                run_id=task_uuid,
+                job_name=get_job_name(task),
+                end_time=task_instance.end_date.isoformat(),
+                task=task_metadata,
+            )
+
+        self.executor.submit(on_failure)
+
+    @hookimpl
+    def on_starting(self, component):
+        self.log.debug("on_starting: %s", component.__class__.__name__)
+        self.executor = ThreadPoolExecutor(max_workers=8, thread_name_prefix="openlineage_")
+
+    @hookimpl
+    def before_stopping(self, component):
+        self.log.debug("before_stopping: %s", component.__class__.__name__)
+        self.executor.shutdown(wait=True)

Review Comment:
   Can we place a timeout on this wait so it doesn't hang blocking the task for ever?



##########
airflow/providers/openlineage/utils/__init__.py:
##########
@@ -0,0 +1,499 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def get_connection_uri(conn):
+    """
+    Return the connection URI for the given ID. We first attempt to lookup
+    the connection URI via AIRFLOW_CONN_<conn_id>, else fallback on querying
+    the Airflow's connection table.
+    """
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not _filtered_query_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def _filtered_query_params(k: str):
+    unfiltered_snowflake_keys = [
+        "extra__snowflake__warehouse",
+        "extra__snowflake__account",
+        "extra__snowflake__database",
+    ]
+    filtered_key_substrings = [
+        "aws_access_key_id",
+        "aws_secret_access_key",
+        "extra__snowflake__",
+    ]
+    return k not in unfiltered_snowflake_keys and any(substr in k for substr in filtered_key_substrings)
+
+
+def get_normalized_postgres_connection_uri(conn):
+    """
+    URIs starting with postgresql:// and postgres:// are both valid
+    PostgreSQL connection strings. This function normalizes it to
+    postgres:// as canonical name according to OpenLineage spec.
+    """
+    uri = get_connection_uri(conn)
+    if uri.startswith("postgresql"):
+        uri = uri.replace("postgresql", "postgres", 1)
+    return uri
+
+
+def get_connection(conn_id) -> Connection | None:
+    from airflow.hooks.base import BaseHook
+
+    try:
+        return BaseHook.get_connection(conn_id=conn_id)
+    except Exception:
+        return None
+
+
+def get_job_name(task):
+    return f"{task.dag_id}.{task.task_id}"
+
+
+def get_custom_facets(
+    dagrun, task, is_external_trigger: bool, task_instance: TaskInstance | None = None
+) -> dict[str, Any]:
+    custom_facets = {
+        "airflow_runArgs": AirflowRunArgsRunFacet(is_external_trigger),
+        "airflow_version": AirflowVersionRunFacet.from_dagrun_and_task(dagrun, task),
+    }
+    # check for -1 comes from SmartSensor compatibility with dynamic task mapping
+    # this comes from Airflow code
+    if hasattr(task_instance, "map_index") and getattr(task_instance, "map_index") != -1:
+        custom_facets["airflow_mappedTask"] = AirflowMappedTaskRunFacet.from_task_instance(task_instance)
+    return custom_facets
+
+
+class InfoJsonEncodable(dict):
+    """
+    Airflow objects might not be json-encodable overall.
+
+    The class provides additional attributes to control
+    what and how is encoded:
+    * renames: a dictionary of attribute name changes
+    * casts: a dictionary consisting of attribute names
+             and corresponding methods that should change
+             object value
+    * includes: list of attributes to be included in encoding
+    * excludes: list of attributes to be excluded from encoding
+
+    Don't use both includes and excludes.
+    """
+
+    renames: dict[str, str] = {}
+    casts: dict[str, Any] = {}
+    includes: list[str] = []
+    excludes: list[str] = []
+
+    def __init__(self, obj):
+        self.obj = obj
+        self._fields = []
+
+        self._cast_fields()
+        self._rename_fields()
+        self._include_fields()
+        dict.__init__(
+            self,
+            **{field: InfoJsonEncodable._cast_basic_types(getattr(self, field)) for field in self._fields},
+        )
+
+    @staticmethod
+    def _cast_basic_types(value):
+        if isinstance(value, datetime.datetime):
+            return value.isoformat()
+        if isinstance(value, (set, list, tuple)):
+            return str(list(value))
+        return value
+
+    def _rename_fields(self):
+        for field, renamed in self.renames.items():
+            if hasattr(self.obj, field):
+                setattr(self, renamed, getattr(self.obj, field))
+                self._fields.append(renamed)
+
+    def _cast_fields(self):
+        for field, func in self.casts.items():
+            setattr(self, field, func(self.obj))
+            self._fields.append(field)
+
+    def _include_fields(self):
+        if self.includes and self.excludes:
+            raise Exception("Don't use both includes and excludes.")
+        if self.includes:
+            for field in self.includes:
+                if field in self._fields or not hasattr(self.obj, field):
+                    continue
+                setattr(self, field, getattr(self.obj, field))
+                self._fields.append(field)
+        else:
+            for field, val in self.obj.__dict__.items():
+                if field in self._fields or field in self.excludes or field in self.renames:
+                    continue
+                setattr(self, field, val)
+                self._fields.append(field)
+
+
+class DagInfo(InfoJsonEncodable):
+    """Defines encoding DAG object to JSON."""
+
+    includes = ["dag_id", "schedule_interval", "tags", "start_date"]
+    casts = {"timetable": lambda dag: dag.timetable.serialize() if getattr(dag, "timetable", None) else None}
+    renames = {"_dag_id": "dag_id"}
+
+
+class DagRunInfo(InfoJsonEncodable):
+    """Defines encoding DagRun object to JSON."""
+
+    includes = [
+        "conf",
+        "dag_id",
+        "data_interval_start",
+        "data_interval_end",
+        "external_trigger",
+        "run_id",
+        "run_type",
+        "start_date",
+    ]
+
+
+class TaskInstanceInfo(InfoJsonEncodable):
+    """Defines encoding TaskInstance object to JSON."""
+
+    includes = ["duration", "try_number", "pool"]
+    casts = {
+        "map_index": lambda ti: ti.map_index
+        if hasattr(ti, "map_index") and getattr(ti, "map_index") != -1
+        else None
+    }
+
+
+class TaskInfo(InfoJsonEncodable):
+    """Defines encoding BaseOperator/AbstractOperator object to JSON."""
+
+    renames = {
+        "_BaseOperator__init_kwargs": "args",
+        "_BaseOperator__from_mapped": "mapped",
+        "_downstream_task_ids": "downstream_task_ids",
+        "_upstream_task_ids": "upstream_task_ids",
+    }
+    excludes = [
+        "_BaseOperator__instantiated",
+        "_dag",
+        "_hook",
+        "_log",
+        "_outlets",
+        "_inlets",
+        "_lock_for_execution",
+        "handler",
+        "params",
+        "python_callable",
+        "retry_delay",
+    ]
+    casts = {
+        "operator_class": lambda task: f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",  # noqa
+        "task_group": lambda task: TaskGroupInfo(task.task_group)
+        if hasattr(task, "task_group") and getattr(task.task_group, "_group_id", None)
+        else None,
+    }
+
+
+class TaskGroupInfo(InfoJsonEncodable):
+    """Defines encoding TaskGroup object to JSON."""
+
+    renames = {
+        "_group_id": "group_id",
+    }
+    includes = [
+        "downstream_group_ids",
+        "downstream_task_ids",
+        "prefix_group_id",
+        "tooltip",
+        "upstream_group_ids",
+        "upstream_task_ids",
+    ]
+
+
+def get_airflow_run_facet(
+    dag_run: DagRun,
+    dag: DAG,
+    task_instance: TaskInstance,
+    task: BaseOperator,
+    task_uuid: str,
+):
+    return {
+        "airflow": json.loads(
+            json.dumps(
+                asdict(
+                    AirflowRunFacet(
+                        dag=DagInfo(dag),
+                        dagRun=DagRunInfo(dag_run),
+                        taskInstance=TaskInstanceInfo(task_instance),
+                        task=TaskInfo(task),
+                        taskUuid=task_uuid,
+                    )
+                ),
+                default=str,
+            )
+        )
+    }
+
+
+def get_dagrun_start_end(dagrun: DagRun, dag: DAG):
+    try:
+        return dagrun.data_interval_start, dagrun.data_interval_end
+    except AttributeError:
+        # Airflow < 2.2 before adding data interval
+        pass
+    start = dagrun.execution_date
+    end = dag.following_schedule(start)
+    return start, end or start
+
+
+def import_from_string(path: str):
+    try:
+        module_path, target = path.rsplit(".", 1)
+        module = importlib.import_module(module_path)
+        return getattr(module, target)
+    except Exception as e:
+        log.warning(e)
+        raise ImportError(f"Failed to import {path}") from e
+
+
+def try_import_from_string(path: str):
+    try:
+        return import_from_string(path)
+    except ImportError:
+        return None
+
+
+def redact_with_exclusions(source: Any):
+    try:
+        from airflow.utils.log.secrets_masker import (
+            _secrets_masker,
+            should_hide_value_for_key,
+        )
+    except ImportError:
+        return source
+    import copy
+
+    sm = copy.deepcopy(_secrets_masker())

Review Comment:
   Accessing `_secrets_masker` directly is a little bit naughty. Do you need to?



##########
airflow/providers/openlineage/plugins/macros.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.
+from __future__ import annotations
+
+import os
+import typing
+
+from airflow.providers.openlineage.plugins.adapter import OpenLineageAdapter
+
+if typing.TYPE_CHECKING:
+    from airflow.models import BaseOperator, TaskInstance
+
+_JOB_NAMESPACE = os.getenv("OPENLINEAGE_NAMESPACE", "default")
+
+
+def lineage_run_id(task: BaseOperator, task_instance: TaskInstance):
+    """
+    Macro function which returns the generated run id for a given task. This
+    can be used to forward the run id from a task to a child run so the job
+    hierarchy is preserved. Invoke as a jinja template, e.g.
+
+    PythonOperator(
+        task_id='render_template',
+        python_callable=my_task_function,
+        op_args=['{{ lineage_run_id(task, task_instance) }}'], # lineage_run_id macro invoked

Review Comment:
   Oh, all you seem to get out of the task is the task_id, which ti has anyway.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1159665956


##########
airflow/providers/openlineage/utils/converters.py:
##########
@@ -0,0 +1,33 @@
+# 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
+
+from airflow.lineage.entities import Table
+from openlineage.client.run import Dataset
+
+
+def convert_to_ol_dataset(obj):

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] eladkal commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "eladkal (via GitHub)" <gi...@apache.org>.
eladkal commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1160812862


##########
airflow/providers/openlineage/extractors/base.py:
##########
@@ -0,0 +1,106 @@
+# 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
+
+from abc import ABC, abstractmethod
+
+from attrs import Factory, define
+
+from airflow.utils.log.logging_mixin import LoggingMixin
+from openlineage.client.facet import BaseFacet
+from openlineage.client.run import Dataset
+
+
+@define
+class OperatorLineage:
+    """Structure returned from lineage extraction."""
+
+    inputs: list[Dataset] = Factory(list)
+    outputs: list[Dataset] = Factory(list)
+    run_facets: dict[str, BaseFacet] = Factory(dict)
+    job_facets: dict[str, BaseFacet] = Factory(dict)
+
+
+class BaseExtractor(ABC, LoggingMixin):
+    """
+    Abstract base extractor class.
+
+    This is used mostly to maintain support for custom extractors.
+    """
+
+    _allowed_query_params: list[str] = []
+
+    def __init__(self, operator):  # type: ignore
+        super().__init__()
+        self.operator = operator
+
+    @classmethod
+    @abstractmethod
+    def get_operator_classnames(cls) -> list[str]:
+        """
+        Implement this method returning list of operators that extractor works for.
+
+        Particularly, in Airflow 2 some operators are deprecated and simply subclass the new
+        implementation, for example airflow.contrib.operators.bigquery_operator.BigQueryOperator.
+        The BigQueryExtractor needs to work with both of them.

Review Comment:
   Please explain this.
   There is no `BigQueryOperator` in the code base.
   
   Deprecated operators and new versions of operators are backward compatible. They should not have special treatment.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] JDarDagran commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "JDarDagran (via GitHub)" <gi...@apache.org>.
JDarDagran commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1134147706


##########
airflow/providers/openlineage/plugins/macros.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.
+from __future__ import annotations
+
+import os
+import typing
+
+from airflow.providers.openlineage.plugins.adapter import OpenLineageAdapter
+
+if typing.TYPE_CHECKING:
+    from airflow.models import BaseOperator, TaskInstance
+
+_JOB_NAMESPACE = os.getenv("OPENLINEAGE_NAMESPACE", "default")
+
+
+def lineage_run_id(task: BaseOperator, task_instance: TaskInstance):
+    """
+    Macro function which returns the generated run id for a given task. This
+    can be used to forward the run id from a task to a child run so the job
+    hierarchy is preserved. Invoke as a jinja template, e.g.
+
+    PythonOperator(
+        task_id='render_template',
+        python_callable=my_task_function,
+        op_args=['{{ lineage_run_id(task, task_instance) }}'], # lineage_run_id macro invoked

Review Comment:
   You're rightt, `task_instance` should be enough in this case.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1128157295


##########
airflow/providers/openlineage/extractors/base.py:
##########
@@ -0,0 +1,133 @@
+# 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 json
+from abc import ABC, abstractmethod
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attr

Review Comment:
   🤔 
   ```
   The classic attr that powered the venerable [attr.s](https://www.attrs.org/en/stable/api.html#attr.s) and [attr.ib](https://www.attrs.org/en/stable/api.html#attr.ib)
   The modern attrs that only contains most modern APIs and relies on [attrs.define](https://www.attrs.org/en/stable/api.html#attrs.define) and [attrs.field](https://www.attrs.org/en/stable/api.html#attrs.field) to define your classes. Additionally it offers some attr APIs with nicer defaults (e.g. [attrs.asdict](https://www.attrs.org/en/stable/api.html#attrs.asdict)).
   ```
   Let's go with `attrs` then.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on pull request #29940: Add base OpenLineage provider implementation

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on PR #29940:
URL: https://github.com/apache/airflow/pull/29940#issuecomment-1483424096

   @julienledem  @mobuchowski  @bolkedebruin  
   
   If I may suggest, I think what works REALLY well for future references are ADRs (Architecture Decision Records). I have tried it in the past without much of the success for "new" things. But I found it works REALLY well, when you have an existing solution and either rewrite it or bring it to a new context, especially if you know the reasoning and learning from the past.
   
   This is what we've done when we re-wrote breeze in Python, I used the opportunity to capture all the decisions in the form of ADRs and I think it is great - for me but also for anyone who asks a question "why are we doing this and that". I used the "rewrite" opportunity to capture all the big decisions there. As the result, every time someone asked (or contested) some of the decisions, I was - pretty much always - able to send the person to the right ADR rather than explain it over and over again.
   
   Some links to follow:
   
   * Nice intro to ADRS: https://cognitect.com/blog/2011/11/15/documenting-architecture-decisions
   * Our ADRs for breeze: https://github.com/apache/airflow/tree/main/dev/breeze/doc/adr
   * I used this tool for ADR management: https://github.com/npryce/adr-tools (not really needed, ADRs are meant to be maintained "by hand", it just makes it a little easier to do it consistently).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1135409668


##########
airflow/providers/openlineage/plugins/openlineage.py:
##########
@@ -0,0 +1,46 @@
+# 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 airflow.plugins_manager import AirflowPlugin
+from airflow.providers.openlineage.plugins.macros import lineage_parent_id, lineage_run_id
+
+
+def _is_disabled():
+    return os.getenv("OPENLINEAGE_DISABLED", None) in [True, "true", "True"]
+
+
+if _is_disabled():  # type: ignore
+    # Provide empty plugin when OL is disabled
+    class OpenLineageProviderPlugin(AirflowPlugin):
+        """OpenLineage plugin that provides macros only"""
+
+        name = "OpenLineageProviderPlugin"
+        macros = [lineage_run_id, lineage_parent_id]
+
+else:
+    from airflow.providers.openlineage.plugins.listener import OpenLineageListener
+
+    # Provide entrypoint airflow plugin that registers listener module
+    class OpenLineageProviderPlugin(AirflowPlugin):  # type: ignore
+        """OpenLineage plugin that provides listener module and macros"""
+
+        name = "OpenLineageProviderPlugin"
+        listeners = [OpenLineageListener()]
+        macros = [lineage_run_id, lineage_parent_id]

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1135359042


##########
airflow/providers/openlineage/extractors/base.py:
##########
@@ -0,0 +1,133 @@
+# 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 json
+from abc import ABC, abstractmethod
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+from attrs import Factory, define
+
+from airflow.utils.log.logging_mixin import LoggingMixin
+from openlineage.client.facet import BaseFacet
+from openlineage.client.run import Dataset
+
+
+@define
+class OperatorLineage:
+    """Structure returned from lineage extraction."""
+
+    inputs: list[Dataset] = Factory(list)
+    outputs: list[Dataset] = Factory(list)
+    run_facets: dict[str, BaseFacet] = Factory(dict)
+    job_facets: dict[str, BaseFacet] = Factory(dict)
+
+
+class BaseExtractor(ABC, LoggingMixin):
+    """
+    Abstract base extractor class.
+
+    This is used mostly to maintain support for custom extractors.
+    """
+
+    _allowed_query_params: list[str] = []
+
+    def __init__(self, operator):
+        super().__init__()
+        self.operator = operator
+        self.patch()
+
+    def patch(self):
+        # Extractor should register extension methods or patches to operator here
+        pass
+
+    @classmethod
+    def get_operator_classnames(cls) -> list[str]:
+        """
+        Implement this method returning list of operators that extractor works for.
+        Particularly, in Airflow 2 some operators are deprecated and simply subclass the new
+        implementation, for example BigQueryOperator:
+        https://github.com/apache/airflow/blob/main/airflow/contrib/operators/bigquery_operator.py
+        The BigQueryExtractor needs to work with both of them.
+        :return:
+        """
+        raise NotImplementedError()
+
+    def validate(self):
+        assert self.operator.__class__.__name__ in self.get_operator_classnames()
+
+    @abstractmethod
+    def extract(self) -> OperatorLineage | None:
+        pass
+
+    def extract_on_complete(self, task_instance) -> OperatorLineage | None:
+        return self.extract()
+
+    @classmethod
+    def get_connection_uri(cls, conn):
+        """
+        Return the connection URI for the given ID. We first attempt to lookup
+        the connection URI via AIRFLOW_CONN_<conn_id>, else fallback on querying
+        the Airflow's connection table.

Review Comment:
   Changed this to `redacted_connection_uri` and added appropriate comment.
   
   This method uses `conn.get_uri()` to get connection URI, then removes potential username, password, and known `sensitive` query parts.  I've moved passing that second part to being provided by extractor/operator that calls it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] ashb commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "ashb (via GitHub)" <gi...@apache.org>.
ashb commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1138367004


##########
airflow/providers/openlineage/extractors/base.py:
##########
@@ -0,0 +1,133 @@
+# 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 json
+from abc import ABC, abstractmethod
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+from attrs import Factory, define
+
+from airflow.utils.log.logging_mixin import LoggingMixin
+from openlineage.client.facet import BaseFacet
+from openlineage.client.run import Dataset
+
+
+@define
+class OperatorLineage:
+    """Structure returned from lineage extraction."""
+
+    inputs: list[Dataset] = Factory(list)
+    outputs: list[Dataset] = Factory(list)
+    run_facets: dict[str, BaseFacet] = Factory(dict)
+    job_facets: dict[str, BaseFacet] = Factory(dict)
+
+
+class BaseExtractor(ABC, LoggingMixin):
+    """
+    Abstract base extractor class.
+
+    This is used mostly to maintain support for custom extractors.
+    """
+
+    _allowed_query_params: list[str] = []
+
+    def __init__(self, operator):
+        super().__init__()
+        self.operator = operator
+        self.patch()
+
+    def patch(self):
+        # Extractor should register extension methods or patches to operator here
+        pass
+
+    @classmethod

Review Comment:
   So it does.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1162996424


##########
docs/spelling_wordlist.txt:
##########
@@ -161,6 +161,7 @@ BestCandidate
 Bigquery
 bigquery
 BigQueryHook
+BigQueryOperator

Review Comment:
   Removed, as it's removed from comment.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1157621916


##########
airflow/providers/openlineage/utils/utils.py:
##########
@@ -0,0 +1,482 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from contextlib import suppress
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attrs
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def redacted_connection_uri(conn: Connection, filtered_params=None, filtered_prefixes=None):
+    """
+    Return the connection URI for the given Connection.
+    This method additionally filters URI by removing query parameters that are known to carry sensitive data
+    like username, password, access key.
+    """
+    if filtered_prefixes is None:
+        filtered_prefixes = []
+    if filtered_params is None:
+        filtered_params = []
+
+    def filter_key_params(k: str):
+        return k not in filtered_params and any(substr in k for substr in filtered_prefixes)
+
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not filter_key_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def get_normalized_postgres_connection_uri(conn):
+    """
+    URIs starting with postgresql:// and postgres:// are both valid
+    PostgreSQL connection strings. This function normalizes it to
+    postgres:// as canonical name according to OpenLineage spec.
+    """
+    uri = redacted_connection_uri(conn)
+    if uri.startswith("postgresql"):
+        uri = uri.replace("postgresql", "postgres", 1)
+    return uri
+
+
+def get_connection(conn_id) -> Connection | None:
+    from airflow.hooks.base import BaseHook
+
+    with suppress(Exception):
+        return BaseHook.get_connection(conn_id=conn_id)
+    return None
+
+
+def get_job_name(task):
+    return f"{task.dag_id}.{task.task_id}"
+
+
+def get_custom_facets(
+    dagrun, task, is_external_trigger: bool, task_instance: TaskInstance | None = None
+) -> dict[str, Any]:
+    custom_facets = {
+        "airflow_runArgs": AirflowRunArgsRunFacet(is_external_trigger),
+        "airflow_version": AirflowVersionRunFacet.from_dagrun_and_task(dagrun, task),
+    }
+    # check for -1 comes from SmartSensor compatibility with dynamic task mapping
+    # this comes from Airflow code
+    if hasattr(task_instance, "map_index") and getattr(task_instance, "map_index") != -1:
+        custom_facets["airflow_mappedTask"] = AirflowMappedTaskRunFacet.from_task_instance(task_instance)
+    return custom_facets
+
+
+class InfoJsonEncodable(dict):
+    """
+    Airflow objects might not be json-encodable overall.
+
+    The class provides additional attributes to control
+    what and how is encoded:
+    * renames: a dictionary of attribute name changes
+    * casts: a dictionary consisting of attribute names
+             and corresponding methods that should change
+             object value
+    * includes: list of attributes to be included in encoding
+    * excludes: list of attributes to be excluded from encoding
+
+    Don't use both includes and excludes.
+    """
+
+    renames: dict[str, str] = {}
+    casts: dict[str, Any] = {}
+    includes: list[str] = []
+    excludes: list[str] = []
+
+    def __init__(self, obj):
+        self.obj = obj
+        self._fields = []
+
+        self._cast_fields()
+        self._rename_fields()
+        self._include_fields()
+        dict.__init__(
+            self,
+            **{field: InfoJsonEncodable._cast_basic_types(getattr(self, field)) for field in self._fields},
+        )
+
+    @staticmethod
+    def _cast_basic_types(value):
+        if isinstance(value, datetime.datetime):
+            return value.isoformat()
+        if isinstance(value, (set, list, tuple)):
+            return str(list(value))
+        return value
+
+    def _rename_fields(self):
+        for field, renamed in self.renames.items():
+            if hasattr(self.obj, field):
+                setattr(self, renamed, getattr(self.obj, field))
+                self._fields.append(renamed)
+
+    def _cast_fields(self):
+        for field, func in self.casts.items():
+            setattr(self, field, func(self.obj))
+            self._fields.append(field)
+
+    def _include_fields(self):
+        if self.includes and self.excludes:
+            raise Exception("Don't use both includes and excludes.")
+        if self.includes:
+            for field in self.includes:
+                if field in self._fields or not hasattr(self.obj, field):
+                    continue
+                setattr(self, field, getattr(self.obj, field))
+                self._fields.append(field)
+        else:
+            for field, val in self.obj.__dict__.items():
+                if field in self._fields or field in self.excludes or field in self.renames:
+                    continue
+                setattr(self, field, val)
+                self._fields.append(field)
+
+
+class DagInfo(InfoJsonEncodable):
+    """Defines encoding DAG object to JSON."""
+
+    includes = ["dag_id", "schedule_interval", "tags", "start_date"]
+    casts = {"timetable": lambda dag: dag.timetable.serialize() if getattr(dag, "timetable", None) else None}
+    renames = {"_dag_id": "dag_id"}
+
+
+class DagRunInfo(InfoJsonEncodable):
+    """Defines encoding DagRun object to JSON."""
+
+    includes = [
+        "conf",
+        "dag_id",
+        "data_interval_start",
+        "data_interval_end",
+        "external_trigger",
+        "run_id",
+        "run_type",
+        "start_date",
+    ]
+
+
+class TaskInstanceInfo(InfoJsonEncodable):
+    """Defines encoding TaskInstance object to JSON."""
+
+    includes = ["duration", "try_number", "pool"]
+    casts = {
+        "map_index": lambda ti: ti.map_index
+        if hasattr(ti, "map_index") and getattr(ti, "map_index") != -1
+        else None
+    }
+
+
+class TaskInfo(InfoJsonEncodable):
+    """Defines encoding BaseOperator/AbstractOperator object to JSON."""
+
+    renames = {
+        "_BaseOperator__init_kwargs": "args",
+        "_BaseOperator__from_mapped": "mapped",
+        "_downstream_task_ids": "downstream_task_ids",
+        "_upstream_task_ids": "upstream_task_ids",
+    }
+    excludes = [
+        "_BaseOperator__instantiated",
+        "_dag",
+        "_hook",
+        "_log",
+        "_outlets",
+        "_inlets",
+        "_lock_for_execution",
+        "handler",
+        "params",
+        "python_callable",
+        "retry_delay",
+    ]
+    casts = {
+        "operator_class": lambda task: f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",  # noqa
+        "task_group": lambda task: TaskGroupInfo(task.task_group)
+        if hasattr(task, "task_group") and getattr(task.task_group, "_group_id", None)
+        else None,
+    }
+
+
+class TaskGroupInfo(InfoJsonEncodable):
+    """Defines encoding TaskGroup object to JSON."""
+
+    renames = {
+        "_group_id": "group_id",
+    }
+    includes = [
+        "downstream_group_ids",
+        "downstream_task_ids",
+        "prefix_group_id",
+        "tooltip",
+        "upstream_group_ids",
+        "upstream_task_ids",
+    ]
+
+
+def get_airflow_run_facet(
+    dag_run: DagRun,
+    dag: DAG,
+    task_instance: TaskInstance,
+    task: BaseOperator,
+    task_uuid: str,
+):
+    return {
+        "airflow": json.loads(
+            json.dumps(
+                asdict(
+                    AirflowRunFacet(
+                        dag=DagInfo(dag),
+                        dagRun=DagRunInfo(dag_run),
+                        taskInstance=TaskInstanceInfo(task_instance),
+                        task=TaskInfo(task),
+                        taskUuid=task_uuid,
+                    )
+                ),
+                default=str,
+            )
+        )
+    }
+
+
+def import_from_string(path: str):
+    try:
+        module_path, target = path.rsplit(".", 1)
+        module = importlib.import_module(module_path)
+        return getattr(module, target)
+    except Exception as e:
+        log.warning(e)
+        raise ImportError(f"Failed to import {path}") from e
+
+
+def try_import_from_string(path: str):
+    with suppress(ImportError):
+        return import_from_string(path)
+
+
+def redact_with_exclusions(source: Any):
+    """This function redacts sensitive data similar to SecretsMasker in Airflow logs.
+    The difference is that MAX_RECURSION_DEPTH is way higher - due to the structure of OL events
+    we need more depth.
+    Also, we allow data structures to specify data that needs not to be redacted by specifying
+    _skip_redact list.
+    """
+    try:
+        from airflow.utils.log.secrets_masker import (
+            _secrets_masker,
+            should_hide_value_for_key,
+        )
+    except ImportError:
+        return source
+
+    sm = _secrets_masker()
+    MAX_RECURSION_DEPTH = 20
+
+    def _redact(item, name: str | None, depth: int):
+        if depth > MAX_RECURSION_DEPTH:
+            return item
+        try:
+            if name and should_hide_value_for_key(name):
+                return sm._redact_all(item, depth)
+            if isinstance(item, dict):
+                return {
+                    dict_key: _redact(subval, name=dict_key, depth=(depth + 1))
+                    for dict_key, subval in item.items()
+                }
+            elif attrs.has(item.__class__) or (is_json_serializable(item) and hasattr(item, "__dict__")):
+                for dict_key, subval in item.__dict__.items():

Review Comment:
   For slotted classes, we can use `attrs.asdict(obj, recurse=False)`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] ashb commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "ashb (via GitHub)" <gi...@apache.org>.
ashb commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1152067626


##########
airflow/providers/openlineage/extractors/__init__.py:
##########
@@ -0,0 +1,23 @@
+# 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
+
+from airflow.providers.openlineage.extractors.base import BaseExtractor, OperatorLineage
+from airflow.providers.openlineage.extractors.manager import ExtractorManager
+
+__all__ = ["BaseExtractor", "OperatorLineage", "ExtractorManager"]  # type: ignore

Review Comment:
   ```suggestion
   __all__ = ["BaseExtractor", "OperatorLineage", "ExtractorManager"]
   ```
   Shouldn't be needed anymore



##########
airflow/providers/openlineage/extractors/base.py:
##########
@@ -0,0 +1,111 @@
+# 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
+
+from abc import ABC, abstractmethod
+
+from attrs import Factory, define
+
+from airflow.utils.log.logging_mixin import LoggingMixin
+from openlineage.client.facet import BaseFacet
+from openlineage.client.run import Dataset
+
+
+@define
+class OperatorLineage:
+    """Structure returned from lineage extraction."""
+
+    inputs: list[Dataset] = Factory(list)
+    outputs: list[Dataset] = Factory(list)
+    run_facets: dict[str, BaseFacet] = Factory(dict)
+    job_facets: dict[str, BaseFacet] = Factory(dict)
+
+
+class BaseExtractor(ABC, LoggingMixin):
+    """
+    Abstract base extractor class.
+
+    This is used mostly to maintain support for custom extractors.
+    """
+
+    _allowed_query_params: list[str] = []
+
+    def __init__(self, operator):
+        super().__init__()
+        self.operator = operator
+        self.patch()
+
+    def patch(self):
+        # Extractor should register extension methods or patches to operator here
+        pass
+
+    @classmethod
+    @abstractmethod
+    def get_operator_classnames(cls) -> list[str]:
+        """
+        Implement this method returning list of operators that extractor works for.

Review Comment:
   ```suggestion
           Implement this method returning list of operators that extractor works for.
           
   ```
   
   Good practice is to have a blank line should be between the synopsis/short desc/first sentence and the rest. It only matters if these docs get rendered via Sphinx, but when they do, it shows a nice short description in the list.



##########
airflow/providers/openlineage/extractors/manager.py:
##########
@@ -0,0 +1,150 @@
+# 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 airflow.providers.openlineage.extractors import BaseExtractor, OperatorLineage
+from airflow.providers.openlineage.extractors.base import DefaultExtractor
+from airflow.providers.openlineage.plugins.facets import (
+    UnknownOperatorAttributeRunFacet,
+    UnknownOperatorInstance,
+)
+from airflow.providers.openlineage.utils.utils import get_operator_class
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class ExtractorManager(LoggingMixin):
+    """Class abstracting management of custom extractors."""
+
+    def __init__(self):
+        super().__init__()
+        self.extractors: dict[str, type[BaseExtractor]] = {}
+        self.default_extractor = DefaultExtractor
+
+        # Comma-separated extractors in OPENLINEAGE_EXTRACTORS variable.
+        # Extractors should implement BaseExtractor
+        from airflow.providers.openlineage.utils import import_from_string
+
+        # TODO: use airflow config with OL backup
+        env_extractors = os.getenv("OPENLINEAGE_EXTRACTORS")
+        if env_extractors is not None:
+            for extractor in env_extractors.split(";"):
+                extractor: type[BaseExtractor] = import_from_string(extractor.strip())
+                for operator_class in extractor.get_operator_classnames():
+                    self.extractors[operator_class] = extractor
+
+    def add_extractor(self, operator: str, extractor: type[BaseExtractor]):
+        self.extractors[operator] = extractor
+
+    def extract_metadata(self, dagrun, task, complete: bool = False, task_instance=None) -> OperatorLineage:
+        extractor = self._get_extractor(task)
+        task_info = (
+            f"task_type={task.task_type} "
+            f"airflow_dag_id={task.dag_id} "
+            f"task_id={task.task_id} "
+            f"airflow_run_id={dagrun.run_id} "
+        )
+
+        if extractor:
+            # Extracting advanced metadata is only possible when extractor for particular operator
+            # is defined. Without it, we can't extract any input or output data.
+            try:
+                self.log.debug("Using extractor %s %s", extractor.__class__.__name__, str(task_info))
+                if complete:
+                    task_metadata = extractor.extract_on_complete(task_instance)
+                else:
+                    task_metadata = extractor.extract()
+
+                self.log.debug("Found task metadata for operation %s: %s", task.task_id, str(task_metadata))
+                if task_metadata:
+                    if (not task_metadata.inputs) and (not task_metadata.outputs):
+                        inlets = task.get_inlet_defs()
+                        outlets = task.get_outlet_defs()
+                        self.extract_inlets_and_outlets(task_metadata, inlets, outlets)
+
+                    return task_metadata
+
+            except Exception as e:
+                self.log.exception(
+                    "Failed to extract metadata using found extractor %s - %s %s", extractor, e, task_info
+                )
+        else:
+            self.log.debug("Unable to find an extractor %s", task_info)
+
+            # Only include the unkonwnSourceAttribute facet if there is no extractor
+            task_metadata = OperatorLineage(
+                run_facets={
+                    "unknownSourceAttribute": UnknownOperatorAttributeRunFacet(
+                        unknownItems=[
+                            UnknownOperatorInstance(
+                                name=get_operator_class(task).__name__,
+                                properties={attr: value for attr, value in task.__dict__.items()},
+                            )
+                        ]
+                    )
+                },
+            )
+            inlets = task.get_inlet_defs()
+            outlets = task.get_outlet_defs()

Review Comment:
   These are deprecated now
   ```suggestion
               inlets = task.inlets
               outlets = task.outlets
   ```
   
   (Added/changed in #25767, which was 2.4. So maybe this is fine as it its?



##########
airflow/providers/openlineage/extractors/manager.py:
##########
@@ -0,0 +1,150 @@
+# 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 airflow.providers.openlineage.extractors import BaseExtractor, OperatorLineage
+from airflow.providers.openlineage.extractors.base import DefaultExtractor
+from airflow.providers.openlineage.plugins.facets import (
+    UnknownOperatorAttributeRunFacet,
+    UnknownOperatorInstance,
+)
+from airflow.providers.openlineage.utils.utils import get_operator_class
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class ExtractorManager(LoggingMixin):
+    """Class abstracting management of custom extractors."""
+
+    def __init__(self):
+        super().__init__()
+        self.extractors: dict[str, type[BaseExtractor]] = {}
+        self.default_extractor = DefaultExtractor
+
+        # Comma-separated extractors in OPENLINEAGE_EXTRACTORS variable.
+        # Extractors should implement BaseExtractor
+        from airflow.providers.openlineage.utils import import_from_string
+
+        # TODO: use airflow config with OL backup
+        env_extractors = os.getenv("OPENLINEAGE_EXTRACTORS")

Review Comment:
   Just making a note so we don't forget to do this



##########
airflow/providers/openlineage/utils/utils.py:
##########
@@ -0,0 +1,482 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from contextlib import suppress
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attrs
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__

Review Comment:
   Duck-typing  is more the python way, and is likely more future-proof
   ```suggestion
       return getattr(task, "operator_class", task.__class__)
   ```



##########
airflow/providers/openlineage/utils/utils.py:
##########
@@ -0,0 +1,482 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from contextlib import suppress
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attrs
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,

Review Comment:
   What about timetables? What about Dataset-triggered dags?



##########
airflow/providers/openlineage/plugins/facets.py:
##########
@@ -0,0 +1,116 @@
+# 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
+
+from attrs import define
+
+from airflow.providers.openlineage import version as OPENLINEAGE_AIRFLOW_VERSION
+from airflow.version import version as AIRFLOW_VERSION
+from openlineage.client.facet import BaseFacet
+from openlineage.client.utils import RedactMixin
+
+
+@define(slots=False)
+class AirflowVersionRunFacet(BaseFacet):
+    """Run facet containing task and DAG info"""
+
+    operator: str
+    taskInfo: dict[str, object]
+    airflowVersion: str
+    openlineageAirflowVersion: str
+
+    _additional_skip_redact: list[str] = [
+        "operator",
+        "airflowVersion",
+        "openlineageAirflowVersion",
+    ]
+
+    @classmethod
+    def from_dagrun_and_task(cls, dagrun, task):
+        # task.__dict__ may contain values uncastable to str
+        from airflow.providers.openlineage.utils import get_operator_class, to_json_encodable
+
+        task_info = to_json_encodable(task)

Review Comment:
   This serializes every property of the task -- that seems very a bit overkill? I guess it's not too bad though as it only goes one level down, not recursively to inside things like `self.conn` if a task has it.



##########
airflow/providers/openlineage/utils/utils.py:
##########
@@ -0,0 +1,482 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from contextlib import suppress
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attrs
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG

Review Comment:
   Not a constant (which is what the case implies) so either leave the import as `DAG` or make it `AirflowDag` please



##########
airflow/providers/openlineage/utils/utils.py:
##########
@@ -0,0 +1,482 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from contextlib import suppress
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attrs
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def redacted_connection_uri(conn: Connection, filtered_params=None, filtered_prefixes=None):
+    """
+    Return the connection URI for the given Connection.
+    This method additionally filters URI by removing query parameters that are known to carry sensitive data
+    like username, password, access key.
+    """
+    if filtered_prefixes is None:
+        filtered_prefixes = []
+    if filtered_params is None:
+        filtered_params = []
+
+    def filter_key_params(k: str):
+        return k not in filtered_params and any(substr in k for substr in filtered_prefixes)
+
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not filter_key_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def get_normalized_postgres_connection_uri(conn):
+    """
+    URIs starting with postgresql:// and postgres:// are both valid
+    PostgreSQL connection strings. This function normalizes it to
+    postgres:// as canonical name according to OpenLineage spec.
+    """
+    uri = redacted_connection_uri(conn)
+    if uri.startswith("postgresql"):
+        uri = uri.replace("postgresql", "postgres", 1)
+    return uri
+
+
+def get_connection(conn_id) -> Connection | None:
+    from airflow.hooks.base import BaseHook
+
+    with suppress(Exception):
+        return BaseHook.get_connection(conn_id=conn_id)
+    return None
+
+
+def get_job_name(task):
+    return f"{task.dag_id}.{task.task_id}"
+
+
+def get_custom_facets(
+    dagrun, task, is_external_trigger: bool, task_instance: TaskInstance | None = None
+) -> dict[str, Any]:
+    custom_facets = {
+        "airflow_runArgs": AirflowRunArgsRunFacet(is_external_trigger),
+        "airflow_version": AirflowVersionRunFacet.from_dagrun_and_task(dagrun, task),
+    }
+    # check for -1 comes from SmartSensor compatibility with dynamic task mapping
+    # this comes from Airflow code
+    if hasattr(task_instance, "map_index") and getattr(task_instance, "map_index") != -1:
+        custom_facets["airflow_mappedTask"] = AirflowMappedTaskRunFacet.from_task_instance(task_instance)
+    return custom_facets
+
+
+class InfoJsonEncodable(dict):
+    """
+    Airflow objects might not be json-encodable overall.
+
+    The class provides additional attributes to control
+    what and how is encoded:
+    * renames: a dictionary of attribute name changes
+    * casts: a dictionary consisting of attribute names
+             and corresponding methods that should change
+             object value
+    * includes: list of attributes to be included in encoding
+    * excludes: list of attributes to be excluded from encoding
+
+    Don't use both includes and excludes.
+    """
+
+    renames: dict[str, str] = {}
+    casts: dict[str, Any] = {}
+    includes: list[str] = []
+    excludes: list[str] = []
+
+    def __init__(self, obj):
+        self.obj = obj
+        self._fields = []
+
+        self._cast_fields()
+        self._rename_fields()
+        self._include_fields()
+        dict.__init__(
+            self,
+            **{field: InfoJsonEncodable._cast_basic_types(getattr(self, field)) for field in self._fields},
+        )
+
+    @staticmethod
+    def _cast_basic_types(value):
+        if isinstance(value, datetime.datetime):
+            return value.isoformat()
+        if isinstance(value, (set, list, tuple)):
+            return str(list(value))
+        return value
+
+    def _rename_fields(self):
+        for field, renamed in self.renames.items():
+            if hasattr(self.obj, field):
+                setattr(self, renamed, getattr(self.obj, field))
+                self._fields.append(renamed)
+
+    def _cast_fields(self):
+        for field, func in self.casts.items():
+            setattr(self, field, func(self.obj))
+            self._fields.append(field)
+
+    def _include_fields(self):
+        if self.includes and self.excludes:
+            raise Exception("Don't use both includes and excludes.")
+        if self.includes:
+            for field in self.includes:
+                if field in self._fields or not hasattr(self.obj, field):
+                    continue
+                setattr(self, field, getattr(self.obj, field))
+                self._fields.append(field)
+        else:
+            for field, val in self.obj.__dict__.items():
+                if field in self._fields or field in self.excludes or field in self.renames:
+                    continue
+                setattr(self, field, val)
+                self._fields.append(field)
+
+
+class DagInfo(InfoJsonEncodable):
+    """Defines encoding DAG object to JSON."""
+
+    includes = ["dag_id", "schedule_interval", "tags", "start_date"]
+    casts = {"timetable": lambda dag: dag.timetable.serialize() if getattr(dag, "timetable", None) else None}
+    renames = {"_dag_id": "dag_id"}
+
+
+class DagRunInfo(InfoJsonEncodable):
+    """Defines encoding DagRun object to JSON."""
+
+    includes = [
+        "conf",
+        "dag_id",
+        "data_interval_start",
+        "data_interval_end",
+        "external_trigger",
+        "run_id",
+        "run_type",
+        "start_date",
+    ]
+
+
+class TaskInstanceInfo(InfoJsonEncodable):
+    """Defines encoding TaskInstance object to JSON."""
+
+    includes = ["duration", "try_number", "pool"]
+    casts = {
+        "map_index": lambda ti: ti.map_index
+        if hasattr(ti, "map_index") and getattr(ti, "map_index") != -1
+        else None
+    }
+
+
+class TaskInfo(InfoJsonEncodable):
+    """Defines encoding BaseOperator/AbstractOperator object to JSON."""
+
+    renames = {
+        "_BaseOperator__init_kwargs": "args",
+        "_BaseOperator__from_mapped": "mapped",
+        "_downstream_task_ids": "downstream_task_ids",
+        "_upstream_task_ids": "upstream_task_ids",
+    }
+    excludes = [
+        "_BaseOperator__instantiated",
+        "_dag",
+        "_hook",
+        "_log",
+        "_outlets",
+        "_inlets",
+        "_lock_for_execution",
+        "handler",
+        "params",
+        "python_callable",
+        "retry_delay",
+    ]
+    casts = {
+        "operator_class": lambda task: f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",  # noqa
+        "task_group": lambda task: TaskGroupInfo(task.task_group)
+        if hasattr(task, "task_group") and getattr(task.task_group, "_group_id", None)
+        else None,
+    }
+
+
+class TaskGroupInfo(InfoJsonEncodable):
+    """Defines encoding TaskGroup object to JSON."""
+
+    renames = {
+        "_group_id": "group_id",
+    }
+    includes = [
+        "downstream_group_ids",
+        "downstream_task_ids",
+        "prefix_group_id",
+        "tooltip",
+        "upstream_group_ids",
+        "upstream_task_ids",
+    ]
+
+
+def get_airflow_run_facet(
+    dag_run: DagRun,
+    dag: DAG,
+    task_instance: TaskInstance,
+    task: BaseOperator,
+    task_uuid: str,
+):
+    return {
+        "airflow": json.loads(
+            json.dumps(
+                asdict(
+                    AirflowRunFacet(
+                        dag=DagInfo(dag),
+                        dagRun=DagRunInfo(dag_run),
+                        taskInstance=TaskInstanceInfo(task_instance),
+                        task=TaskInfo(task),
+                        taskUuid=task_uuid,
+                    )
+                ),
+                default=str,
+            )
+        )
+    }
+
+
+def import_from_string(path: str):
+    try:
+        module_path, target = path.rsplit(".", 1)
+        module = importlib.import_module(module_path)
+        return getattr(module, target)
+    except Exception as e:
+        log.warning(e)
+        raise ImportError(f"Failed to import {path}") from e

Review Comment:
   The same as `from airflow.utils.module_loading import import_string`? Can we use that instead?



##########
airflow/providers/openlineage/extractors/base.py:
##########
@@ -0,0 +1,111 @@
+# 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
+
+from abc import ABC, abstractmethod
+
+from attrs import Factory, define
+
+from airflow.utils.log.logging_mixin import LoggingMixin
+from openlineage.client.facet import BaseFacet
+from openlineage.client.run import Dataset
+
+
+@define
+class OperatorLineage:
+    """Structure returned from lineage extraction."""
+
+    inputs: list[Dataset] = Factory(list)
+    outputs: list[Dataset] = Factory(list)
+    run_facets: dict[str, BaseFacet] = Factory(dict)
+    job_facets: dict[str, BaseFacet] = Factory(dict)
+
+
+class BaseExtractor(ABC, LoggingMixin):
+    """
+    Abstract base extractor class.
+
+    This is used mostly to maintain support for custom extractors.
+    """
+
+    _allowed_query_params: list[str] = []
+
+    def __init__(self, operator):

Review Comment:
   We should probably start adding typing hints to this. In this case I guess its?
   
   ```suggestion
       def __init__(self, operator: BaseOperator):
   ```



##########
tests/providers/openlineage/conftest.py:
##########
@@ -0,0 +1,72 @@
+# 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.
+
+# Copyright 2018-2023 contributors to the OpenLineage project
+# SPDX-License-Identifier: Apache-2.0
+from __future__ import annotations
+
+import logging
+import os
+from unittest.mock import patch
+
+import pytest
+
+log = logging.getLogger(__name__)
+
+
+@pytest.fixture(scope="function")
+def remove_redshift_conn():
+    if "REDSHIFT_CONN" in os.environ:
+        del os.environ["REDSHIFT_CONN"]
+    if "WRITE_SCHEMA" in os.environ:
+        del os.environ["WRITE_SCHEMA"]
+
+
+@pytest.fixture(scope="function")
+def we_module_env():
+    os.environ["REDSHIFT_CONN"] = "postgresql://user:password@host.io:1234/db"
+    os.environ["WRITE_SCHEMA"] = "testing"
+
+
+@pytest.fixture(scope="function")
+def dagbag():
+    log.debug("dagbag()")
+    os.environ["AIRFLOW__CORE__SQL_ALCHEMY_CONN"] = "sqlite://"

Review Comment:
   Airflow tests generally shouldn't set this -- Airflow will (I think) already be loaded, so this might not actually have any effect, but regardless, the airflow test suite sets up the DB.



##########
airflow/providers/openlineage/extractors/manager.py:
##########
@@ -0,0 +1,150 @@
+# 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 airflow.providers.openlineage.extractors import BaseExtractor, OperatorLineage
+from airflow.providers.openlineage.extractors.base import DefaultExtractor
+from airflow.providers.openlineage.plugins.facets import (
+    UnknownOperatorAttributeRunFacet,
+    UnknownOperatorInstance,
+)
+from airflow.providers.openlineage.utils.utils import get_operator_class
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class ExtractorManager(LoggingMixin):
+    """Class abstracting management of custom extractors."""
+
+    def __init__(self):
+        super().__init__()
+        self.extractors: dict[str, type[BaseExtractor]] = {}
+        self.default_extractor = DefaultExtractor
+
+        # Comma-separated extractors in OPENLINEAGE_EXTRACTORS variable.
+        # Extractors should implement BaseExtractor
+        from airflow.providers.openlineage.utils import import_from_string
+
+        # TODO: use airflow config with OL backup
+        env_extractors = os.getenv("OPENLINEAGE_EXTRACTORS")
+        if env_extractors is not None:
+            for extractor in env_extractors.split(";"):
+                extractor: type[BaseExtractor] = import_from_string(extractor.strip())
+                for operator_class in extractor.get_operator_classnames():
+                    self.extractors[operator_class] = extractor
+
+    def add_extractor(self, operator: str, extractor: type[BaseExtractor]):
+        self.extractors[operator] = extractor
+
+    def extract_metadata(self, dagrun, task, complete: bool = False, task_instance=None) -> OperatorLineage:
+        extractor = self._get_extractor(task)
+        task_info = (
+            f"task_type={task.task_type} "
+            f"airflow_dag_id={task.dag_id} "
+            f"task_id={task.task_id} "
+            f"airflow_run_id={dagrun.run_id} "
+        )
+
+        if extractor:
+            # Extracting advanced metadata is only possible when extractor for particular operator
+            # is defined. Without it, we can't extract any input or output data.
+            try:
+                self.log.debug("Using extractor %s %s", extractor.__class__.__name__, str(task_info))
+                if complete:
+                    task_metadata = extractor.extract_on_complete(task_instance)
+                else:
+                    task_metadata = extractor.extract()
+
+                self.log.debug("Found task metadata for operation %s: %s", task.task_id, str(task_metadata))
+                if task_metadata:
+                    if (not task_metadata.inputs) and (not task_metadata.outputs):
+                        inlets = task.get_inlet_defs()
+                        outlets = task.get_outlet_defs()
+                        self.extract_inlets_and_outlets(task_metadata, inlets, outlets)
+
+                    return task_metadata
+
+            except Exception as e:
+                self.log.exception(
+                    "Failed to extract metadata using found extractor %s - %s %s", extractor, e, task_info
+                )
+        else:
+            self.log.debug("Unable to find an extractor %s", task_info)
+
+            # Only include the unkonwnSourceAttribute facet if there is no extractor
+            task_metadata = OperatorLineage(
+                run_facets={
+                    "unknownSourceAttribute": UnknownOperatorAttributeRunFacet(
+                        unknownItems=[
+                            UnknownOperatorInstance(
+                                name=get_operator_class(task).__name__,
+                                properties={attr: value for attr, value in task.__dict__.items()},
+                            )
+                        ]
+                    )
+                },
+            )
+            inlets = task.get_inlet_defs()
+            outlets = task.get_outlet_defs()
+            self.extract_inlets_and_outlets(task_metadata, inlets, outlets)
+            return task_metadata
+
+        return OperatorLineage()
+
+    def _get_extractor_class(self, clazz: type) -> type[BaseExtractor] | None:
+        name = clazz.__name__
+        if name in self.extractors:
+            return self.extractors[name]
+
+        def method_exists(method_name):
+            method = getattr(clazz, method_name, None)
+            if method:
+                return callable(method)
+
+        if method_exists("get_openlineage_facets_on_start") or method_exists(
+            "get_openlineage_facets_on_complete"
+        ):
+            return self.default_extractor
+        return None
+
+    def _get_extractor(self, task) -> BaseExtractor | None:
+        # TODO: Re-enable in Extractor PR
+        # self.instantiate_abstract_extractors(task)
+        extractor = self._get_extractor_class(get_operator_class(task))
+        self.log.debug("extractor for %s is %s", task.__class__, extractor)

Review Comment:
   `get_operator_class` isn't needed, `task.task_type` should be used, so `_get_extractor_class` should take a classname, not a class.



##########
airflow/providers/openlineage/utils/converters.py:
##########
@@ -0,0 +1,33 @@
+# 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
+
+from airflow.lineage.entities import Table
+from openlineage.client.run import Dataset
+
+
+def convert_to_ol_dataset(obj):

Review Comment:
   This is currently(?) only called from extractors.manager -- could this be a method on there instead?



##########
airflow/providers/openlineage/extractors/base.py:
##########
@@ -0,0 +1,111 @@
+# 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
+
+from abc import ABC, abstractmethod
+
+from attrs import Factory, define
+
+from airflow.utils.log.logging_mixin import LoggingMixin
+from openlineage.client.facet import BaseFacet
+from openlineage.client.run import Dataset
+
+
+@define
+class OperatorLineage:
+    """Structure returned from lineage extraction."""
+
+    inputs: list[Dataset] = Factory(list)
+    outputs: list[Dataset] = Factory(list)
+    run_facets: dict[str, BaseFacet] = Factory(dict)
+    job_facets: dict[str, BaseFacet] = Factory(dict)
+
+
+class BaseExtractor(ABC, LoggingMixin):
+    """
+    Abstract base extractor class.
+
+    This is used mostly to maintain support for custom extractors.
+    """
+
+    _allowed_query_params: list[str] = []
+
+    def __init__(self, operator):
+        super().__init__()
+        self.operator = operator
+        self.patch()
+
+    def patch(self):
+        # Extractor should register extension methods or patches to operator here
+        pass
+
+    @classmethod
+    @abstractmethod
+    def get_operator_classnames(cls) -> list[str]:
+        """
+        Implement this method returning list of operators that extractor works for.
+        Particularly, in Airflow 2 some operators are deprecated and simply subclass the new
+        implementation, for example BigQueryOperator:
+        https://github.com/apache/airflow/blob/main/airflow/contrib/operators/bigquery_operator.py

Review Comment:
   Probably shouldn't link to `main` branch here, but a short ref sha else this link will not point to the right place in future



##########
airflow/providers/openlineage/utils/utils.py:
##########
@@ -0,0 +1,482 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from contextlib import suppress
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attrs
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def redacted_connection_uri(conn: Connection, filtered_params=None, filtered_prefixes=None):
+    """
+    Return the connection URI for the given Connection.
+    This method additionally filters URI by removing query parameters that are known to carry sensitive data
+    like username, password, access key.
+    """
+    if filtered_prefixes is None:
+        filtered_prefixes = []
+    if filtered_params is None:
+        filtered_params = []
+
+    def filter_key_params(k: str):
+        return k not in filtered_params and any(substr in k for substr in filtered_prefixes)
+
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not filter_key_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def get_normalized_postgres_connection_uri(conn):
+    """
+    URIs starting with postgresql:// and postgres:// are both valid
+    PostgreSQL connection strings. This function normalizes it to
+    postgres:// as canonical name according to OpenLineage spec.
+    """
+    uri = redacted_connection_uri(conn)
+    if uri.startswith("postgresql"):
+        uri = uri.replace("postgresql", "postgres", 1)
+    return uri
+
+
+def get_connection(conn_id) -> Connection | None:
+    from airflow.hooks.base import BaseHook
+
+    with suppress(Exception):
+        return BaseHook.get_connection(conn_id=conn_id)
+    return None
+
+
+def get_job_name(task):
+    return f"{task.dag_id}.{task.task_id}"

Review Comment:
   What about mapped tasks? Need to be handled here or not?



##########
airflow/providers/openlineage/utils/utils.py:
##########
@@ -0,0 +1,482 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from contextlib import suppress
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attrs
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def redacted_connection_uri(conn: Connection, filtered_params=None, filtered_prefixes=None):
+    """
+    Return the connection URI for the given Connection.
+    This method additionally filters URI by removing query parameters that are known to carry sensitive data
+    like username, password, access key.
+    """
+    if filtered_prefixes is None:
+        filtered_prefixes = []
+    if filtered_params is None:
+        filtered_params = []
+
+    def filter_key_params(k: str):
+        return k not in filtered_params and any(substr in k for substr in filtered_prefixes)
+
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not filter_key_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def get_normalized_postgres_connection_uri(conn):
+    """
+    URIs starting with postgresql:// and postgres:// are both valid
+    PostgreSQL connection strings. This function normalizes it to
+    postgres:// as canonical name according to OpenLineage spec.
+    """
+    uri = redacted_connection_uri(conn)
+    if uri.startswith("postgresql"):
+        uri = uri.replace("postgresql", "postgres", 1)
+    return uri
+
+
+def get_connection(conn_id) -> Connection | None:
+    from airflow.hooks.base import BaseHook
+
+    with suppress(Exception):

Review Comment:
   This seems very broad? Where is this used?



##########
airflow/providers/openlineage/utils/utils.py:
##########
@@ -0,0 +1,482 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from contextlib import suppress
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attrs
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:

Review Comment:
   Does any version of airflow have `task.file_path`? 🤔 



##########
airflow/providers/openlineage/provider.yaml:
##########
@@ -0,0 +1,42 @@
+# 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.
+
+---
+package-name: apache-airflow-providers-openlineage
+name: OpenLineage Airflow
+description: |
+  `OpenLineage <https://openlineage.io/>`__
+
+versions:
+  - 1.0.0
+
+dependencies:
+  - apache-airflow>=2.5.1
+  - apache-airflow-providers-common-sql>=1.3.1
+  - attrs>=22.2
+  - openlineage-integration-common>=0.20.6
+  - openlineage-python>=0.20.6
+
+integrations:
+  - integration-name: OpenLineage
+    external-doc-url: https://openlineage.io
+    logo: /integration-logos/openlineage/openlineage.svg
+    tags: [apache]

Review Comment:
   I don't think this should be `apache`, as OpenLineage is not an ASF project



##########
airflow/providers/openlineage/utils/utils.py:
##########
@@ -0,0 +1,482 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from contextlib import suppress
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attrs
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def redacted_connection_uri(conn: Connection, filtered_params=None, filtered_prefixes=None):
+    """
+    Return the connection URI for the given Connection.
+    This method additionally filters URI by removing query parameters that are known to carry sensitive data
+    like username, password, access key.
+    """
+    if filtered_prefixes is None:
+        filtered_prefixes = []
+    if filtered_params is None:
+        filtered_params = []
+
+    def filter_key_params(k: str):
+        return k not in filtered_params and any(substr in k for substr in filtered_prefixes)
+
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not filter_key_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def get_normalized_postgres_connection_uri(conn):
+    """
+    URIs starting with postgresql:// and postgres:// are both valid
+    PostgreSQL connection strings. This function normalizes it to
+    postgres:// as canonical name according to OpenLineage spec.
+    """
+    uri = redacted_connection_uri(conn)
+    if uri.startswith("postgresql"):
+        uri = uri.replace("postgresql", "postgres", 1)
+    return uri
+
+
+def get_connection(conn_id) -> Connection | None:
+    from airflow.hooks.base import BaseHook
+
+    with suppress(Exception):
+        return BaseHook.get_connection(conn_id=conn_id)
+    return None
+
+
+def get_job_name(task):
+    return f"{task.dag_id}.{task.task_id}"
+
+
+def get_custom_facets(
+    dagrun, task, is_external_trigger: bool, task_instance: TaskInstance | None = None
+) -> dict[str, Any]:
+    custom_facets = {
+        "airflow_runArgs": AirflowRunArgsRunFacet(is_external_trigger),
+        "airflow_version": AirflowVersionRunFacet.from_dagrun_and_task(dagrun, task),

Review Comment:
   I guess this is what OL expects? But the name of the facet doesn't match my expectations of what is contained in here



##########
tests/providers/openlineage/conftest.py:
##########
@@ -0,0 +1,72 @@
+# 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.
+
+# Copyright 2018-2023 contributors to the OpenLineage project
+# SPDX-License-Identifier: Apache-2.0
+from __future__ import annotations
+
+import logging
+import os
+from unittest.mock import patch
+
+import pytest
+
+log = logging.getLogger(__name__)
+
+
+@pytest.fixture(scope="function")
+def remove_redshift_conn():
+    if "REDSHIFT_CONN" in os.environ:
+        del os.environ["REDSHIFT_CONN"]
+    if "WRITE_SCHEMA" in os.environ:
+        del os.environ["WRITE_SCHEMA"]
+
+
+@pytest.fixture(scope="function")
+def we_module_env():
+    os.environ["REDSHIFT_CONN"] = "postgresql://user:password@host.io:1234/db"
+    os.environ["WRITE_SCHEMA"] = "testing"

Review Comment:
   I think this will work, and it'll automatically get cleared after the test function finishes.
   
   ```suggestion
   @pytest.fixture(scope="function")
   def we_module_env(monkeypatch):
       monkeypatch.setenv("REDSHIFT_CONN", "postgresql://user:password@host.io:1234/db")
       monkeypatch.setenv("WRITE_SCHEMA", "testing")
   ```



##########
airflow/providers/openlineage/plugins/listener.py:
##########
@@ -0,0 +1,189 @@
+# 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 logging
+from concurrent.futures import Executor, ThreadPoolExecutor
+from typing import TYPE_CHECKING
+
+from airflow.listeners import hookimpl
+from airflow.providers.openlineage.extractors import ExtractorManager
+from airflow.providers.openlineage.plugins.adapter import OpenLineageAdapter
+from airflow.providers.openlineage.utils import (
+    get_airflow_run_facet,
+    get_custom_facets,
+    get_job_name,
+    get_task_location,
+    print_exception,
+)
+
+if TYPE_CHECKING:
+    from sqlalchemy.orm import Session
+
+    from airflow.models import DagRun, TaskInstance
+
+
+class OpenLineageListener:
+    """
+    OpenLineage listener
+    Sends events on task instance and dag run starts, completes and failures.
+    """
+
+    def __init__(self):
+        self.log = logging.getLogger(__name__)
+        self.executor: Executor = None  # type: ignore
+        self.extractor_manager = ExtractorManager()
+        self.adapter = OpenLineageAdapter()
+
+    @hookimpl
+    def on_task_instance_running(
+        self, previous_state, task_instance: TaskInstance, session: Session  # This will always be QUEUED
+    ):
+        if not hasattr(task_instance, "task"):
+            self.log.warning(
+                f"No task set for TI object task_id: {task_instance.task_id} - "
+                f"dag_id: {task_instance.dag_id} - run_id {task_instance.run_id}"
+            )
+            return
+
+        self.log.debug("OpenLineage listener got notification about task instance start")
+        dagrun = task_instance.dag_run
+        task = task_instance.task
+        dag = task.dag
+
+        @print_exception
+        def on_running():
+            # that's a workaround to detect task running from deferred state
+            # we return here because Airflow 2.3 needs task from deferred state
+            if task_instance.next_method is not None:
+                return
+            parent_run_id = self.adapter.build_dag_run_id(dag.dag_id, dagrun.run_id)
+
+            task_uuid = self.adapter.build_task_instance_run_id(
+                task.task_id, task_instance.execution_date, task_instance.try_number
+            )
+
+            task_metadata = self.extractor_manager.extract_metadata(dagrun, task)
+
+            self.adapter.start_task(
+                run_id=task_uuid,
+                job_name=get_job_name(task),
+                job_description=dag.description,
+                event_time=task_instance.start_date.isoformat(),
+                parent_job_name=dag.dag_id,
+                parent_run_id=parent_run_id,
+                code_location=get_task_location(task),
+                nominal_start_time=dagrun.data_interval_start.isoformat(),
+                nominal_end_time=dagrun.data_interval_end.isoformat(),
+                owners=dag.owner.split(", "),
+                task=task_metadata,
+                run_facets={
+                    **task_metadata.run_facets,
+                    **get_custom_facets(dagrun, task, dagrun.external_trigger, task_instance),
+                    **get_airflow_run_facet(dagrun, dag, task_instance, task, task_uuid),
+                },
+            )
+
+        self.executor.submit(on_running)
+
+    @hookimpl
+    def on_task_instance_success(self, previous_state, task_instance: TaskInstance, session):
+        self.log.debug("OpenLineage listener got notification about task instance success")
+
+        dagrun = task_instance.dag_run
+        task = task_instance.task
+
+        task_uuid = OpenLineageAdapter.build_task_instance_run_id(
+            task.task_id, task_instance.execution_date, task_instance.try_number - 1
+        )
+
+        @print_exception
+        def on_success():
+            task_metadata = self.extractor_manager.extract_metadata(
+                dagrun, task, complete=True, task_instance=task_instance
+            )
+            self.adapter.complete_task(
+                run_id=task_uuid,
+                job_name=get_job_name(task),
+                end_time=task_instance.end_date.isoformat(),
+                task=task_metadata,
+            )
+
+        self.executor.submit(on_success)
+
+    @hookimpl
+    def on_task_instance_failed(self, previous_state, task_instance: TaskInstance, session):
+        self.log.debug("OpenLineage listener got notification about task instance failure")
+
+        dagrun = task_instance.dag_run
+        task = task_instance.task
+
+        task_uuid = OpenLineageAdapter.build_task_instance_run_id(
+            task.task_id, task_instance.execution_date, task_instance.try_number - 1
+        )
+
+        @print_exception
+        def on_failure():
+            task_metadata = self.extractor_manager.extract_metadata(
+                dagrun, task, complete=True, task_instance=task_instance
+            )
+
+            self.adapter.fail_task(
+                run_id=task_uuid,
+                job_name=get_job_name(task),
+                end_time=task_instance.end_date.isoformat(),
+                task=task_metadata,
+            )
+
+        self.executor.submit(on_failure)
+
+    @hookimpl
+    def on_starting(self, component):
+        self.log.debug("on_starting: %s", component.__class__.__name__)
+        self.executor = ThreadPoolExecutor(max_workers=8, thread_name_prefix="openlineage_")
+
+    @hookimpl
+    def before_stopping(self, component):
+        self.log.debug("before_stopping: %s", component.__class__.__name__)
+        self.executor.shutdown(wait=True)

Review Comment:
   Yes please. 5s? 30s?



##########
airflow/providers/openlineage/utils/utils.py:
##########
@@ -0,0 +1,482 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from contextlib import suppress
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attrs
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def redacted_connection_uri(conn: Connection, filtered_params=None, filtered_prefixes=None):
+    """
+    Return the connection URI for the given Connection.
+    This method additionally filters URI by removing query parameters that are known to carry sensitive data
+    like username, password, access key.
+    """
+    if filtered_prefixes is None:
+        filtered_prefixes = []
+    if filtered_params is None:
+        filtered_params = []
+
+    def filter_key_params(k: str):
+        return k not in filtered_params and any(substr in k for substr in filtered_prefixes)
+
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not filter_key_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def get_normalized_postgres_connection_uri(conn):
+    """
+    URIs starting with postgresql:// and postgres:// are both valid
+    PostgreSQL connection strings. This function normalizes it to
+    postgres:// as canonical name according to OpenLineage spec.
+    """
+    uri = redacted_connection_uri(conn)
+    if uri.startswith("postgresql"):
+        uri = uri.replace("postgresql", "postgres", 1)
+    return uri
+
+
+def get_connection(conn_id) -> Connection | None:
+    from airflow.hooks.base import BaseHook
+
+    with suppress(Exception):
+        return BaseHook.get_connection(conn_id=conn_id)
+    return None
+
+
+def get_job_name(task):
+    return f"{task.dag_id}.{task.task_id}"
+
+
+def get_custom_facets(
+    dagrun, task, is_external_trigger: bool, task_instance: TaskInstance | None = None
+) -> dict[str, Any]:
+    custom_facets = {
+        "airflow_runArgs": AirflowRunArgsRunFacet(is_external_trigger),
+        "airflow_version": AirflowVersionRunFacet.from_dagrun_and_task(dagrun, task),
+    }
+    # check for -1 comes from SmartSensor compatibility with dynamic task mapping
+    # this comes from Airflow code
+    if hasattr(task_instance, "map_index") and getattr(task_instance, "map_index") != -1:
+        custom_facets["airflow_mappedTask"] = AirflowMappedTaskRunFacet.from_task_instance(task_instance)
+    return custom_facets
+
+
+class InfoJsonEncodable(dict):
+    """
+    Airflow objects might not be json-encodable overall.
+
+    The class provides additional attributes to control
+    what and how is encoded:
+    * renames: a dictionary of attribute name changes
+    * casts: a dictionary consisting of attribute names
+             and corresponding methods that should change
+             object value
+    * includes: list of attributes to be included in encoding
+    * excludes: list of attributes to be excluded from encoding
+
+    Don't use both includes and excludes.
+    """
+
+    renames: dict[str, str] = {}
+    casts: dict[str, Any] = {}
+    includes: list[str] = []
+    excludes: list[str] = []
+
+    def __init__(self, obj):
+        self.obj = obj
+        self._fields = []
+
+        self._cast_fields()
+        self._rename_fields()
+        self._include_fields()
+        dict.__init__(
+            self,
+            **{field: InfoJsonEncodable._cast_basic_types(getattr(self, field)) for field in self._fields},
+        )
+
+    @staticmethod
+    def _cast_basic_types(value):
+        if isinstance(value, datetime.datetime):
+            return value.isoformat()
+        if isinstance(value, (set, list, tuple)):
+            return str(list(value))
+        return value
+
+    def _rename_fields(self):
+        for field, renamed in self.renames.items():
+            if hasattr(self.obj, field):
+                setattr(self, renamed, getattr(self.obj, field))
+                self._fields.append(renamed)
+
+    def _cast_fields(self):
+        for field, func in self.casts.items():
+            setattr(self, field, func(self.obj))
+            self._fields.append(field)
+
+    def _include_fields(self):
+        if self.includes and self.excludes:
+            raise Exception("Don't use both includes and excludes.")
+        if self.includes:
+            for field in self.includes:
+                if field in self._fields or not hasattr(self.obj, field):
+                    continue
+                setattr(self, field, getattr(self.obj, field))
+                self._fields.append(field)
+        else:
+            for field, val in self.obj.__dict__.items():
+                if field in self._fields or field in self.excludes or field in self.renames:
+                    continue
+                setattr(self, field, val)
+                self._fields.append(field)
+
+
+class DagInfo(InfoJsonEncodable):
+    """Defines encoding DAG object to JSON."""
+
+    includes = ["dag_id", "schedule_interval", "tags", "start_date"]
+    casts = {"timetable": lambda dag: dag.timetable.serialize() if getattr(dag, "timetable", None) else None}
+    renames = {"_dag_id": "dag_id"}
+
+
+class DagRunInfo(InfoJsonEncodable):
+    """Defines encoding DagRun object to JSON."""
+
+    includes = [
+        "conf",
+        "dag_id",
+        "data_interval_start",
+        "data_interval_end",
+        "external_trigger",
+        "run_id",
+        "run_type",
+        "start_date",
+    ]
+
+
+class TaskInstanceInfo(InfoJsonEncodable):
+    """Defines encoding TaskInstance object to JSON."""
+
+    includes = ["duration", "try_number", "pool"]
+    casts = {
+        "map_index": lambda ti: ti.map_index
+        if hasattr(ti, "map_index") and getattr(ti, "map_index") != -1
+        else None
+    }
+
+
+class TaskInfo(InfoJsonEncodable):
+    """Defines encoding BaseOperator/AbstractOperator object to JSON."""
+
+    renames = {
+        "_BaseOperator__init_kwargs": "args",
+        "_BaseOperator__from_mapped": "mapped",
+        "_downstream_task_ids": "downstream_task_ids",
+        "_upstream_task_ids": "upstream_task_ids",
+    }
+    excludes = [
+        "_BaseOperator__instantiated",
+        "_dag",
+        "_hook",
+        "_log",
+        "_outlets",
+        "_inlets",
+        "_lock_for_execution",
+        "handler",
+        "params",
+        "python_callable",
+        "retry_delay",
+    ]
+    casts = {
+        "operator_class": lambda task: f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",  # noqa
+        "task_group": lambda task: TaskGroupInfo(task.task_group)
+        if hasattr(task, "task_group") and getattr(task.task_group, "_group_id", None)
+        else None,
+    }
+
+
+class TaskGroupInfo(InfoJsonEncodable):
+    """Defines encoding TaskGroup object to JSON."""
+
+    renames = {
+        "_group_id": "group_id",
+    }
+    includes = [
+        "downstream_group_ids",
+        "downstream_task_ids",
+        "prefix_group_id",
+        "tooltip",
+        "upstream_group_ids",
+        "upstream_task_ids",
+    ]
+
+
+def get_airflow_run_facet(
+    dag_run: DagRun,
+    dag: DAG,
+    task_instance: TaskInstance,
+    task: BaseOperator,
+    task_uuid: str,
+):
+    return {
+        "airflow": json.loads(
+            json.dumps(
+                asdict(
+                    AirflowRunFacet(
+                        dag=DagInfo(dag),
+                        dagRun=DagRunInfo(dag_run),
+                        taskInstance=TaskInstanceInfo(task_instance),
+                        task=TaskInfo(task),
+                        taskUuid=task_uuid,
+                    )
+                ),
+                default=str,
+            )
+        )
+    }
+
+
+def import_from_string(path: str):
+    try:
+        module_path, target = path.rsplit(".", 1)
+        module = importlib.import_module(module_path)
+        return getattr(module, target)
+    except Exception as e:
+        log.warning(e)
+        raise ImportError(f"Failed to import {path}") from e
+
+
+def try_import_from_string(path: str):
+    with suppress(ImportError):
+        return import_from_string(path)
+
+
+def redact_with_exclusions(source: Any):
+    """This function redacts sensitive data similar to SecretsMasker in Airflow logs.
+    The difference is that MAX_RECURSION_DEPTH is way higher - due to the structure of OL events
+    we need more depth.
+    Also, we allow data structures to specify data that needs not to be redacted by specifying
+    _skip_redact list.
+    """
+    try:
+        from airflow.utils.log.secrets_masker import (
+            _secrets_masker,
+            should_hide_value_for_key,
+        )
+    except ImportError:
+        return source
+
+    sm = _secrets_masker()
+    MAX_RECURSION_DEPTH = 20
+
+    def _redact(item, name: str | None, depth: int):
+        if depth > MAX_RECURSION_DEPTH:
+            return item
+        try:
+            if name and should_hide_value_for_key(name):
+                return sm._redact_all(item, depth)
+            if isinstance(item, dict):
+                return {
+                    dict_key: _redact(subval, name=dict_key, depth=(depth + 1))
+                    for dict_key, subval in item.items()
+                }
+            elif attrs.has(item.__class__) or (is_json_serializable(item) and hasattr(item, "__dict__")):
+                for dict_key, subval in item.__dict__.items():

Review Comment:
   Attrs classes are not always going to have a `__dict__` -- does this matter?



##########
airflow/providers/openlineage/utils/utils.py:
##########
@@ -0,0 +1,482 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from contextlib import suppress
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attrs
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def redacted_connection_uri(conn: Connection, filtered_params=None, filtered_prefixes=None):
+    """
+    Return the connection URI for the given Connection.
+    This method additionally filters URI by removing query parameters that are known to carry sensitive data
+    like username, password, access key.
+    """
+    if filtered_prefixes is None:
+        filtered_prefixes = []
+    if filtered_params is None:
+        filtered_params = []
+
+    def filter_key_params(k: str):
+        return k not in filtered_params and any(substr in k for substr in filtered_prefixes)
+
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not filter_key_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def get_normalized_postgres_connection_uri(conn):
+    """
+    URIs starting with postgresql:// and postgres:// are both valid
+    PostgreSQL connection strings. This function normalizes it to
+    postgres:// as canonical name according to OpenLineage spec.
+    """
+    uri = redacted_connection_uri(conn)
+    if uri.startswith("postgresql"):
+        uri = uri.replace("postgresql", "postgres", 1)
+    return uri
+
+
+def get_connection(conn_id) -> Connection | None:
+    from airflow.hooks.base import BaseHook
+
+    with suppress(Exception):
+        return BaseHook.get_connection(conn_id=conn_id)
+    return None
+
+
+def get_job_name(task):
+    return f"{task.dag_id}.{task.task_id}"
+
+
+def get_custom_facets(
+    dagrun, task, is_external_trigger: bool, task_instance: TaskInstance | None = None
+) -> dict[str, Any]:
+    custom_facets = {
+        "airflow_runArgs": AirflowRunArgsRunFacet(is_external_trigger),
+        "airflow_version": AirflowVersionRunFacet.from_dagrun_and_task(dagrun, task),
+    }
+    # check for -1 comes from SmartSensor compatibility with dynamic task mapping
+    # this comes from Airflow code
+    if hasattr(task_instance, "map_index") and getattr(task_instance, "map_index") != -1:
+        custom_facets["airflow_mappedTask"] = AirflowMappedTaskRunFacet.from_task_instance(task_instance)
+    return custom_facets
+
+
+class InfoJsonEncodable(dict):
+    """
+    Airflow objects might not be json-encodable overall.
+
+    The class provides additional attributes to control
+    what and how is encoded:
+    * renames: a dictionary of attribute name changes
+    * casts: a dictionary consisting of attribute names
+             and corresponding methods that should change
+             object value
+    * includes: list of attributes to be included in encoding
+    * excludes: list of attributes to be excluded from encoding
+
+    Don't use both includes and excludes.
+    """
+
+    renames: dict[str, str] = {}
+    casts: dict[str, Any] = {}
+    includes: list[str] = []
+    excludes: list[str] = []
+
+    def __init__(self, obj):
+        self.obj = obj
+        self._fields = []
+
+        self._cast_fields()
+        self._rename_fields()
+        self._include_fields()
+        dict.__init__(
+            self,
+            **{field: InfoJsonEncodable._cast_basic_types(getattr(self, field)) for field in self._fields},
+        )
+
+    @staticmethod
+    def _cast_basic_types(value):
+        if isinstance(value, datetime.datetime):
+            return value.isoformat()
+        if isinstance(value, (set, list, tuple)):
+            return str(list(value))
+        return value
+
+    def _rename_fields(self):
+        for field, renamed in self.renames.items():
+            if hasattr(self.obj, field):
+                setattr(self, renamed, getattr(self.obj, field))
+                self._fields.append(renamed)
+
+    def _cast_fields(self):
+        for field, func in self.casts.items():
+            setattr(self, field, func(self.obj))
+            self._fields.append(field)
+
+    def _include_fields(self):
+        if self.includes and self.excludes:
+            raise Exception("Don't use both includes and excludes.")
+        if self.includes:
+            for field in self.includes:
+                if field in self._fields or not hasattr(self.obj, field):
+                    continue
+                setattr(self, field, getattr(self.obj, field))
+                self._fields.append(field)
+        else:
+            for field, val in self.obj.__dict__.items():
+                if field in self._fields or field in self.excludes or field in self.renames:
+                    continue
+                setattr(self, field, val)
+                self._fields.append(field)
+
+
+class DagInfo(InfoJsonEncodable):
+    """Defines encoding DAG object to JSON."""
+
+    includes = ["dag_id", "schedule_interval", "tags", "start_date"]
+    casts = {"timetable": lambda dag: dag.timetable.serialize() if getattr(dag, "timetable", None) else None}
+    renames = {"_dag_id": "dag_id"}
+
+
+class DagRunInfo(InfoJsonEncodable):
+    """Defines encoding DagRun object to JSON."""
+
+    includes = [
+        "conf",
+        "dag_id",
+        "data_interval_start",
+        "data_interval_end",
+        "external_trigger",
+        "run_id",
+        "run_type",
+        "start_date",
+    ]
+
+
+class TaskInstanceInfo(InfoJsonEncodable):
+    """Defines encoding TaskInstance object to JSON."""
+
+    includes = ["duration", "try_number", "pool"]
+    casts = {
+        "map_index": lambda ti: ti.map_index
+        if hasattr(ti, "map_index") and getattr(ti, "map_index") != -1
+        else None
+    }
+
+
+class TaskInfo(InfoJsonEncodable):
+    """Defines encoding BaseOperator/AbstractOperator object to JSON."""
+
+    renames = {
+        "_BaseOperator__init_kwargs": "args",
+        "_BaseOperator__from_mapped": "mapped",
+        "_downstream_task_ids": "downstream_task_ids",
+        "_upstream_task_ids": "upstream_task_ids",
+    }
+    excludes = [
+        "_BaseOperator__instantiated",
+        "_dag",
+        "_hook",
+        "_log",
+        "_outlets",
+        "_inlets",
+        "_lock_for_execution",
+        "handler",
+        "params",
+        "python_callable",
+        "retry_delay",
+    ]
+    casts = {
+        "operator_class": lambda task: f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",  # noqa
+        "task_group": lambda task: TaskGroupInfo(task.task_group)
+        if hasattr(task, "task_group") and getattr(task.task_group, "_group_id", None)
+        else None,
+    }
+
+
+class TaskGroupInfo(InfoJsonEncodable):
+    """Defines encoding TaskGroup object to JSON."""
+
+    renames = {
+        "_group_id": "group_id",
+    }
+    includes = [
+        "downstream_group_ids",
+        "downstream_task_ids",
+        "prefix_group_id",
+        "tooltip",
+        "upstream_group_ids",
+        "upstream_task_ids",
+    ]
+
+
+def get_airflow_run_facet(
+    dag_run: DagRun,
+    dag: DAG,
+    task_instance: TaskInstance,
+    task: BaseOperator,
+    task_uuid: str,
+):
+    return {
+        "airflow": json.loads(
+            json.dumps(
+                asdict(
+                    AirflowRunFacet(
+                        dag=DagInfo(dag),
+                        dagRun=DagRunInfo(dag_run),
+                        taskInstance=TaskInstanceInfo(task_instance),
+                        task=TaskInfo(task),
+                        taskUuid=task_uuid,
+                    )
+                ),
+                default=str,
+            )
+        )
+    }
+
+
+def import_from_string(path: str):
+    try:
+        module_path, target = path.rsplit(".", 1)
+        module = importlib.import_module(module_path)
+        return getattr(module, target)
+    except Exception as e:
+        log.warning(e)
+        raise ImportError(f"Failed to import {path}") from e
+
+
+def try_import_from_string(path: str):
+    with suppress(ImportError):
+        return import_from_string(path)
+
+
+def redact_with_exclusions(source: Any):
+    """This function redacts sensitive data similar to SecretsMasker in Airflow logs.
+    The difference is that MAX_RECURSION_DEPTH is way higher - due to the structure of OL events
+    we need more depth.
+    Also, we allow data structures to specify data that needs not to be redacted by specifying
+    _skip_redact list.
+    """
+    try:
+        from airflow.utils.log.secrets_masker import (
+            _secrets_masker,
+            should_hide_value_for_key,
+        )
+    except ImportError:
+        return source
+
+    sm = _secrets_masker()
+    MAX_RECURSION_DEPTH = 20
+
+    def _redact(item, name: str | None, depth: int):
+        if depth > MAX_RECURSION_DEPTH:
+            return item

Review Comment:
   I'd really love to find a way that doesn't end up with a second copy of this code -- espeically given this is a security-like control.



##########
tests/providers/openlineage/conftest.py:
##########
@@ -0,0 +1,72 @@
+# 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.
+
+# Copyright 2018-2023 contributors to the OpenLineage project
+# SPDX-License-Identifier: Apache-2.0
+from __future__ import annotations
+
+import logging
+import os
+from unittest.mock import patch
+
+import pytest
+
+log = logging.getLogger(__name__)
+
+
+@pytest.fixture(scope="function")
+def remove_redshift_conn():
+    if "REDSHIFT_CONN" in os.environ:
+        del os.environ["REDSHIFT_CONN"]
+    if "WRITE_SCHEMA" in os.environ:
+        del os.environ["WRITE_SCHEMA"]
+
+
+@pytest.fixture(scope="function")
+def we_module_env():
+    os.environ["REDSHIFT_CONN"] = "postgresql://user:password@host.io:1234/db"
+    os.environ["WRITE_SCHEMA"] = "testing"
+
+
+@pytest.fixture(scope="function")
+def dagbag():
+    log.debug("dagbag()")
+    os.environ["AIRFLOW__CORE__SQL_ALCHEMY_CONN"] = "sqlite://"
+    os.environ["OPENLINEAGE_NAMESPACE"] = "test-marquez"
+
+    import airflow.utils.db as db_utils
+    from airflow import settings
+
+    db_utils.resetdb(settings.RBAC)

Review Comment:
   This function is defined as
   ```
   def resetdb(session: Session = NEW_SESSION, skip_init: bool = False):
   ```
   
   `settings.RBAC` hasn't existed for a long time either. (like Since 2.0?!)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1156100152


##########
airflow/providers/openlineage/utils/utils.py:
##########
@@ -0,0 +1,482 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from contextlib import suppress
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attrs
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def redacted_connection_uri(conn: Connection, filtered_params=None, filtered_prefixes=None):
+    """
+    Return the connection URI for the given Connection.
+    This method additionally filters URI by removing query parameters that are known to carry sensitive data
+    like username, password, access key.
+    """
+    if filtered_prefixes is None:
+        filtered_prefixes = []
+    if filtered_params is None:
+        filtered_params = []
+
+    def filter_key_params(k: str):
+        return k not in filtered_params and any(substr in k for substr in filtered_prefixes)
+
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not filter_key_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def get_normalized_postgres_connection_uri(conn):
+    """
+    URIs starting with postgresql:// and postgres:// are both valid
+    PostgreSQL connection strings. This function normalizes it to
+    postgres:// as canonical name according to OpenLineage spec.
+    """
+    uri = redacted_connection_uri(conn)
+    if uri.startswith("postgresql"):
+        uri = uri.replace("postgresql", "postgres", 1)
+    return uri
+
+
+def get_connection(conn_id) -> Connection | None:
+    from airflow.hooks.base import BaseHook
+
+    with suppress(Exception):
+        return BaseHook.get_connection(conn_id=conn_id)
+    return None
+
+
+def get_job_name(task):
+    return f"{task.dag_id}.{task.task_id}"

Review Comment:
   We model mapped tasks as the same job - with facet providing additional info. https://github.com/apache/airflow/pull/29940/files#diff-9160b0ef03eafca04d898afdebfdbfba285f09ba7f37bb70a8b6cf592905078cR68
   
   This could be expanded, of course. We generally think it's better to emit more events and make lineage backends handle them, rather than limiting it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1156137008


##########
airflow/providers/openlineage/utils/utils.py:
##########
@@ -0,0 +1,482 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from contextlib import suppress
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attrs
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def redacted_connection_uri(conn: Connection, filtered_params=None, filtered_prefixes=None):
+    """
+    Return the connection URI for the given Connection.
+    This method additionally filters URI by removing query parameters that are known to carry sensitive data
+    like username, password, access key.
+    """
+    if filtered_prefixes is None:
+        filtered_prefixes = []
+    if filtered_params is None:
+        filtered_params = []
+
+    def filter_key_params(k: str):
+        return k not in filtered_params and any(substr in k for substr in filtered_prefixes)
+
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not filter_key_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def get_normalized_postgres_connection_uri(conn):
+    """
+    URIs starting with postgresql:// and postgres:// are both valid
+    PostgreSQL connection strings. This function normalizes it to
+    postgres:// as canonical name according to OpenLineage spec.
+    """
+    uri = redacted_connection_uri(conn)
+    if uri.startswith("postgresql"):
+        uri = uri.replace("postgresql", "postgres", 1)
+    return uri
+
+
+def get_connection(conn_id) -> Connection | None:
+    from airflow.hooks.base import BaseHook
+
+    with suppress(Exception):
+        return BaseHook.get_connection(conn_id=conn_id)
+    return None
+
+
+def get_job_name(task):
+    return f"{task.dag_id}.{task.task_id}"
+
+
+def get_custom_facets(
+    dagrun, task, is_external_trigger: bool, task_instance: TaskInstance | None = None
+) -> dict[str, Any]:
+    custom_facets = {
+        "airflow_runArgs": AirflowRunArgsRunFacet(is_external_trigger),
+        "airflow_version": AirflowVersionRunFacet.from_dagrun_and_task(dagrun, task),
+    }
+    # check for -1 comes from SmartSensor compatibility with dynamic task mapping
+    # this comes from Airflow code
+    if hasattr(task_instance, "map_index") and getattr(task_instance, "map_index") != -1:
+        custom_facets["airflow_mappedTask"] = AirflowMappedTaskRunFacet.from_task_instance(task_instance)
+    return custom_facets
+
+
+class InfoJsonEncodable(dict):
+    """
+    Airflow objects might not be json-encodable overall.
+
+    The class provides additional attributes to control
+    what and how is encoded:
+    * renames: a dictionary of attribute name changes
+    * casts: a dictionary consisting of attribute names
+             and corresponding methods that should change
+             object value
+    * includes: list of attributes to be included in encoding
+    * excludes: list of attributes to be excluded from encoding
+
+    Don't use both includes and excludes.
+    """
+
+    renames: dict[str, str] = {}
+    casts: dict[str, Any] = {}
+    includes: list[str] = []
+    excludes: list[str] = []
+
+    def __init__(self, obj):
+        self.obj = obj
+        self._fields = []
+
+        self._cast_fields()
+        self._rename_fields()
+        self._include_fields()
+        dict.__init__(
+            self,
+            **{field: InfoJsonEncodable._cast_basic_types(getattr(self, field)) for field in self._fields},
+        )
+
+    @staticmethod
+    def _cast_basic_types(value):
+        if isinstance(value, datetime.datetime):
+            return value.isoformat()
+        if isinstance(value, (set, list, tuple)):
+            return str(list(value))
+        return value
+
+    def _rename_fields(self):
+        for field, renamed in self.renames.items():
+            if hasattr(self.obj, field):
+                setattr(self, renamed, getattr(self.obj, field))
+                self._fields.append(renamed)
+
+    def _cast_fields(self):
+        for field, func in self.casts.items():
+            setattr(self, field, func(self.obj))
+            self._fields.append(field)
+
+    def _include_fields(self):
+        if self.includes and self.excludes:
+            raise Exception("Don't use both includes and excludes.")
+        if self.includes:
+            for field in self.includes:
+                if field in self._fields or not hasattr(self.obj, field):
+                    continue
+                setattr(self, field, getattr(self.obj, field))
+                self._fields.append(field)
+        else:
+            for field, val in self.obj.__dict__.items():
+                if field in self._fields or field in self.excludes or field in self.renames:
+                    continue
+                setattr(self, field, val)
+                self._fields.append(field)
+
+
+class DagInfo(InfoJsonEncodable):
+    """Defines encoding DAG object to JSON."""
+
+    includes = ["dag_id", "schedule_interval", "tags", "start_date"]
+    casts = {"timetable": lambda dag: dag.timetable.serialize() if getattr(dag, "timetable", None) else None}
+    renames = {"_dag_id": "dag_id"}
+
+
+class DagRunInfo(InfoJsonEncodable):
+    """Defines encoding DagRun object to JSON."""
+
+    includes = [
+        "conf",
+        "dag_id",
+        "data_interval_start",
+        "data_interval_end",
+        "external_trigger",
+        "run_id",
+        "run_type",
+        "start_date",
+    ]
+
+
+class TaskInstanceInfo(InfoJsonEncodable):
+    """Defines encoding TaskInstance object to JSON."""
+
+    includes = ["duration", "try_number", "pool"]
+    casts = {
+        "map_index": lambda ti: ti.map_index
+        if hasattr(ti, "map_index") and getattr(ti, "map_index") != -1
+        else None
+    }
+
+
+class TaskInfo(InfoJsonEncodable):
+    """Defines encoding BaseOperator/AbstractOperator object to JSON."""
+
+    renames = {
+        "_BaseOperator__init_kwargs": "args",
+        "_BaseOperator__from_mapped": "mapped",
+        "_downstream_task_ids": "downstream_task_ids",
+        "_upstream_task_ids": "upstream_task_ids",
+    }
+    excludes = [
+        "_BaseOperator__instantiated",
+        "_dag",
+        "_hook",
+        "_log",
+        "_outlets",
+        "_inlets",
+        "_lock_for_execution",
+        "handler",
+        "params",
+        "python_callable",
+        "retry_delay",
+    ]
+    casts = {
+        "operator_class": lambda task: f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",  # noqa
+        "task_group": lambda task: TaskGroupInfo(task.task_group)
+        if hasattr(task, "task_group") and getattr(task.task_group, "_group_id", None)
+        else None,
+    }
+
+
+class TaskGroupInfo(InfoJsonEncodable):
+    """Defines encoding TaskGroup object to JSON."""
+
+    renames = {
+        "_group_id": "group_id",
+    }
+    includes = [
+        "downstream_group_ids",
+        "downstream_task_ids",
+        "prefix_group_id",
+        "tooltip",
+        "upstream_group_ids",
+        "upstream_task_ids",
+    ]
+
+
+def get_airflow_run_facet(
+    dag_run: DagRun,
+    dag: DAG,
+    task_instance: TaskInstance,
+    task: BaseOperator,
+    task_uuid: str,
+):
+    return {
+        "airflow": json.loads(
+            json.dumps(
+                asdict(
+                    AirflowRunFacet(
+                        dag=DagInfo(dag),
+                        dagRun=DagRunInfo(dag_run),
+                        taskInstance=TaskInstanceInfo(task_instance),
+                        task=TaskInfo(task),
+                        taskUuid=task_uuid,
+                    )
+                ),
+                default=str,
+            )
+        )
+    }
+
+
+def import_from_string(path: str):
+    try:
+        module_path, target = path.rsplit(".", 1)
+        module = importlib.import_module(module_path)
+        return getattr(module, target)
+    except Exception as e:
+        log.warning(e)
+        raise ImportError(f"Failed to import {path}") from e
+
+
+def try_import_from_string(path: str):
+    with suppress(ImportError):
+        return import_from_string(path)
+
+
+def redact_with_exclusions(source: Any):
+    """This function redacts sensitive data similar to SecretsMasker in Airflow logs.
+    The difference is that MAX_RECURSION_DEPTH is way higher - due to the structure of OL events
+    we need more depth.
+    Also, we allow data structures to specify data that needs not to be redacted by specifying
+    _skip_redact list.
+    """
+    try:
+        from airflow.utils.log.secrets_masker import (
+            _secrets_masker,
+            should_hide_value_for_key,
+        )
+    except ImportError:
+        return source
+
+    sm = _secrets_masker()
+    MAX_RECURSION_DEPTH = 20
+
+    def _redact(item, name: str | None, depth: int):
+        if depth > MAX_RECURSION_DEPTH:
+            return item

Review Comment:
   We'd need ability to exclude attributes on external objects too - so something like magic variable or method that dictates which fields not to redact? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] JDarDagran commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "JDarDagran (via GitHub)" <gi...@apache.org>.
JDarDagran commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1131510697


##########
airflow/providers/openlineage/provider.yaml:
##########
@@ -0,0 +1,42 @@
+# 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.
+
+---
+package-name: apache-airflow-providers-openlineage
+name: OpenLineage Airflow
+description: |
+  `OpenLineage <https://openlineage.io/>`__
+
+versions:
+  - 1.0.0
+
+dependencies:
+  - apache-airflow>=2.5.1

Review Comment:
   This needs to be set as the most recent published Airflow version. Otherwise, it breaks on Airflow 2.3 installation and provider verification step. Ideally, we'd want to have it set to the first version released with any core changes needed to make OpenLineage provider working (not sure if it'll be 2.5.2 or 2.6.0)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] bolkedebruin commented on pull request #29940: Add base OpenLineage provider implementation

Posted by "bolkedebruin (via GitHub)" <gi...@apache.org>.
bolkedebruin commented on PR #29940:
URL: https://github.com/apache/airflow/pull/29940#issuecomment-1473616525

   Can I say that this integration needs extensive docs and architecture description (inside those docs)? It is quite opaque to me know how this works, why it needs workers at all, what it does to my running system to have those workers, how does it affect task runs, what if the code fails did my task fail? etc etc.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on pull request #29940: Add base OpenLineage provider implementation

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on PR #29940:
URL: https://github.com/apache/airflow/pull/29940#issuecomment-1473678754

   BTW. @bolkedebruin  I think the docs is not nearly enough (and sometimes too much docs is overwhelming).  The CI of ours is - I think - documented to the maximum extend possible - including state diagrams, context why we are doing what we are doing, architecture decision records describing the decisions, reference of parameters etc. etc. Still I am the SPOF there big time. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1135407190


##########
airflow/providers/openlineage/utils/__init__.py:
##########
@@ -0,0 +1,499 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def get_connection_uri(conn):
+    """
+    Return the connection URI for the given ID. We first attempt to lookup
+    the connection URI via AIRFLOW_CONN_<conn_id>, else fallback on querying
+    the Airflow's connection table.
+    """
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not _filtered_query_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def _filtered_query_params(k: str):
+    unfiltered_snowflake_keys = [
+        "extra__snowflake__warehouse",
+        "extra__snowflake__account",
+        "extra__snowflake__database",
+    ]

Review Comment:
   Removed it, will add to extractor/operator directly.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1135417063


##########
airflow/providers/openlineage/plugins/facets.py:
##########
@@ -0,0 +1,116 @@
+# 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
+
+from attrs import define, field
+
+from airflow.providers.openlineage import version as OPENLINEAGE_AIRFLOW_VERSION
+from airflow.version import version as AIRFLOW_VERSION
+from openlineage.client.facet import BaseFacet
+from openlineage.client.utils import RedactMixin
+
+
+@define(slots=False)
+class AirflowVersionRunFacet(BaseFacet):
+    """Run facet containing task and DAG info"""
+
+    operator: str = field()
+    taskInfo: dict[str, object] = field()
+    airflowVersion: str = field()
+    openlineageAirflowVersion: str = field()
+
+    _additional_skip_redact: list[str] = [
+        "operator",
+        "airflowVersion",
+        "openlineageAirflowVersion",
+    ]
+
+    @classmethod
+    def from_dagrun_and_task(cls, dagrun, task):
+        # task.__dict__ may contain values uncastable to str
+        from airflow.providers.openlineage.utils import get_operator_class, to_json_encodable
+
+        task_info = to_json_encodable(task)
+        task_info["dag_run"] = to_json_encodable(dagrun)
+
+        return cls(
+            operator=f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",
+            taskInfo=task_info,
+            airflowVersion=AIRFLOW_VERSION,
+            openlineageAirflowVersion=OPENLINEAGE_AIRFLOW_VERSION,
+        )
+
+
+@define(slots=False)
+class AirflowRunArgsRunFacet(BaseFacet):
+    """Run facet pointing if DAG was triggered manually"""
+
+    externalTrigger: bool = field(default=False)
+
+    _additional_skip_redact: list[str] = ["externalTrigger"]
+
+
+@define(slots=False)
+class AirflowMappedTaskRunFacet(BaseFacet):
+    """Run facet containing information about mapped tasks"""
+
+    mapIndex: int = field()
+    operatorClass: str = field()
+
+    _additional_skip_redact: list[str] = ["operatorClass"]
+
+    @classmethod
+    def from_task_instance(cls, task_instance):
+        task = task_instance.task
+        from airflow.providers.openlineage.utils import get_operator_class
+
+        return cls(
+            mapIndex=task_instance.map_index,
+            operatorClass=f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",
+        )
+
+
+@define(slots=False)
+class AirflowRunFacet(BaseFacet):
+    """Composite Airflow run facet."""
+
+    dag: dict = field()
+    dagRun: dict = field()
+    task: dict = field()
+    taskInstance: dict = field()
+    taskUuid: str = field()
+
+
+@define(slots=False)
+class UnknownOperatorInstance(RedactMixin):
+    """
+    Describes an unknown operator - specifies the (class) name of the operator
+    and its properties
+    """
+
+    name: str = field()
+    properties: dict[str, object] = field()
+    type: str = field(default="operator")

Review Comment:
   Done.



##########
airflow/providers/openlineage/plugins/facets.py:
##########
@@ -0,0 +1,116 @@
+# 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
+
+from attrs import define, field
+
+from airflow.providers.openlineage import version as OPENLINEAGE_AIRFLOW_VERSION
+from airflow.version import version as AIRFLOW_VERSION
+from openlineage.client.facet import BaseFacet
+from openlineage.client.utils import RedactMixin
+
+
+@define(slots=False)
+class AirflowVersionRunFacet(BaseFacet):
+    """Run facet containing task and DAG info"""
+
+    operator: str = field()
+    taskInfo: dict[str, object] = field()
+    airflowVersion: str = field()
+    openlineageAirflowVersion: str = field()
+
+    _additional_skip_redact: list[str] = [
+        "operator",
+        "airflowVersion",
+        "openlineageAirflowVersion",
+    ]
+
+    @classmethod
+    def from_dagrun_and_task(cls, dagrun, task):
+        # task.__dict__ may contain values uncastable to str
+        from airflow.providers.openlineage.utils import get_operator_class, to_json_encodable
+
+        task_info = to_json_encodable(task)
+        task_info["dag_run"] = to_json_encodable(dagrun)
+
+        return cls(
+            operator=f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",
+            taskInfo=task_info,
+            airflowVersion=AIRFLOW_VERSION,
+            openlineageAirflowVersion=OPENLINEAGE_AIRFLOW_VERSION,
+        )
+
+
+@define(slots=False)
+class AirflowRunArgsRunFacet(BaseFacet):
+    """Run facet pointing if DAG was triggered manually"""
+
+    externalTrigger: bool = field(default=False)
+
+    _additional_skip_redact: list[str] = ["externalTrigger"]
+
+
+@define(slots=False)
+class AirflowMappedTaskRunFacet(BaseFacet):
+    """Run facet containing information about mapped tasks"""
+
+    mapIndex: int = field()
+    operatorClass: str = field()

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] JDarDagran commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "JDarDagran (via GitHub)" <gi...@apache.org>.
JDarDagran commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1134072255


##########
airflow/providers/openlineage/extractors/extractors.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.
+
+from __future__ import annotations
+
+import os
+
+from airflow.providers.openlineage.extractors.base import BaseExtractor, DefaultExtractor
+
+
+class Extractors:
+    """
+    This exposes implemented extractors, while hiding ones that require additional, unmet
+    dependency. Patchers are a category of extractor that needs to hook up to operator's
+    internals during DAG creation.
+    """
+
+    def __init__(self):
+        # Do not expose extractors relying on external dependencies that are not installed
+        self.extractors = {}
+        self.default_extractor = DefaultExtractor
+
+        # Comma-separated extractors in OPENLINEAGE_EXTRACTORS variable.
+        # Extractors should implement BaseExtractor
+        from airflow.providers.openlineage.utils import import_from_string
+
+        env_extractors = os.getenv("OPENLINEAGE_EXTRACTORS")
+        if env_extractors is not None:
+            for extractor in env_extractors.split(";"):

Review Comment:
   `;` was supported in `openlineage-airflow`. I think that for a while we'd want to maintain support of custom extractors as it was. Eventually, we could split on `;` or `,`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] JDarDagran commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "JDarDagran (via GitHub)" <gi...@apache.org>.
JDarDagran commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1134072255


##########
airflow/providers/openlineage/extractors/extractors.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.
+
+from __future__ import annotations
+
+import os
+
+from airflow.providers.openlineage.extractors.base import BaseExtractor, DefaultExtractor
+
+
+class Extractors:
+    """
+    This exposes implemented extractors, while hiding ones that require additional, unmet
+    dependency. Patchers are a category of extractor that needs to hook up to operator's
+    internals during DAG creation.
+    """
+
+    def __init__(self):
+        # Do not expose extractors relying on external dependencies that are not installed
+        self.extractors = {}
+        self.default_extractor = DefaultExtractor
+
+        # Comma-separated extractors in OPENLINEAGE_EXTRACTORS variable.
+        # Extractors should implement BaseExtractor
+        from airflow.providers.openlineage.utils import import_from_string
+
+        env_extractors = os.getenv("OPENLINEAGE_EXTRACTORS")
+        if env_extractors is not None:
+            for extractor in env_extractors.split(";"):

Review Comment:
   `;` was supported in `openlineage-airflow`. I think we want to maintain suppot fo custom extractors as it was for a while. Eventually, we could split on `;` or `,`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1135303901


##########
airflow/providers/openlineage/extractors/base.py:
##########
@@ -0,0 +1,133 @@
+# 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 json
+from abc import ABC, abstractmethod
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+from attrs import Factory, define
+
+from airflow.utils.log.logging_mixin import LoggingMixin
+from openlineage.client.facet import BaseFacet
+from openlineage.client.run import Dataset
+
+
+@define
+class OperatorLineage:
+    """Structure returned from lineage extraction."""
+
+    inputs: list[Dataset] = Factory(list)
+    outputs: list[Dataset] = Factory(list)
+    run_facets: dict[str, BaseFacet] = Factory(dict)
+    job_facets: dict[str, BaseFacet] = Factory(dict)
+
+
+class BaseExtractor(ABC, LoggingMixin):
+    """
+    Abstract base extractor class.
+
+    This is used mostly to maintain support for custom extractors.
+    """
+
+    _allowed_query_params: list[str] = []
+
+    def __init__(self, operator):
+        super().__init__()
+        self.operator = operator
+        self.patch()
+
+    def patch(self):
+        # Extractor should register extension methods or patches to operator here
+        pass
+
+    @classmethod
+    def get_operator_classnames(cls) -> list[str]:
+        """
+        Implement this method returning list of operators that extractor works for.
+        Particularly, in Airflow 2 some operators are deprecated and simply subclass the new
+        implementation, for example BigQueryOperator:
+        https://github.com/apache/airflow/blob/main/airflow/contrib/operators/bigquery_operator.py
+        The BigQueryExtractor needs to work with both of them.
+        :return:
+        """
+        raise NotImplementedError()
+
+    def validate(self):
+        assert self.operator.__class__.__name__ in self.get_operator_classnames()

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] JDarDagran commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "JDarDagran (via GitHub)" <gi...@apache.org>.
JDarDagran commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1131510697


##########
airflow/providers/openlineage/provider.yaml:
##########
@@ -0,0 +1,42 @@
+# 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.
+
+---
+package-name: apache-airflow-providers-openlineage
+name: OpenLineage Airflow
+description: |
+  `OpenLineage <https://openlineage.io/>`__
+
+versions:
+  - 1.0.0
+
+dependencies:
+  - apache-airflow>=2.5.1

Review Comment:
   This needs to be set as the most recent published Airflow version. Ideally, we'd want to have it set to the first version released with any core changes needed to make OpenLineage provider working (not sure if it'll be 2.5.2 or 2.6.0)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1135406226


##########
airflow/providers/openlineage/utils/converters.py:
##########
@@ -0,0 +1,33 @@
+# 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
+
+from airflow.lineage.entities import Table
+from openlineage.client.run import Dataset
+
+
+def convert_to_dataset(obj):

Review Comment:
   Done



##########
airflow/providers/openlineage/utils/__init__.py:
##########
@@ -0,0 +1,499 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def get_connection_uri(conn):
+    """
+    Return the connection URI for the given ID. We first attempt to lookup
+    the connection URI via AIRFLOW_CONN_<conn_id>, else fallback on querying
+    the Airflow's connection table.
+    """
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not _filtered_query_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def _filtered_query_params(k: str):
+    unfiltered_snowflake_keys = [
+        "extra__snowflake__warehouse",
+        "extra__snowflake__account",
+        "extra__snowflake__database",
+    ]
+    filtered_key_substrings = [
+        "aws_access_key_id",
+        "aws_secret_access_key",
+        "extra__snowflake__",
+    ]
+    return k not in unfiltered_snowflake_keys and any(substr in k for substr in filtered_key_substrings)
+
+
+def get_normalized_postgres_connection_uri(conn):
+    """
+    URIs starting with postgresql:// and postgres:// are both valid
+    PostgreSQL connection strings. This function normalizes it to
+    postgres:// as canonical name according to OpenLineage spec.
+    """
+    uri = get_connection_uri(conn)
+    if uri.startswith("postgresql"):
+        uri = uri.replace("postgresql", "postgres", 1)
+    return uri
+
+
+def get_connection(conn_id) -> Connection | None:
+    from airflow.hooks.base import BaseHook
+
+    try:
+        return BaseHook.get_connection(conn_id=conn_id)
+    except Exception:
+        return None
+
+
+def get_job_name(task):
+    return f"{task.dag_id}.{task.task_id}"
+
+
+def get_custom_facets(
+    dagrun, task, is_external_trigger: bool, task_instance: TaskInstance | None = None
+) -> dict[str, Any]:
+    custom_facets = {
+        "airflow_runArgs": AirflowRunArgsRunFacet(is_external_trigger),
+        "airflow_version": AirflowVersionRunFacet.from_dagrun_and_task(dagrun, task),
+    }
+    # check for -1 comes from SmartSensor compatibility with dynamic task mapping
+    # this comes from Airflow code
+    if hasattr(task_instance, "map_index") and getattr(task_instance, "map_index") != -1:
+        custom_facets["airflow_mappedTask"] = AirflowMappedTaskRunFacet.from_task_instance(task_instance)
+    return custom_facets
+
+
+class InfoJsonEncodable(dict):
+    """
+    Airflow objects might not be json-encodable overall.
+
+    The class provides additional attributes to control
+    what and how is encoded:
+    * renames: a dictionary of attribute name changes
+    * casts: a dictionary consisting of attribute names
+             and corresponding methods that should change
+             object value
+    * includes: list of attributes to be included in encoding
+    * excludes: list of attributes to be excluded from encoding
+
+    Don't use both includes and excludes.
+    """
+
+    renames: dict[str, str] = {}
+    casts: dict[str, Any] = {}
+    includes: list[str] = []
+    excludes: list[str] = []
+
+    def __init__(self, obj):
+        self.obj = obj
+        self._fields = []
+
+        self._cast_fields()
+        self._rename_fields()
+        self._include_fields()
+        dict.__init__(
+            self,
+            **{field: InfoJsonEncodable._cast_basic_types(getattr(self, field)) for field in self._fields},
+        )
+
+    @staticmethod
+    def _cast_basic_types(value):
+        if isinstance(value, datetime.datetime):
+            return value.isoformat()
+        if isinstance(value, (set, list, tuple)):
+            return str(list(value))
+        return value
+
+    def _rename_fields(self):
+        for field, renamed in self.renames.items():
+            if hasattr(self.obj, field):
+                setattr(self, renamed, getattr(self.obj, field))
+                self._fields.append(renamed)
+
+    def _cast_fields(self):
+        for field, func in self.casts.items():
+            setattr(self, field, func(self.obj))
+            self._fields.append(field)
+
+    def _include_fields(self):
+        if self.includes and self.excludes:
+            raise Exception("Don't use both includes and excludes.")
+        if self.includes:
+            for field in self.includes:
+                if field in self._fields or not hasattr(self.obj, field):
+                    continue
+                setattr(self, field, getattr(self.obj, field))
+                self._fields.append(field)
+        else:
+            for field, val in self.obj.__dict__.items():
+                if field in self._fields or field in self.excludes or field in self.renames:
+                    continue
+                setattr(self, field, val)
+                self._fields.append(field)
+
+
+class DagInfo(InfoJsonEncodable):
+    """Defines encoding DAG object to JSON."""
+
+    includes = ["dag_id", "schedule_interval", "tags", "start_date"]
+    casts = {"timetable": lambda dag: dag.timetable.serialize() if getattr(dag, "timetable", None) else None}
+    renames = {"_dag_id": "dag_id"}
+
+
+class DagRunInfo(InfoJsonEncodable):
+    """Defines encoding DagRun object to JSON."""
+
+    includes = [
+        "conf",
+        "dag_id",
+        "data_interval_start",
+        "data_interval_end",
+        "external_trigger",
+        "run_id",
+        "run_type",
+        "start_date",
+    ]
+
+
+class TaskInstanceInfo(InfoJsonEncodable):
+    """Defines encoding TaskInstance object to JSON."""
+
+    includes = ["duration", "try_number", "pool"]
+    casts = {
+        "map_index": lambda ti: ti.map_index
+        if hasattr(ti, "map_index") and getattr(ti, "map_index") != -1
+        else None
+    }
+
+
+class TaskInfo(InfoJsonEncodable):
+    """Defines encoding BaseOperator/AbstractOperator object to JSON."""
+
+    renames = {
+        "_BaseOperator__init_kwargs": "args",
+        "_BaseOperator__from_mapped": "mapped",
+        "_downstream_task_ids": "downstream_task_ids",
+        "_upstream_task_ids": "upstream_task_ids",
+    }
+    excludes = [
+        "_BaseOperator__instantiated",
+        "_dag",
+        "_hook",
+        "_log",
+        "_outlets",
+        "_inlets",
+        "_lock_for_execution",
+        "handler",
+        "params",
+        "python_callable",
+        "retry_delay",
+    ]
+    casts = {
+        "operator_class": lambda task: f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",  # noqa
+        "task_group": lambda task: TaskGroupInfo(task.task_group)
+        if hasattr(task, "task_group") and getattr(task.task_group, "_group_id", None)
+        else None,
+    }
+
+
+class TaskGroupInfo(InfoJsonEncodable):
+    """Defines encoding TaskGroup object to JSON."""
+
+    renames = {
+        "_group_id": "group_id",
+    }
+    includes = [
+        "downstream_group_ids",
+        "downstream_task_ids",
+        "prefix_group_id",
+        "tooltip",
+        "upstream_group_ids",
+        "upstream_task_ids",
+    ]
+
+
+def get_airflow_run_facet(
+    dag_run: DagRun,
+    dag: DAG,
+    task_instance: TaskInstance,
+    task: BaseOperator,
+    task_uuid: str,
+):
+    return {
+        "airflow": json.loads(
+            json.dumps(
+                asdict(
+                    AirflowRunFacet(
+                        dag=DagInfo(dag),
+                        dagRun=DagRunInfo(dag_run),
+                        taskInstance=TaskInstanceInfo(task_instance),
+                        task=TaskInfo(task),
+                        taskUuid=task_uuid,
+                    )
+                ),
+                default=str,
+            )
+        )
+    }
+
+
+def get_dagrun_start_end(dagrun: DagRun, dag: DAG):
+    try:
+        return dagrun.data_interval_start, dagrun.data_interval_end
+    except AttributeError:
+        # Airflow < 2.2 before adding data interval
+        pass
+    start = dagrun.execution_date
+    end = dag.following_schedule(start)
+    return start, end or start
+
+
+def import_from_string(path: str):
+    try:
+        module_path, target = path.rsplit(".", 1)
+        module = importlib.import_module(module_path)
+        return getattr(module, target)
+    except Exception as e:
+        log.warning(e)
+        raise ImportError(f"Failed to import {path}") from e
+
+
+def try_import_from_string(path: str):
+    try:
+        return import_from_string(path)
+    except ImportError:
+        return None
+
+
+def redact_with_exclusions(source: Any):
+    try:
+        from airflow.utils.log.secrets_masker import (
+            _secrets_masker,
+            should_hide_value_for_key,
+        )
+    except ImportError:
+        return source
+    import copy
+
+    sm = copy.deepcopy(_secrets_masker())
+    MAX_RECURSION_DEPTH = 20
+
+    def _redact(item, name: str | None, depth: int):
+        if depth > MAX_RECURSION_DEPTH:
+            return item
+        try:
+            if name and should_hide_value_for_key(name):
+                return sm._redact_all(item, depth)
+            if isinstance(item, dict):
+                return {
+                    dict_key: _redact(subval, name=dict_key, depth=(depth + 1))
+                    for dict_key, subval in item.items()
+                }
+            elif is_dataclass(item) or (is_json_serializable(item) and hasattr(item, "__dict__")):
+                for dict_key, subval in item.__dict__.items():
+                    if _is_name_redactable(dict_key, item):
+                        setattr(
+                            item,
+                            dict_key,
+                            _redact(subval, name=dict_key, depth=(depth + 1)),
+                        )
+                return item
+            elif isinstance(item, str):
+                if sm.replacer:
+                    return sm.replacer.sub("***", item)
+                return item
+            elif isinstance(item, (tuple, set)):
+                return tuple(_redact(subval, name=None, depth=(depth + 1)) for subval in item)
+            elif isinstance(item, list):
+                return [_redact(subval, name=None, depth=(depth + 1)) for subval in item]
+            else:
+                return item
+        except Exception as e:
+            log.warning(
+                "Unable to redact %s" "Error was: %s: %s",
+                repr(item),
+                type(e).__name__,
+                str(e),
+            )
+            return item
+
+    return _redact(source, name=None, depth=0)
+
+
+def is_dataclass(item):
+    return getattr(item.__class__, "__attrs_attrs__", None) is not None

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1135407400


##########
airflow/providers/openlineage/utils/__init__.py:
##########
@@ -0,0 +1,499 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def get_connection_uri(conn):

Review Comment:
   Removed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1155876746


##########
airflow/providers/openlineage/extractors/manager.py:
##########
@@ -0,0 +1,150 @@
+# 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 airflow.providers.openlineage.extractors import BaseExtractor, OperatorLineage
+from airflow.providers.openlineage.extractors.base import DefaultExtractor
+from airflow.providers.openlineage.plugins.facets import (
+    UnknownOperatorAttributeRunFacet,
+    UnknownOperatorInstance,
+)
+from airflow.providers.openlineage.utils.utils import get_operator_class
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class ExtractorManager(LoggingMixin):
+    """Class abstracting management of custom extractors."""
+
+    def __init__(self):
+        super().__init__()
+        self.extractors: dict[str, type[BaseExtractor]] = {}
+        self.default_extractor = DefaultExtractor
+
+        # Comma-separated extractors in OPENLINEAGE_EXTRACTORS variable.
+        # Extractors should implement BaseExtractor
+        from airflow.providers.openlineage.utils import import_from_string
+
+        # TODO: use airflow config with OL backup
+        env_extractors = os.getenv("OPENLINEAGE_EXTRACTORS")
+        if env_extractors is not None:
+            for extractor in env_extractors.split(";"):
+                extractor: type[BaseExtractor] = import_from_string(extractor.strip())
+                for operator_class in extractor.get_operator_classnames():
+                    self.extractors[operator_class] = extractor
+
+    def add_extractor(self, operator: str, extractor: type[BaseExtractor]):
+        self.extractors[operator] = extractor
+
+    def extract_metadata(self, dagrun, task, complete: bool = False, task_instance=None) -> OperatorLineage:
+        extractor = self._get_extractor(task)
+        task_info = (
+            f"task_type={task.task_type} "
+            f"airflow_dag_id={task.dag_id} "
+            f"task_id={task.task_id} "
+            f"airflow_run_id={dagrun.run_id} "
+        )
+
+        if extractor:
+            # Extracting advanced metadata is only possible when extractor for particular operator
+            # is defined. Without it, we can't extract any input or output data.
+            try:
+                self.log.debug("Using extractor %s %s", extractor.__class__.__name__, str(task_info))
+                if complete:
+                    task_metadata = extractor.extract_on_complete(task_instance)
+                else:
+                    task_metadata = extractor.extract()
+
+                self.log.debug("Found task metadata for operation %s: %s", task.task_id, str(task_metadata))
+                if task_metadata:
+                    if (not task_metadata.inputs) and (not task_metadata.outputs):
+                        inlets = task.get_inlet_defs()
+                        outlets = task.get_outlet_defs()
+                        self.extract_inlets_and_outlets(task_metadata, inlets, outlets)
+
+                    return task_metadata
+
+            except Exception as e:
+                self.log.exception(
+                    "Failed to extract metadata using found extractor %s - %s %s", extractor, e, task_info
+                )
+        else:
+            self.log.debug("Unable to find an extractor %s", task_info)
+
+            # Only include the unkonwnSourceAttribute facet if there is no extractor
+            task_metadata = OperatorLineage(
+                run_facets={
+                    "unknownSourceAttribute": UnknownOperatorAttributeRunFacet(
+                        unknownItems=[
+                            UnknownOperatorInstance(
+                                name=get_operator_class(task).__name__,
+                                properties={attr: value for attr, value in task.__dict__.items()},
+                            )
+                        ]
+                    )
+                },
+            )
+            inlets = task.get_inlet_defs()
+            outlets = task.get_outlet_defs()
+            self.extract_inlets_and_outlets(task_metadata, inlets, outlets)
+            return task_metadata
+
+        return OperatorLineage()
+
+    def _get_extractor_class(self, clazz: type) -> type[BaseExtractor] | None:
+        name = clazz.__name__
+        if name in self.extractors:
+            return self.extractors[name]
+
+        def method_exists(method_name):
+            method = getattr(clazz, method_name, None)
+            if method:
+                return callable(method)
+
+        if method_exists("get_openlineage_facets_on_start") or method_exists(
+            "get_openlineage_facets_on_complete"
+        ):
+            return self.default_extractor
+        return None
+
+    def _get_extractor(self, task) -> BaseExtractor | None:
+        # TODO: Re-enable in Extractor PR
+        # self.instantiate_abstract_extractors(task)
+        extractor = self._get_extractor_class(get_operator_class(task))
+        self.log.debug("extractor for %s is %s", task.__class__, extractor)

Review Comment:
   Yeah - we won't have a problem with `DecoratedMappedOperator` or `MappedOperator` in 2.6



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] ashb commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "ashb (via GitHub)" <gi...@apache.org>.
ashb commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1159791665


##########
airflow/providers/openlineage/extractors/manager.py:
##########
@@ -0,0 +1,150 @@
+# 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 airflow.providers.openlineage.extractors import BaseExtractor, OperatorLineage
+from airflow.providers.openlineage.extractors.base import DefaultExtractor
+from airflow.providers.openlineage.plugins.facets import (
+    UnknownOperatorAttributeRunFacet,
+    UnknownOperatorInstance,
+)
+from airflow.providers.openlineage.utils.utils import get_operator_class
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class ExtractorManager(LoggingMixin):
+    """Class abstracting management of custom extractors."""
+
+    def __init__(self):
+        super().__init__()
+        self.extractors: dict[str, type[BaseExtractor]] = {}
+        self.default_extractor = DefaultExtractor
+
+        # Comma-separated extractors in OPENLINEAGE_EXTRACTORS variable.
+        # Extractors should implement BaseExtractor
+        from airflow.providers.openlineage.utils import import_from_string
+
+        # TODO: use airflow config with OL backup
+        env_extractors = os.getenv("OPENLINEAGE_EXTRACTORS")

Review Comment:
   https://github.com/apache/airflow/issues/29670



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] JDarDagran commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "JDarDagran (via GitHub)" <gi...@apache.org>.
JDarDagran commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1134089174


##########
airflow/providers/openlineage/extractors/extractors.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.
+
+from __future__ import annotations
+
+import os
+
+from airflow.providers.openlineage.extractors.base import BaseExtractor, DefaultExtractor
+
+
+class Extractors:
+    """
+    This exposes implemented extractors, while hiding ones that require additional, unmet
+    dependency. Patchers are a category of extractor that needs to hook up to operator's
+    internals during DAG creation.
+    """
+
+    def __init__(self):
+        # Do not expose extractors relying on external dependencies that are not installed
+        self.extractors = {}
+        self.default_extractor = DefaultExtractor
+
+        # Comma-separated extractors in OPENLINEAGE_EXTRACTORS variable.
+        # Extractors should implement BaseExtractor
+        from airflow.providers.openlineage.utils import import_from_string
+
+        env_extractors = os.getenv("OPENLINEAGE_EXTRACTORS")

Review Comment:
   Actually, we have a series of issues opened linked to AIP-53, one of them is #29670. Along with that we'd like to integration OpenLineage configuration into Airflow config.
   Besides that, it's also a matter of backwards compatibility for `openlineage-airflow` (current OL-Airflow integration outside of Airflow provider) in which you could put custom extractors in this env var.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1135351614


##########
airflow/providers/openlineage/extractors/__init__.py:
##########
@@ -0,0 +1,24 @@
+# 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
+
+from airflow.providers.openlineage.extractors.base import BaseExtractor, OperatorLineage
+from airflow.providers.openlineage.extractors.extractors import Extractors
+from airflow.providers.openlineage.extractors.manager import ExtractorManager
+
+__all__ = [Extractors, BaseExtractor, OperatorLineage, ExtractorManager]  # type: ignore

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1135406709


##########
airflow/providers/openlineage/utils/__init__.py:
##########
@@ -0,0 +1,499 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def get_connection_uri(conn):
+    """
+    Return the connection URI for the given ID. We first attempt to lookup
+    the connection URI via AIRFLOW_CONN_<conn_id>, else fallback on querying
+    the Airflow's connection table.
+    """
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not _filtered_query_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def _filtered_query_params(k: str):
+    unfiltered_snowflake_keys = [
+        "extra__snowflake__warehouse",
+        "extra__snowflake__account",
+        "extra__snowflake__database",
+    ]
+    filtered_key_substrings = [
+        "aws_access_key_id",
+        "aws_secret_access_key",
+        "extra__snowflake__",
+    ]
+    return k not in unfiltered_snowflake_keys and any(substr in k for substr in filtered_key_substrings)
+
+
+def get_normalized_postgres_connection_uri(conn):
+    """
+    URIs starting with postgresql:// and postgres:// are both valid
+    PostgreSQL connection strings. This function normalizes it to
+    postgres:// as canonical name according to OpenLineage spec.
+    """
+    uri = get_connection_uri(conn)
+    if uri.startswith("postgresql"):
+        uri = uri.replace("postgresql", "postgres", 1)
+    return uri
+
+
+def get_connection(conn_id) -> Connection | None:
+    from airflow.hooks.base import BaseHook
+
+    try:
+        return BaseHook.get_connection(conn_id=conn_id)
+    except Exception:
+        return None
+
+
+def get_job_name(task):
+    return f"{task.dag_id}.{task.task_id}"
+
+
+def get_custom_facets(
+    dagrun, task, is_external_trigger: bool, task_instance: TaskInstance | None = None
+) -> dict[str, Any]:
+    custom_facets = {
+        "airflow_runArgs": AirflowRunArgsRunFacet(is_external_trigger),
+        "airflow_version": AirflowVersionRunFacet.from_dagrun_and_task(dagrun, task),
+    }
+    # check for -1 comes from SmartSensor compatibility with dynamic task mapping
+    # this comes from Airflow code
+    if hasattr(task_instance, "map_index") and getattr(task_instance, "map_index") != -1:
+        custom_facets["airflow_mappedTask"] = AirflowMappedTaskRunFacet.from_task_instance(task_instance)
+    return custom_facets
+
+
+class InfoJsonEncodable(dict):
+    """
+    Airflow objects might not be json-encodable overall.
+
+    The class provides additional attributes to control
+    what and how is encoded:
+    * renames: a dictionary of attribute name changes
+    * casts: a dictionary consisting of attribute names
+             and corresponding methods that should change
+             object value
+    * includes: list of attributes to be included in encoding
+    * excludes: list of attributes to be excluded from encoding
+
+    Don't use both includes and excludes.
+    """
+
+    renames: dict[str, str] = {}
+    casts: dict[str, Any] = {}
+    includes: list[str] = []
+    excludes: list[str] = []
+
+    def __init__(self, obj):
+        self.obj = obj
+        self._fields = []
+
+        self._cast_fields()
+        self._rename_fields()
+        self._include_fields()
+        dict.__init__(
+            self,
+            **{field: InfoJsonEncodable._cast_basic_types(getattr(self, field)) for field in self._fields},
+        )
+
+    @staticmethod
+    def _cast_basic_types(value):
+        if isinstance(value, datetime.datetime):
+            return value.isoformat()
+        if isinstance(value, (set, list, tuple)):
+            return str(list(value))
+        return value
+
+    def _rename_fields(self):
+        for field, renamed in self.renames.items():
+            if hasattr(self.obj, field):
+                setattr(self, renamed, getattr(self.obj, field))
+                self._fields.append(renamed)
+
+    def _cast_fields(self):
+        for field, func in self.casts.items():
+            setattr(self, field, func(self.obj))
+            self._fields.append(field)
+
+    def _include_fields(self):
+        if self.includes and self.excludes:
+            raise Exception("Don't use both includes and excludes.")
+        if self.includes:
+            for field in self.includes:
+                if field in self._fields or not hasattr(self.obj, field):
+                    continue
+                setattr(self, field, getattr(self.obj, field))
+                self._fields.append(field)
+        else:
+            for field, val in self.obj.__dict__.items():
+                if field in self._fields or field in self.excludes or field in self.renames:
+                    continue
+                setattr(self, field, val)
+                self._fields.append(field)
+
+
+class DagInfo(InfoJsonEncodable):
+    """Defines encoding DAG object to JSON."""
+
+    includes = ["dag_id", "schedule_interval", "tags", "start_date"]
+    casts = {"timetable": lambda dag: dag.timetable.serialize() if getattr(dag, "timetable", None) else None}
+    renames = {"_dag_id": "dag_id"}
+
+
+class DagRunInfo(InfoJsonEncodable):
+    """Defines encoding DagRun object to JSON."""
+
+    includes = [
+        "conf",
+        "dag_id",
+        "data_interval_start",
+        "data_interval_end",
+        "external_trigger",
+        "run_id",
+        "run_type",
+        "start_date",
+    ]
+
+
+class TaskInstanceInfo(InfoJsonEncodable):
+    """Defines encoding TaskInstance object to JSON."""
+
+    includes = ["duration", "try_number", "pool"]
+    casts = {
+        "map_index": lambda ti: ti.map_index
+        if hasattr(ti, "map_index") and getattr(ti, "map_index") != -1
+        else None
+    }
+
+
+class TaskInfo(InfoJsonEncodable):
+    """Defines encoding BaseOperator/AbstractOperator object to JSON."""
+
+    renames = {
+        "_BaseOperator__init_kwargs": "args",
+        "_BaseOperator__from_mapped": "mapped",
+        "_downstream_task_ids": "downstream_task_ids",
+        "_upstream_task_ids": "upstream_task_ids",
+    }
+    excludes = [
+        "_BaseOperator__instantiated",
+        "_dag",
+        "_hook",
+        "_log",
+        "_outlets",
+        "_inlets",
+        "_lock_for_execution",
+        "handler",
+        "params",
+        "python_callable",
+        "retry_delay",
+    ]
+    casts = {
+        "operator_class": lambda task: f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",  # noqa
+        "task_group": lambda task: TaskGroupInfo(task.task_group)
+        if hasattr(task, "task_group") and getattr(task.task_group, "_group_id", None)
+        else None,
+    }
+
+
+class TaskGroupInfo(InfoJsonEncodable):
+    """Defines encoding TaskGroup object to JSON."""
+
+    renames = {
+        "_group_id": "group_id",
+    }
+    includes = [
+        "downstream_group_ids",
+        "downstream_task_ids",
+        "prefix_group_id",
+        "tooltip",
+        "upstream_group_ids",
+        "upstream_task_ids",
+    ]
+
+
+def get_airflow_run_facet(
+    dag_run: DagRun,
+    dag: DAG,
+    task_instance: TaskInstance,
+    task: BaseOperator,
+    task_uuid: str,
+):
+    return {
+        "airflow": json.loads(
+            json.dumps(
+                asdict(
+                    AirflowRunFacet(
+                        dag=DagInfo(dag),
+                        dagRun=DagRunInfo(dag_run),
+                        taskInstance=TaskInstanceInfo(task_instance),
+                        task=TaskInfo(task),
+                        taskUuid=task_uuid,
+                    )
+                ),
+                default=str,
+            )
+        )
+    }
+
+
+def get_dagrun_start_end(dagrun: DagRun, dag: DAG):
+    try:
+        return dagrun.data_interval_start, dagrun.data_interval_end
+    except AttributeError:
+        # Airflow < 2.2 before adding data interval
+        pass
+    start = dagrun.execution_date
+    end = dag.following_schedule(start)
+    return start, end or start
+
+
+def import_from_string(path: str):
+    try:
+        module_path, target = path.rsplit(".", 1)
+        module = importlib.import_module(module_path)
+        return getattr(module, target)
+    except Exception as e:
+        log.warning(e)
+        raise ImportError(f"Failed to import {path}") from e
+
+
+def try_import_from_string(path: str):
+    try:
+        return import_from_string(path)
+    except ImportError:
+        return None
+
+
+def redact_with_exclusions(source: Any):
+    try:
+        from airflow.utils.log.secrets_masker import (
+            _secrets_masker,
+            should_hide_value_for_key,
+        )
+    except ImportError:
+        return source
+    import copy
+
+    sm = copy.deepcopy(_secrets_masker())

Review Comment:
   Removed - we use it directly now.



##########
airflow/providers/openlineage/utils/__init__.py:
##########
@@ -0,0 +1,499 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def get_connection_uri(conn):
+    """
+    Return the connection URI for the given ID. We first attempt to lookup
+    the connection URI via AIRFLOW_CONN_<conn_id>, else fallback on querying
+    the Airflow's connection table.
+    """
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not _filtered_query_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def _filtered_query_params(k: str):
+    unfiltered_snowflake_keys = [
+        "extra__snowflake__warehouse",
+        "extra__snowflake__account",
+        "extra__snowflake__database",
+    ]
+    filtered_key_substrings = [
+        "aws_access_key_id",
+        "aws_secret_access_key",
+        "extra__snowflake__",
+    ]
+    return k not in unfiltered_snowflake_keys and any(substr in k for substr in filtered_key_substrings)
+
+
+def get_normalized_postgres_connection_uri(conn):
+    """
+    URIs starting with postgresql:// and postgres:// are both valid
+    PostgreSQL connection strings. This function normalizes it to
+    postgres:// as canonical name according to OpenLineage spec.
+    """
+    uri = get_connection_uri(conn)
+    if uri.startswith("postgresql"):
+        uri = uri.replace("postgresql", "postgres", 1)
+    return uri
+
+
+def get_connection(conn_id) -> Connection | None:
+    from airflow.hooks.base import BaseHook
+
+    try:
+        return BaseHook.get_connection(conn_id=conn_id)
+    except Exception:
+        return None
+
+
+def get_job_name(task):
+    return f"{task.dag_id}.{task.task_id}"
+
+
+def get_custom_facets(
+    dagrun, task, is_external_trigger: bool, task_instance: TaskInstance | None = None
+) -> dict[str, Any]:
+    custom_facets = {
+        "airflow_runArgs": AirflowRunArgsRunFacet(is_external_trigger),
+        "airflow_version": AirflowVersionRunFacet.from_dagrun_and_task(dagrun, task),
+    }
+    # check for -1 comes from SmartSensor compatibility with dynamic task mapping
+    # this comes from Airflow code
+    if hasattr(task_instance, "map_index") and getattr(task_instance, "map_index") != -1:
+        custom_facets["airflow_mappedTask"] = AirflowMappedTaskRunFacet.from_task_instance(task_instance)
+    return custom_facets
+
+
+class InfoJsonEncodable(dict):
+    """
+    Airflow objects might not be json-encodable overall.
+
+    The class provides additional attributes to control
+    what and how is encoded:
+    * renames: a dictionary of attribute name changes
+    * casts: a dictionary consisting of attribute names
+             and corresponding methods that should change
+             object value
+    * includes: list of attributes to be included in encoding
+    * excludes: list of attributes to be excluded from encoding
+
+    Don't use both includes and excludes.
+    """
+
+    renames: dict[str, str] = {}
+    casts: dict[str, Any] = {}
+    includes: list[str] = []
+    excludes: list[str] = []
+
+    def __init__(self, obj):
+        self.obj = obj
+        self._fields = []
+
+        self._cast_fields()
+        self._rename_fields()
+        self._include_fields()
+        dict.__init__(
+            self,
+            **{field: InfoJsonEncodable._cast_basic_types(getattr(self, field)) for field in self._fields},
+        )
+
+    @staticmethod
+    def _cast_basic_types(value):
+        if isinstance(value, datetime.datetime):
+            return value.isoformat()
+        if isinstance(value, (set, list, tuple)):
+            return str(list(value))
+        return value
+
+    def _rename_fields(self):
+        for field, renamed in self.renames.items():
+            if hasattr(self.obj, field):
+                setattr(self, renamed, getattr(self.obj, field))
+                self._fields.append(renamed)
+
+    def _cast_fields(self):
+        for field, func in self.casts.items():
+            setattr(self, field, func(self.obj))
+            self._fields.append(field)
+
+    def _include_fields(self):
+        if self.includes and self.excludes:
+            raise Exception("Don't use both includes and excludes.")
+        if self.includes:
+            for field in self.includes:
+                if field in self._fields or not hasattr(self.obj, field):
+                    continue
+                setattr(self, field, getattr(self.obj, field))
+                self._fields.append(field)
+        else:
+            for field, val in self.obj.__dict__.items():
+                if field in self._fields or field in self.excludes or field in self.renames:
+                    continue
+                setattr(self, field, val)
+                self._fields.append(field)
+
+
+class DagInfo(InfoJsonEncodable):
+    """Defines encoding DAG object to JSON."""
+
+    includes = ["dag_id", "schedule_interval", "tags", "start_date"]
+    casts = {"timetable": lambda dag: dag.timetable.serialize() if getattr(dag, "timetable", None) else None}
+    renames = {"_dag_id": "dag_id"}
+
+
+class DagRunInfo(InfoJsonEncodable):
+    """Defines encoding DagRun object to JSON."""
+
+    includes = [
+        "conf",
+        "dag_id",
+        "data_interval_start",
+        "data_interval_end",
+        "external_trigger",
+        "run_id",
+        "run_type",
+        "start_date",
+    ]
+
+
+class TaskInstanceInfo(InfoJsonEncodable):
+    """Defines encoding TaskInstance object to JSON."""
+
+    includes = ["duration", "try_number", "pool"]
+    casts = {
+        "map_index": lambda ti: ti.map_index
+        if hasattr(ti, "map_index") and getattr(ti, "map_index") != -1
+        else None
+    }
+
+
+class TaskInfo(InfoJsonEncodable):
+    """Defines encoding BaseOperator/AbstractOperator object to JSON."""
+
+    renames = {
+        "_BaseOperator__init_kwargs": "args",
+        "_BaseOperator__from_mapped": "mapped",
+        "_downstream_task_ids": "downstream_task_ids",
+        "_upstream_task_ids": "upstream_task_ids",
+    }
+    excludes = [
+        "_BaseOperator__instantiated",
+        "_dag",
+        "_hook",
+        "_log",
+        "_outlets",
+        "_inlets",
+        "_lock_for_execution",
+        "handler",
+        "params",
+        "python_callable",
+        "retry_delay",
+    ]
+    casts = {
+        "operator_class": lambda task: f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",  # noqa
+        "task_group": lambda task: TaskGroupInfo(task.task_group)
+        if hasattr(task, "task_group") and getattr(task.task_group, "_group_id", None)
+        else None,
+    }
+
+
+class TaskGroupInfo(InfoJsonEncodable):
+    """Defines encoding TaskGroup object to JSON."""
+
+    renames = {
+        "_group_id": "group_id",
+    }
+    includes = [
+        "downstream_group_ids",
+        "downstream_task_ids",
+        "prefix_group_id",
+        "tooltip",
+        "upstream_group_ids",
+        "upstream_task_ids",
+    ]
+
+
+def get_airflow_run_facet(
+    dag_run: DagRun,
+    dag: DAG,
+    task_instance: TaskInstance,
+    task: BaseOperator,
+    task_uuid: str,
+):
+    return {
+        "airflow": json.loads(
+            json.dumps(
+                asdict(
+                    AirflowRunFacet(
+                        dag=DagInfo(dag),
+                        dagRun=DagRunInfo(dag_run),
+                        taskInstance=TaskInstanceInfo(task_instance),
+                        task=TaskInfo(task),
+                        taskUuid=task_uuid,
+                    )
+                ),
+                default=str,
+            )
+        )
+    }
+
+
+def get_dagrun_start_end(dagrun: DagRun, dag: DAG):
+    try:
+        return dagrun.data_interval_start, dagrun.data_interval_end
+    except AttributeError:
+        # Airflow < 2.2 before adding data interval
+        pass
+    start = dagrun.execution_date
+    end = dag.following_schedule(start)
+    return start, end or start
+
+
+def import_from_string(path: str):
+    try:
+        module_path, target = path.rsplit(".", 1)
+        module = importlib.import_module(module_path)
+        return getattr(module, target)
+    except Exception as e:
+        log.warning(e)
+        raise ImportError(f"Failed to import {path}") from e
+
+
+def try_import_from_string(path: str):
+    try:
+        return import_from_string(path)
+    except ImportError:
+        return None

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1156173832


##########
airflow/providers/openlineage/utils/utils.py:
##########
@@ -0,0 +1,482 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from contextlib import suppress
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attrs
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def redacted_connection_uri(conn: Connection, filtered_params=None, filtered_prefixes=None):
+    """
+    Return the connection URI for the given Connection.
+    This method additionally filters URI by removing query parameters that are known to carry sensitive data
+    like username, password, access key.
+    """
+    if filtered_prefixes is None:
+        filtered_prefixes = []
+    if filtered_params is None:
+        filtered_params = []
+
+    def filter_key_params(k: str):
+        return k not in filtered_params and any(substr in k for substr in filtered_prefixes)
+
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not filter_key_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def get_normalized_postgres_connection_uri(conn):
+    """
+    URIs starting with postgresql:// and postgres:// are both valid
+    PostgreSQL connection strings. This function normalizes it to
+    postgres:// as canonical name according to OpenLineage spec.
+    """
+    uri = redacted_connection_uri(conn)
+    if uri.startswith("postgresql"):
+        uri = uri.replace("postgresql", "postgres", 1)
+    return uri
+
+
+def get_connection(conn_id) -> Connection | None:
+    from airflow.hooks.base import BaseHook
+
+    with suppress(Exception):
+        return BaseHook.get_connection(conn_id=conn_id)
+    return None
+
+
+def get_job_name(task):
+    return f"{task.dag_id}.{task.task_id}"
+
+
+def get_custom_facets(
+    dagrun, task, is_external_trigger: bool, task_instance: TaskInstance | None = None
+) -> dict[str, Any]:
+    custom_facets = {
+        "airflow_runArgs": AirflowRunArgsRunFacet(is_external_trigger),
+        "airflow_version": AirflowVersionRunFacet.from_dagrun_and_task(dagrun, task),

Review Comment:
   I actually think the best point to remove those is now... I'll check if that's a good idea. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1156114615


##########
airflow/providers/openlineage/plugins/listener.py:
##########
@@ -0,0 +1,189 @@
+# 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 logging
+from concurrent.futures import Executor, ThreadPoolExecutor
+from typing import TYPE_CHECKING
+
+from airflow.listeners import hookimpl
+from airflow.providers.openlineage.extractors import ExtractorManager
+from airflow.providers.openlineage.plugins.adapter import OpenLineageAdapter
+from airflow.providers.openlineage.utils import (
+    get_airflow_run_facet,
+    get_custom_facets,
+    get_job_name,
+    get_task_location,
+    print_exception,
+)
+
+if TYPE_CHECKING:
+    from sqlalchemy.orm import Session
+
+    from airflow.models import DagRun, TaskInstance
+
+
+class OpenLineageListener:
+    """
+    OpenLineage listener
+    Sends events on task instance and dag run starts, completes and failures.
+    """
+
+    def __init__(self):
+        self.log = logging.getLogger(__name__)
+        self.executor: Executor = None  # type: ignore
+        self.extractor_manager = ExtractorManager()
+        self.adapter = OpenLineageAdapter()
+
+    @hookimpl
+    def on_task_instance_running(
+        self, previous_state, task_instance: TaskInstance, session: Session  # This will always be QUEUED
+    ):
+        if not hasattr(task_instance, "task"):
+            self.log.warning(
+                f"No task set for TI object task_id: {task_instance.task_id} - "
+                f"dag_id: {task_instance.dag_id} - run_id {task_instance.run_id}"
+            )
+            return
+
+        self.log.debug("OpenLineage listener got notification about task instance start")
+        dagrun = task_instance.dag_run
+        task = task_instance.task
+        dag = task.dag
+
+        @print_exception
+        def on_running():
+            # that's a workaround to detect task running from deferred state
+            # we return here because Airflow 2.3 needs task from deferred state
+            if task_instance.next_method is not None:
+                return
+            parent_run_id = self.adapter.build_dag_run_id(dag.dag_id, dagrun.run_id)
+
+            task_uuid = self.adapter.build_task_instance_run_id(
+                task.task_id, task_instance.execution_date, task_instance.try_number
+            )
+
+            task_metadata = self.extractor_manager.extract_metadata(dagrun, task)
+
+            self.adapter.start_task(
+                run_id=task_uuid,
+                job_name=get_job_name(task),
+                job_description=dag.description,
+                event_time=task_instance.start_date.isoformat(),
+                parent_job_name=dag.dag_id,
+                parent_run_id=parent_run_id,
+                code_location=get_task_location(task),
+                nominal_start_time=dagrun.data_interval_start.isoformat(),
+                nominal_end_time=dagrun.data_interval_end.isoformat(),
+                owners=dag.owner.split(", "),
+                task=task_metadata,
+                run_facets={
+                    **task_metadata.run_facets,
+                    **get_custom_facets(dagrun, task, dagrun.external_trigger, task_instance),
+                    **get_airflow_run_facet(dagrun, dag, task_instance, task, task_uuid),
+                },
+            )
+
+        self.executor.submit(on_running)
+
+    @hookimpl
+    def on_task_instance_success(self, previous_state, task_instance: TaskInstance, session):
+        self.log.debug("OpenLineage listener got notification about task instance success")
+
+        dagrun = task_instance.dag_run
+        task = task_instance.task
+
+        task_uuid = OpenLineageAdapter.build_task_instance_run_id(
+            task.task_id, task_instance.execution_date, task_instance.try_number - 1
+        )
+
+        @print_exception
+        def on_success():
+            task_metadata = self.extractor_manager.extract_metadata(
+                dagrun, task, complete=True, task_instance=task_instance
+            )
+            self.adapter.complete_task(
+                run_id=task_uuid,
+                job_name=get_job_name(task),
+                end_time=task_instance.end_date.isoformat(),
+                task=task_metadata,
+            )
+
+        self.executor.submit(on_success)
+
+    @hookimpl
+    def on_task_instance_failed(self, previous_state, task_instance: TaskInstance, session):
+        self.log.debug("OpenLineage listener got notification about task instance failure")
+
+        dagrun = task_instance.dag_run
+        task = task_instance.task
+
+        task_uuid = OpenLineageAdapter.build_task_instance_run_id(
+            task.task_id, task_instance.execution_date, task_instance.try_number - 1
+        )
+
+        @print_exception
+        def on_failure():
+            task_metadata = self.extractor_manager.extract_metadata(
+                dagrun, task, complete=True, task_instance=task_instance
+            )
+
+            self.adapter.fail_task(
+                run_id=task_uuid,
+                job_name=get_job_name(task),
+                end_time=task_instance.end_date.isoformat(),
+                task=task_metadata,
+            )
+
+        self.executor.submit(on_failure)
+
+    @hookimpl
+    def on_starting(self, component):
+        self.log.debug("on_starting: %s", component.__class__.__name__)
+        self.executor = ThreadPoolExecutor(max_workers=8, thread_name_prefix="openlineage_")
+
+    @hookimpl
+    def before_stopping(self, component):
+        self.log.debug("before_stopping: %s", component.__class__.__name__)
+        self.executor.shutdown(wait=True)

Review Comment:
   Looks like it does not pass any timeout value for join: https://github.com/python/cpython/blob/main/Lib/concurrent/futures/thread.py#L235
   
   However, it looks like `airflow.utils.timeout` would be exactly what we need 🙂 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1155944374


##########
airflow/providers/openlineage/utils/utils.py:
##########
@@ -0,0 +1,482 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from contextlib import suppress
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attrs
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,

Review Comment:
   For timetables, `dag.timetable.serialize()` should work, right? 
   
   However, for dataset-triggered dags, should we just serialize `dag.dataset_triggers`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] ashb commented on pull request #29940: Add base OpenLineage provider implementation

Posted by "ashb (via GitHub)" <gi...@apache.org>.
ashb commented on PR #29940:
URL: https://github.com/apache/airflow/pull/29940#issuecomment-1499063018

   Lets see if we can sort out the config masking/redacting - possibly by subclassing in someway, something like this:
   
   ```python
   class OLReadactor(SecretsMasker):
       @classmethod
       def from_masker(cls, other: SecretsMasker):
           instance = cls()
           instance.patterns = other.patterns
           instance.replacer = other.replacer
   
       def _redact(self, item: Redactable, name: str | None, depth: int, max_depth: int) -> Redacted:
           # custom type checks here
           return super()._redact(item, name, depth, max_depth)
   
       def _redact_all(self, item: Redactable, depth: int, max_depth: int) -> Redacted:
           # custom type checks here
           return super()._redact_all(item, depth, max_depth)
   ```
   
   (Not sure if you'd need both redact  and redact_all or if just redact would be enough)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1157612149


##########
airflow/providers/openlineage/extractors/__init__.py:
##########
@@ -0,0 +1,23 @@
+# 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
+
+from airflow.providers.openlineage.extractors.base import BaseExtractor, OperatorLineage
+from airflow.providers.openlineage.extractors.manager import ExtractorManager
+
+__all__ = ["BaseExtractor", "OperatorLineage", "ExtractorManager"]  # type: ignore

Review Comment:
   Fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1162995819


##########
airflow/providers/openlineage/extractors/base.py:
##########
@@ -0,0 +1,106 @@
+# 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
+
+from abc import ABC, abstractmethod
+
+from attrs import Factory, define
+
+from airflow.utils.log.logging_mixin import LoggingMixin
+from openlineage.client.facet import BaseFacet
+from openlineage.client.run import Dataset
+
+
+@define
+class OperatorLineage:
+    """Structure returned from lineage extraction."""
+
+    inputs: list[Dataset] = Factory(list)
+    outputs: list[Dataset] = Factory(list)
+    run_facets: dict[str, BaseFacet] = Factory(dict)
+    job_facets: dict[str, BaseFacet] = Factory(dict)
+
+
+class BaseExtractor(ABC, LoggingMixin):
+    """
+    Abstract base extractor class.
+
+    This is used mostly to maintain support for custom extractors.
+    """
+
+    _allowed_query_params: list[str] = []
+
+    def __init__(self, operator):  # type: ignore
+        super().__init__()
+        self.operator = operator
+
+    @classmethod
+    @abstractmethod
+    def get_operator_classnames(cls) -> list[str]:
+        """
+        Implement this method returning list of operators that extractor works for.
+
+        Particularly, in Airflow 2 some operators are deprecated and simply subclass the new
+        implementation, for example airflow.contrib.operators.bigquery_operator.BigQueryOperator.
+        The BigQueryExtractor needs to work with both of them.

Review Comment:
   > There is no BigQueryOperator in the code base.
   
   I see, it was removed in September. Removed reference to it, just explaining that single extractor can work for multiple operators.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1134444484


##########
airflow/providers/openlineage/extractors/base.py:
##########
@@ -0,0 +1,133 @@
+# 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 json
+from abc import ABC, abstractmethod
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+from attrs import Factory, define
+
+from airflow.utils.log.logging_mixin import LoggingMixin
+from openlineage.client.facet import BaseFacet
+from openlineage.client.run import Dataset
+
+
+@define
+class OperatorLineage:
+    """Structure returned from lineage extraction."""
+
+    inputs: list[Dataset] = Factory(list)
+    outputs: list[Dataset] = Factory(list)
+    run_facets: dict[str, BaseFacet] = Factory(dict)
+    job_facets: dict[str, BaseFacet] = Factory(dict)
+
+
+class BaseExtractor(ABC, LoggingMixin):
+    """
+    Abstract base extractor class.
+
+    This is used mostly to maintain support for custom extractors.
+    """
+
+    _allowed_query_params: list[str] = []
+
+    def __init__(self, operator):
+        super().__init__()
+        self.operator = operator
+        self.patch()
+
+    def patch(self):
+        # Extractor should register extension methods or patches to operator here
+        pass
+
+    @classmethod

Review Comment:
   Python recommends
   
   ```
   @classmethod
   @abstractmethod
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1135419055


##########
airflow/providers/openlineage/plugins/listener.py:
##########
@@ -0,0 +1,189 @@
+# 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 logging
+from concurrent.futures import Executor, ThreadPoolExecutor
+from typing import TYPE_CHECKING
+
+from airflow.listeners import hookimpl
+from airflow.providers.openlineage.extractors import ExtractorManager
+from airflow.providers.openlineage.plugins.adapter import OpenLineageAdapter
+from airflow.providers.openlineage.utils import (
+    get_airflow_run_facet,
+    get_custom_facets,
+    get_job_name,
+    get_task_location,
+    print_exception,
+)
+
+if TYPE_CHECKING:
+    from sqlalchemy.orm import Session
+
+    from airflow.models import DagRun, TaskInstance
+
+
+class OpenLineageListener:
+    """
+    OpenLineage listener
+    Sends events on task instance and dag run starts, completes and failures.
+    """
+
+    def __init__(self):
+        self.log = logging.getLogger(__name__)
+        self.executor: Executor = None  # type: ignore
+        self.extractor_manager = ExtractorManager()
+        self.adapter = OpenLineageAdapter()
+
+    @hookimpl
+    def on_task_instance_running(
+        self, previous_state, task_instance: TaskInstance, session: Session  # This will always be QUEUED
+    ):
+        if not hasattr(task_instance, "task"):
+            self.log.warning(
+                f"No task set for TI object task_id: {task_instance.task_id} - "
+                f"dag_id: {task_instance.dag_id} - run_id {task_instance.run_id}"
+            )
+            return
+
+        self.log.debug("OpenLineage listener got notification about task instance start")
+        dagrun = task_instance.dag_run
+        task = task_instance.task
+        dag = task.dag
+
+        @print_exception
+        def on_running():
+            # that's a workaround to detect task running from deferred state
+            # we return here because Airflow 2.3 needs task from deferred state
+            if task_instance.next_method is not None:
+                return
+            parent_run_id = self.adapter.build_dag_run_id(dag.dag_id, dagrun.run_id)
+
+            task_uuid = self.adapter.build_task_instance_run_id(
+                task.task_id, task_instance.execution_date, task_instance.try_number
+            )
+
+            task_metadata = self.extractor_manager.extract_metadata(dagrun, task)
+
+            self.adapter.start_task(
+                run_id=task_uuid,
+                job_name=get_job_name(task),
+                job_description=dag.description,
+                event_time=task_instance.start_date.isoformat(),
+                parent_job_name=dag.dag_id,
+                parent_run_id=parent_run_id,
+                code_location=get_task_location(task),
+                nominal_start_time=dagrun.data_interval_start.isoformat(),
+                nominal_end_time=dagrun.data_interval_end.isoformat(),
+                owners=dag.owner.split(", "),
+                task=task_metadata,
+                run_facets={
+                    **task_metadata.run_facets,
+                    **get_custom_facets(dagrun, task, dagrun.external_trigger, task_instance),
+                    **get_airflow_run_facet(dagrun, dag, task_instance, task, task_uuid),
+                },
+            )
+
+        self.executor.submit(on_running)
+
+    @hookimpl
+    def on_task_instance_success(self, previous_state, task_instance: TaskInstance, session):
+        self.log.debug("OpenLineage listener got notification about task instance success")
+
+        dagrun = task_instance.dag_run
+        task = task_instance.task
+
+        task_uuid = OpenLineageAdapter.build_task_instance_run_id(
+            task.task_id, task_instance.execution_date, task_instance.try_number - 1
+        )
+
+        @print_exception
+        def on_success():
+            task_metadata = self.extractor_manager.extract_metadata(
+                dagrun, task, complete=True, task_instance=task_instance
+            )
+            self.adapter.complete_task(
+                run_id=task_uuid,
+                job_name=get_job_name(task),
+                end_time=task_instance.end_date.isoformat(),
+                task=task_metadata,
+            )
+
+        self.executor.submit(on_success)
+
+    @hookimpl
+    def on_task_instance_failed(self, previous_state, task_instance: TaskInstance, session):
+        self.log.debug("OpenLineage listener got notification about task instance failure")
+
+        dagrun = task_instance.dag_run
+        task = task_instance.task
+
+        task_uuid = OpenLineageAdapter.build_task_instance_run_id(
+            task.task_id, task_instance.execution_date, task_instance.try_number - 1
+        )
+
+        @print_exception
+        def on_failure():
+            task_metadata = self.extractor_manager.extract_metadata(
+                dagrun, task, complete=True, task_instance=task_instance
+            )
+
+            self.adapter.fail_task(
+                run_id=task_uuid,
+                job_name=get_job_name(task),
+                end_time=task_instance.end_date.isoformat(),
+                task=task_metadata,
+            )
+
+        self.executor.submit(on_failure)
+
+    @hookimpl
+    def on_starting(self, component):
+        self.log.debug("on_starting: %s", component.__class__.__name__)
+        self.executor = ThreadPoolExecutor(max_workers=8, thread_name_prefix="openlineage_")

Review Comment:
   I'm pretty sure we can need more than one to send events from scheduler. At least if as long if we use synchronous requests.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] JDarDagran commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "JDarDagran (via GitHub)" <gi...@apache.org>.
JDarDagran commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1134150448


##########
airflow/providers/openlineage/plugins/openlineage.py:
##########
@@ -0,0 +1,46 @@
+# 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 airflow.plugins_manager import AirflowPlugin
+from airflow.providers.openlineage.plugins.macros import lineage_parent_id, lineage_run_id
+
+
+def _is_disabled():
+    return os.getenv("OPENLINEAGE_DISABLED", None) in [True, "true", "True"]

Review Comment:
   It's the same as for `OPENLINEAGE_EXTRACTORS`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1134399108


##########
airflow/providers/openlineage/utils/__init__.py:
##########
@@ -0,0 +1,499 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def get_connection_uri(conn):
+    """
+    Return the connection URI for the given ID. We first attempt to lookup
+    the connection URI via AIRFLOW_CONN_<conn_id>, else fallback on querying
+    the Airflow's connection table.
+    """
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not _filtered_query_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def _filtered_query_params(k: str):
+    unfiltered_snowflake_keys = [
+        "extra__snowflake__warehouse",
+        "extra__snowflake__account",
+        "extra__snowflake__database",
+    ]
+    filtered_key_substrings = [
+        "aws_access_key_id",
+        "aws_secret_access_key",
+        "extra__snowflake__",
+    ]
+    return k not in unfiltered_snowflake_keys and any(substr in k for substr in filtered_key_substrings)
+
+
+def get_normalized_postgres_connection_uri(conn):
+    """
+    URIs starting with postgresql:// and postgres:// are both valid
+    PostgreSQL connection strings. This function normalizes it to
+    postgres:// as canonical name according to OpenLineage spec.
+    """
+    uri = get_connection_uri(conn)
+    if uri.startswith("postgresql"):
+        uri = uri.replace("postgresql", "postgres", 1)
+    return uri
+
+
+def get_connection(conn_id) -> Connection | None:
+    from airflow.hooks.base import BaseHook
+
+    try:
+        return BaseHook.get_connection(conn_id=conn_id)
+    except Exception:
+        return None
+
+
+def get_job_name(task):
+    return f"{task.dag_id}.{task.task_id}"
+
+
+def get_custom_facets(
+    dagrun, task, is_external_trigger: bool, task_instance: TaskInstance | None = None
+) -> dict[str, Any]:
+    custom_facets = {
+        "airflow_runArgs": AirflowRunArgsRunFacet(is_external_trigger),
+        "airflow_version": AirflowVersionRunFacet.from_dagrun_and_task(dagrun, task),
+    }
+    # check for -1 comes from SmartSensor compatibility with dynamic task mapping
+    # this comes from Airflow code
+    if hasattr(task_instance, "map_index") and getattr(task_instance, "map_index") != -1:
+        custom_facets["airflow_mappedTask"] = AirflowMappedTaskRunFacet.from_task_instance(task_instance)
+    return custom_facets
+
+
+class InfoJsonEncodable(dict):
+    """
+    Airflow objects might not be json-encodable overall.
+
+    The class provides additional attributes to control
+    what and how is encoded:
+    * renames: a dictionary of attribute name changes
+    * casts: a dictionary consisting of attribute names
+             and corresponding methods that should change
+             object value
+    * includes: list of attributes to be included in encoding
+    * excludes: list of attributes to be excluded from encoding
+
+    Don't use both includes and excludes.
+    """
+
+    renames: dict[str, str] = {}
+    casts: dict[str, Any] = {}
+    includes: list[str] = []
+    excludes: list[str] = []
+
+    def __init__(self, obj):
+        self.obj = obj
+        self._fields = []
+
+        self._cast_fields()
+        self._rename_fields()
+        self._include_fields()
+        dict.__init__(
+            self,
+            **{field: InfoJsonEncodable._cast_basic_types(getattr(self, field)) for field in self._fields},
+        )
+
+    @staticmethod
+    def _cast_basic_types(value):
+        if isinstance(value, datetime.datetime):
+            return value.isoformat()
+        if isinstance(value, (set, list, tuple)):
+            return str(list(value))
+        return value
+
+    def _rename_fields(self):
+        for field, renamed in self.renames.items():
+            if hasattr(self.obj, field):
+                setattr(self, renamed, getattr(self.obj, field))
+                self._fields.append(renamed)
+
+    def _cast_fields(self):
+        for field, func in self.casts.items():
+            setattr(self, field, func(self.obj))
+            self._fields.append(field)
+
+    def _include_fields(self):
+        if self.includes and self.excludes:
+            raise Exception("Don't use both includes and excludes.")
+        if self.includes:
+            for field in self.includes:
+                if field in self._fields or not hasattr(self.obj, field):
+                    continue
+                setattr(self, field, getattr(self.obj, field))
+                self._fields.append(field)
+        else:
+            for field, val in self.obj.__dict__.items():
+                if field in self._fields or field in self.excludes or field in self.renames:
+                    continue
+                setattr(self, field, val)
+                self._fields.append(field)
+
+
+class DagInfo(InfoJsonEncodable):
+    """Defines encoding DAG object to JSON."""
+
+    includes = ["dag_id", "schedule_interval", "tags", "start_date"]
+    casts = {"timetable": lambda dag: dag.timetable.serialize() if getattr(dag, "timetable", None) else None}
+    renames = {"_dag_id": "dag_id"}
+
+
+class DagRunInfo(InfoJsonEncodable):
+    """Defines encoding DagRun object to JSON."""
+
+    includes = [
+        "conf",
+        "dag_id",
+        "data_interval_start",
+        "data_interval_end",
+        "external_trigger",
+        "run_id",
+        "run_type",
+        "start_date",
+    ]
+
+
+class TaskInstanceInfo(InfoJsonEncodable):
+    """Defines encoding TaskInstance object to JSON."""
+
+    includes = ["duration", "try_number", "pool"]
+    casts = {
+        "map_index": lambda ti: ti.map_index
+        if hasattr(ti, "map_index") and getattr(ti, "map_index") != -1
+        else None
+    }
+
+
+class TaskInfo(InfoJsonEncodable):
+    """Defines encoding BaseOperator/AbstractOperator object to JSON."""
+
+    renames = {
+        "_BaseOperator__init_kwargs": "args",
+        "_BaseOperator__from_mapped": "mapped",
+        "_downstream_task_ids": "downstream_task_ids",
+        "_upstream_task_ids": "upstream_task_ids",
+    }
+    excludes = [
+        "_BaseOperator__instantiated",
+        "_dag",
+        "_hook",
+        "_log",
+        "_outlets",
+        "_inlets",
+        "_lock_for_execution",
+        "handler",
+        "params",
+        "python_callable",
+        "retry_delay",
+    ]
+    casts = {
+        "operator_class": lambda task: f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",  # noqa
+        "task_group": lambda task: TaskGroupInfo(task.task_group)
+        if hasattr(task, "task_group") and getattr(task.task_group, "_group_id", None)
+        else None,
+    }
+
+
+class TaskGroupInfo(InfoJsonEncodable):
+    """Defines encoding TaskGroup object to JSON."""
+
+    renames = {
+        "_group_id": "group_id",
+    }
+    includes = [
+        "downstream_group_ids",
+        "downstream_task_ids",
+        "prefix_group_id",
+        "tooltip",
+        "upstream_group_ids",
+        "upstream_task_ids",
+    ]
+
+
+def get_airflow_run_facet(
+    dag_run: DagRun,
+    dag: DAG,
+    task_instance: TaskInstance,
+    task: BaseOperator,
+    task_uuid: str,
+):
+    return {
+        "airflow": json.loads(
+            json.dumps(
+                asdict(
+                    AirflowRunFacet(
+                        dag=DagInfo(dag),
+                        dagRun=DagRunInfo(dag_run),
+                        taskInstance=TaskInstanceInfo(task_instance),
+                        task=TaskInfo(task),
+                        taskUuid=task_uuid,
+                    )
+                ),
+                default=str,
+            )
+        )
+    }
+
+
+def get_dagrun_start_end(dagrun: DagRun, dag: DAG):
+    try:
+        return dagrun.data_interval_start, dagrun.data_interval_end
+    except AttributeError:
+        # Airflow < 2.2 before adding data interval
+        pass
+    start = dagrun.execution_date
+    end = dag.following_schedule(start)
+    return start, end or start
+
+
+def import_from_string(path: str):
+    try:
+        module_path, target = path.rsplit(".", 1)
+        module = importlib.import_module(module_path)
+        return getattr(module, target)
+    except Exception as e:
+        log.warning(e)
+        raise ImportError(f"Failed to import {path}") from e
+
+
+def try_import_from_string(path: str):
+    try:
+        return import_from_string(path)
+    except ImportError:
+        return None
+
+
+def redact_with_exclusions(source: Any):

Review Comment:
   1. We increased recursion depth - it was not sufficient for us.
   2. Added possibility to skip redacting predefined attributes as in `_is_name_redactable`.



##########
airflow/providers/openlineage/utils/__init__.py:
##########
@@ -0,0 +1,499 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def get_connection_uri(conn):
+    """
+    Return the connection URI for the given ID. We first attempt to lookup
+    the connection URI via AIRFLOW_CONN_<conn_id>, else fallback on querying
+    the Airflow's connection table.
+    """
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not _filtered_query_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def _filtered_query_params(k: str):
+    unfiltered_snowflake_keys = [
+        "extra__snowflake__warehouse",
+        "extra__snowflake__account",
+        "extra__snowflake__database",
+    ]
+    filtered_key_substrings = [
+        "aws_access_key_id",
+        "aws_secret_access_key",
+        "extra__snowflake__",
+    ]
+    return k not in unfiltered_snowflake_keys and any(substr in k for substr in filtered_key_substrings)
+
+
+def get_normalized_postgres_connection_uri(conn):
+    """
+    URIs starting with postgresql:// and postgres:// are both valid
+    PostgreSQL connection strings. This function normalizes it to
+    postgres:// as canonical name according to OpenLineage spec.
+    """
+    uri = get_connection_uri(conn)
+    if uri.startswith("postgresql"):
+        uri = uri.replace("postgresql", "postgres", 1)
+    return uri
+
+
+def get_connection(conn_id) -> Connection | None:
+    from airflow.hooks.base import BaseHook
+
+    try:
+        return BaseHook.get_connection(conn_id=conn_id)
+    except Exception:
+        return None
+
+
+def get_job_name(task):
+    return f"{task.dag_id}.{task.task_id}"
+
+
+def get_custom_facets(
+    dagrun, task, is_external_trigger: bool, task_instance: TaskInstance | None = None
+) -> dict[str, Any]:
+    custom_facets = {
+        "airflow_runArgs": AirflowRunArgsRunFacet(is_external_trigger),
+        "airflow_version": AirflowVersionRunFacet.from_dagrun_and_task(dagrun, task),
+    }
+    # check for -1 comes from SmartSensor compatibility with dynamic task mapping
+    # this comes from Airflow code
+    if hasattr(task_instance, "map_index") and getattr(task_instance, "map_index") != -1:
+        custom_facets["airflow_mappedTask"] = AirflowMappedTaskRunFacet.from_task_instance(task_instance)
+    return custom_facets
+
+
+class InfoJsonEncodable(dict):
+    """
+    Airflow objects might not be json-encodable overall.
+
+    The class provides additional attributes to control
+    what and how is encoded:
+    * renames: a dictionary of attribute name changes
+    * casts: a dictionary consisting of attribute names
+             and corresponding methods that should change
+             object value
+    * includes: list of attributes to be included in encoding
+    * excludes: list of attributes to be excluded from encoding
+
+    Don't use both includes and excludes.
+    """
+
+    renames: dict[str, str] = {}
+    casts: dict[str, Any] = {}
+    includes: list[str] = []
+    excludes: list[str] = []
+
+    def __init__(self, obj):
+        self.obj = obj
+        self._fields = []
+
+        self._cast_fields()
+        self._rename_fields()
+        self._include_fields()
+        dict.__init__(
+            self,
+            **{field: InfoJsonEncodable._cast_basic_types(getattr(self, field)) for field in self._fields},
+        )
+
+    @staticmethod
+    def _cast_basic_types(value):
+        if isinstance(value, datetime.datetime):
+            return value.isoformat()
+        if isinstance(value, (set, list, tuple)):
+            return str(list(value))
+        return value
+
+    def _rename_fields(self):
+        for field, renamed in self.renames.items():
+            if hasattr(self.obj, field):
+                setattr(self, renamed, getattr(self.obj, field))
+                self._fields.append(renamed)
+
+    def _cast_fields(self):
+        for field, func in self.casts.items():
+            setattr(self, field, func(self.obj))
+            self._fields.append(field)
+
+    def _include_fields(self):
+        if self.includes and self.excludes:
+            raise Exception("Don't use both includes and excludes.")
+        if self.includes:
+            for field in self.includes:
+                if field in self._fields or not hasattr(self.obj, field):
+                    continue
+                setattr(self, field, getattr(self.obj, field))
+                self._fields.append(field)
+        else:
+            for field, val in self.obj.__dict__.items():
+                if field in self._fields or field in self.excludes or field in self.renames:
+                    continue
+                setattr(self, field, val)
+                self._fields.append(field)
+
+
+class DagInfo(InfoJsonEncodable):
+    """Defines encoding DAG object to JSON."""
+
+    includes = ["dag_id", "schedule_interval", "tags", "start_date"]
+    casts = {"timetable": lambda dag: dag.timetable.serialize() if getattr(dag, "timetable", None) else None}
+    renames = {"_dag_id": "dag_id"}
+
+
+class DagRunInfo(InfoJsonEncodable):
+    """Defines encoding DagRun object to JSON."""
+
+    includes = [
+        "conf",
+        "dag_id",
+        "data_interval_start",
+        "data_interval_end",
+        "external_trigger",
+        "run_id",
+        "run_type",
+        "start_date",
+    ]
+
+
+class TaskInstanceInfo(InfoJsonEncodable):
+    """Defines encoding TaskInstance object to JSON."""
+
+    includes = ["duration", "try_number", "pool"]
+    casts = {
+        "map_index": lambda ti: ti.map_index
+        if hasattr(ti, "map_index") and getattr(ti, "map_index") != -1
+        else None
+    }
+
+
+class TaskInfo(InfoJsonEncodable):
+    """Defines encoding BaseOperator/AbstractOperator object to JSON."""
+
+    renames = {
+        "_BaseOperator__init_kwargs": "args",
+        "_BaseOperator__from_mapped": "mapped",
+        "_downstream_task_ids": "downstream_task_ids",
+        "_upstream_task_ids": "upstream_task_ids",
+    }
+    excludes = [
+        "_BaseOperator__instantiated",
+        "_dag",
+        "_hook",
+        "_log",
+        "_outlets",
+        "_inlets",
+        "_lock_for_execution",
+        "handler",
+        "params",
+        "python_callable",
+        "retry_delay",
+    ]
+    casts = {
+        "operator_class": lambda task: f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",  # noqa
+        "task_group": lambda task: TaskGroupInfo(task.task_group)
+        if hasattr(task, "task_group") and getattr(task.task_group, "_group_id", None)
+        else None,
+    }
+
+
+class TaskGroupInfo(InfoJsonEncodable):
+    """Defines encoding TaskGroup object to JSON."""
+
+    renames = {
+        "_group_id": "group_id",
+    }
+    includes = [
+        "downstream_group_ids",
+        "downstream_task_ids",
+        "prefix_group_id",
+        "tooltip",
+        "upstream_group_ids",
+        "upstream_task_ids",
+    ]
+
+
+def get_airflow_run_facet(
+    dag_run: DagRun,
+    dag: DAG,
+    task_instance: TaskInstance,
+    task: BaseOperator,
+    task_uuid: str,
+):
+    return {
+        "airflow": json.loads(
+            json.dumps(
+                asdict(
+                    AirflowRunFacet(
+                        dag=DagInfo(dag),
+                        dagRun=DagRunInfo(dag_run),
+                        taskInstance=TaskInstanceInfo(task_instance),
+                        task=TaskInfo(task),
+                        taskUuid=task_uuid,
+                    )
+                ),
+                default=str,
+            )
+        )
+    }
+
+
+def get_dagrun_start_end(dagrun: DagRun, dag: DAG):
+    try:
+        return dagrun.data_interval_start, dagrun.data_interval_end
+    except AttributeError:
+        # Airflow < 2.2 before adding data interval
+        pass
+    start = dagrun.execution_date
+    end = dag.following_schedule(start)
+    return start, end or start
+
+
+def import_from_string(path: str):
+    try:
+        module_path, target = path.rsplit(".", 1)
+        module = importlib.import_module(module_path)
+        return getattr(module, target)
+    except Exception as e:
+        log.warning(e)
+        raise ImportError(f"Failed to import {path}") from e
+
+
+def try_import_from_string(path: str):
+    try:
+        return import_from_string(path)
+    except ImportError:
+        return None
+
+
+def redact_with_exclusions(source: Any):

Review Comment:
   1. We increased recursion depth - it was not sufficient for us.
   2. Added possibility to skip redacting predefined attributes as in `_is_name_redactable` function.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] JDarDagran commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "JDarDagran (via GitHub)" <gi...@apache.org>.
JDarDagran commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1134615778


##########
airflow/providers/openlineage/plugins/listener.py:
##########
@@ -0,0 +1,189 @@
+# 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 logging
+from concurrent.futures import Executor, ThreadPoolExecutor
+from typing import TYPE_CHECKING
+
+from airflow.listeners import hookimpl
+from airflow.providers.openlineage.extractors import ExtractorManager
+from airflow.providers.openlineage.plugins.adapter import OpenLineageAdapter
+from airflow.providers.openlineage.utils import (
+    get_airflow_run_facet,
+    get_custom_facets,
+    get_job_name,
+    get_task_location,
+    print_exception,
+)
+
+if TYPE_CHECKING:
+    from sqlalchemy.orm import Session
+
+    from airflow.models import DagRun, TaskInstance
+
+
+class OpenLineageListener:
+    """
+    OpenLineage listener
+    Sends events on task instance and dag run starts, completes and failures.
+    """
+
+    def __init__(self):
+        self.log = logging.getLogger(__name__)
+        self.executor: Executor = None  # type: ignore
+        self.extractor_manager = ExtractorManager()
+        self.adapter = OpenLineageAdapter()
+
+    @hookimpl
+    def on_task_instance_running(
+        self, previous_state, task_instance: TaskInstance, session: Session  # This will always be QUEUED
+    ):
+        if not hasattr(task_instance, "task"):
+            self.log.warning(
+                f"No task set for TI object task_id: {task_instance.task_id} - "
+                f"dag_id: {task_instance.dag_id} - run_id {task_instance.run_id}"
+            )
+            return
+
+        self.log.debug("OpenLineage listener got notification about task instance start")
+        dagrun = task_instance.dag_run
+        task = task_instance.task
+        dag = task.dag
+
+        @print_exception
+        def on_running():
+            # that's a workaround to detect task running from deferred state
+            # we return here because Airflow 2.3 needs task from deferred state
+            if task_instance.next_method is not None:
+                return
+            parent_run_id = self.adapter.build_dag_run_id(dag.dag_id, dagrun.run_id)
+
+            task_uuid = self.adapter.build_task_instance_run_id(
+                task.task_id, task_instance.execution_date, task_instance.try_number
+            )
+
+            task_metadata = self.extractor_manager.extract_metadata(dagrun, task)
+
+            self.adapter.start_task(
+                run_id=task_uuid,
+                job_name=get_job_name(task),
+                job_description=dag.description,
+                event_time=task_instance.start_date.isoformat(),
+                parent_job_name=dag.dag_id,
+                parent_run_id=parent_run_id,
+                code_location=get_task_location(task),
+                nominal_start_time=dagrun.data_interval_start.isoformat(),
+                nominal_end_time=dagrun.data_interval_end.isoformat(),
+                owners=dag.owner.split(", "),
+                task=task_metadata,
+                run_facets={
+                    **task_metadata.run_facets,
+                    **get_custom_facets(dagrun, task, dagrun.external_trigger, task_instance),
+                    **get_airflow_run_facet(dagrun, dag, task_instance, task, task_uuid),
+                },
+            )
+
+        self.executor.submit(on_running)
+
+    @hookimpl
+    def on_task_instance_success(self, previous_state, task_instance: TaskInstance, session):
+        self.log.debug("OpenLineage listener got notification about task instance success")
+
+        dagrun = task_instance.dag_run
+        task = task_instance.task
+
+        task_uuid = OpenLineageAdapter.build_task_instance_run_id(
+            task.task_id, task_instance.execution_date, task_instance.try_number - 1
+        )
+
+        @print_exception
+        def on_success():
+            task_metadata = self.extractor_manager.extract_metadata(
+                dagrun, task, complete=True, task_instance=task_instance
+            )
+            self.adapter.complete_task(
+                run_id=task_uuid,
+                job_name=get_job_name(task),
+                end_time=task_instance.end_date.isoformat(),
+                task=task_metadata,
+            )
+
+        self.executor.submit(on_success)
+
+    @hookimpl
+    def on_task_instance_failed(self, previous_state, task_instance: TaskInstance, session):
+        self.log.debug("OpenLineage listener got notification about task instance failure")
+
+        dagrun = task_instance.dag_run
+        task = task_instance.task
+
+        task_uuid = OpenLineageAdapter.build_task_instance_run_id(
+            task.task_id, task_instance.execution_date, task_instance.try_number - 1
+        )
+
+        @print_exception
+        def on_failure():
+            task_metadata = self.extractor_manager.extract_metadata(
+                dagrun, task, complete=True, task_instance=task_instance
+            )
+
+            self.adapter.fail_task(
+                run_id=task_uuid,
+                job_name=get_job_name(task),
+                end_time=task_instance.end_date.isoformat(),
+                task=task_metadata,
+            )
+
+        self.executor.submit(on_failure)
+
+    @hookimpl
+    def on_starting(self, component):
+        self.log.debug("on_starting: %s", component.__class__.__name__)
+        self.executor = ThreadPoolExecutor(max_workers=8, thread_name_prefix="openlineage_")
+
+    @hookimpl
+    def before_stopping(self, component):
+        self.log.debug("before_stopping: %s", component.__class__.__name__)
+        self.executor.shutdown(wait=True)

Review Comment:
   Perhaps we could make the timeout adjustable, with a reasonable default value?



##########
airflow/providers/openlineage/plugins/listener.py:
##########
@@ -0,0 +1,189 @@
+# 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 logging
+from concurrent.futures import Executor, ThreadPoolExecutor
+from typing import TYPE_CHECKING
+
+from airflow.listeners import hookimpl
+from airflow.providers.openlineage.extractors import ExtractorManager
+from airflow.providers.openlineage.plugins.adapter import OpenLineageAdapter
+from airflow.providers.openlineage.utils import (
+    get_airflow_run_facet,
+    get_custom_facets,
+    get_job_name,
+    get_task_location,
+    print_exception,
+)
+
+if TYPE_CHECKING:
+    from sqlalchemy.orm import Session
+
+    from airflow.models import DagRun, TaskInstance
+
+
+class OpenLineageListener:
+    """
+    OpenLineage listener
+    Sends events on task instance and dag run starts, completes and failures.
+    """
+
+    def __init__(self):
+        self.log = logging.getLogger(__name__)
+        self.executor: Executor = None  # type: ignore
+        self.extractor_manager = ExtractorManager()
+        self.adapter = OpenLineageAdapter()
+
+    @hookimpl
+    def on_task_instance_running(
+        self, previous_state, task_instance: TaskInstance, session: Session  # This will always be QUEUED
+    ):
+        if not hasattr(task_instance, "task"):
+            self.log.warning(
+                f"No task set for TI object task_id: {task_instance.task_id} - "
+                f"dag_id: {task_instance.dag_id} - run_id {task_instance.run_id}"
+            )
+            return
+
+        self.log.debug("OpenLineage listener got notification about task instance start")
+        dagrun = task_instance.dag_run
+        task = task_instance.task
+        dag = task.dag
+
+        @print_exception
+        def on_running():
+            # that's a workaround to detect task running from deferred state
+            # we return here because Airflow 2.3 needs task from deferred state
+            if task_instance.next_method is not None:
+                return
+            parent_run_id = self.adapter.build_dag_run_id(dag.dag_id, dagrun.run_id)
+
+            task_uuid = self.adapter.build_task_instance_run_id(
+                task.task_id, task_instance.execution_date, task_instance.try_number
+            )
+
+            task_metadata = self.extractor_manager.extract_metadata(dagrun, task)
+
+            self.adapter.start_task(
+                run_id=task_uuid,
+                job_name=get_job_name(task),
+                job_description=dag.description,
+                event_time=task_instance.start_date.isoformat(),
+                parent_job_name=dag.dag_id,
+                parent_run_id=parent_run_id,
+                code_location=get_task_location(task),
+                nominal_start_time=dagrun.data_interval_start.isoformat(),
+                nominal_end_time=dagrun.data_interval_end.isoformat(),
+                owners=dag.owner.split(", "),
+                task=task_metadata,
+                run_facets={
+                    **task_metadata.run_facets,
+                    **get_custom_facets(dagrun, task, dagrun.external_trigger, task_instance),
+                    **get_airflow_run_facet(dagrun, dag, task_instance, task, task_uuid),
+                },
+            )
+
+        self.executor.submit(on_running)
+
+    @hookimpl
+    def on_task_instance_success(self, previous_state, task_instance: TaskInstance, session):
+        self.log.debug("OpenLineage listener got notification about task instance success")
+
+        dagrun = task_instance.dag_run
+        task = task_instance.task
+
+        task_uuid = OpenLineageAdapter.build_task_instance_run_id(
+            task.task_id, task_instance.execution_date, task_instance.try_number - 1
+        )
+
+        @print_exception
+        def on_success():
+            task_metadata = self.extractor_manager.extract_metadata(
+                dagrun, task, complete=True, task_instance=task_instance
+            )
+            self.adapter.complete_task(
+                run_id=task_uuid,
+                job_name=get_job_name(task),
+                end_time=task_instance.end_date.isoformat(),
+                task=task_metadata,
+            )
+
+        self.executor.submit(on_success)
+
+    @hookimpl
+    def on_task_instance_failed(self, previous_state, task_instance: TaskInstance, session):
+        self.log.debug("OpenLineage listener got notification about task instance failure")
+
+        dagrun = task_instance.dag_run
+        task = task_instance.task
+
+        task_uuid = OpenLineageAdapter.build_task_instance_run_id(
+            task.task_id, task_instance.execution_date, task_instance.try_number - 1
+        )
+
+        @print_exception
+        def on_failure():
+            task_metadata = self.extractor_manager.extract_metadata(
+                dagrun, task, complete=True, task_instance=task_instance
+            )
+
+            self.adapter.fail_task(
+                run_id=task_uuid,
+                job_name=get_job_name(task),
+                end_time=task_instance.end_date.isoformat(),
+                task=task_metadata,
+            )
+
+        self.executor.submit(on_failure)
+
+    @hookimpl
+    def on_starting(self, component):
+        self.log.debug("on_starting: %s", component.__class__.__name__)
+        self.executor = ThreadPoolExecutor(max_workers=8, thread_name_prefix="openlineage_")
+
+    @hookimpl
+    def before_stopping(self, component):
+        self.log.debug("before_stopping: %s", component.__class__.__name__)
+        self.executor.shutdown(wait=True)

Review Comment:
   Perhaps we could make the timeout configurable, with a reasonable default value?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1135359407


##########
airflow/providers/openlineage/extractors/manager.py:
##########
@@ -0,0 +1,119 @@
+# 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
+
+from airflow.providers.openlineage.extractors import BaseExtractor, Extractors, OperatorLineage
+from airflow.providers.openlineage.plugins.facets import (
+    UnknownOperatorAttributeRunFacet,
+    UnknownOperatorInstance,
+)
+from airflow.providers.openlineage.utils import get_operator_class
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class ExtractorManager(LoggingMixin):
+    """Class abstracting management of custom extractors."""
+
+    def __init__(self):
+        self.extractors = {}
+        self.task_to_extractor = Extractors()
+
+    def add_extractor(self, operator, extractor: type[BaseExtractor]):
+        self.task_to_extractor.add_extractor(operator, extractor)
+
+    def extract_metadata(self, dagrun, task, complete: bool = False, task_instance=None) -> OperatorLineage:
+        extractor = self._get_extractor(task)
+        task_info = (
+            f"task_type={get_operator_class(task).__name__} "

Review Comment:
   Fixed.



##########
airflow/providers/openlineage/extractors/manager.py:
##########
@@ -0,0 +1,119 @@
+# 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
+
+from airflow.providers.openlineage.extractors import BaseExtractor, Extractors, OperatorLineage
+from airflow.providers.openlineage.plugins.facets import (
+    UnknownOperatorAttributeRunFacet,
+    UnknownOperatorInstance,
+)
+from airflow.providers.openlineage.utils import get_operator_class
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class ExtractorManager(LoggingMixin):
+    """Class abstracting management of custom extractors."""
+
+    def __init__(self):
+        self.extractors = {}
+        self.task_to_extractor = Extractors()
+
+    def add_extractor(self, operator, extractor: type[BaseExtractor]):
+        self.task_to_extractor.add_extractor(operator, extractor)
+
+    def extract_metadata(self, dagrun, task, complete: bool = False, task_instance=None) -> OperatorLineage:
+        extractor = self._get_extractor(task)
+        task_info = (
+            f"task_type={get_operator_class(task).__name__} "
+            f"airflow_dag_id={task.dag_id} "
+            f"task_id={task.task_id} "
+            f"airflow_run_id={dagrun.run_id} "
+        )
+
+        if extractor:
+            # Extracting advanced metadata is only possible when extractor for particular operator
+            # is defined. Without it, we can't extract any input or output data.
+            try:
+                self.log.debug("Using extractor %s %s", extractor.__class__.__name__, str(task_info))
+                if complete:
+                    task_metadata = extractor.extract_on_complete(task_instance)
+                else:
+                    task_metadata = extractor.extract()
+
+                self.log.debug("Found task metadata for operation %s: %s", task.task_id, str(task_metadata))
+                if task_metadata:
+                    if (not task_metadata.inputs) and (not task_metadata.outputs):
+                        inlets = task.get_inlet_defs()
+                        outlets = task.get_outlet_defs()
+                        self.extract_inlets_and_outlets(task_metadata, inlets, outlets)
+
+                    return task_metadata
+
+            except Exception as e:
+                self.log.exception("Failed to extract metadata %s %s", e, task_info)
+        else:
+            self.log.warning("Unable to find an extractor %s", task_info)
+
+            # Only include the unkonwnSourceAttribute facet if there is no extractor
+            task_metadata = OperatorLineage(
+                run_facets={
+                    "unknownSourceAttribute": UnknownOperatorAttributeRunFacet(
+                        unknownItems=[
+                            UnknownOperatorInstance(
+                                name=get_operator_class(task).__name__,
+                                properties={attr: value for attr, value in task.__dict__.items()},
+                            )
+                        ]
+                    )
+                },
+            )
+            inlets = task.get_inlet_defs()
+            outlets = task.get_outlet_defs()
+            self.extract_inlets_and_outlets(task_metadata, inlets, outlets)
+            return task_metadata
+
+        return OperatorLineage()
+
+    def _get_extractor(self, task) -> BaseExtractor | None:
+        # TODO: Re-enable in Extractor PR
+        # self.task_to_extractor.instantiate_abstract_extractors(task)
+        if task.task_id in self.extractors:
+            return self.extractors[task.task_id]

Review Comment:
   Right now we don't - removed that part.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1157615110


##########
airflow/providers/openlineage/provider.yaml:
##########
@@ -0,0 +1,42 @@
+# 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.
+
+---
+package-name: apache-airflow-providers-openlineage
+name: OpenLineage Airflow
+description: |
+  `OpenLineage <https://openlineage.io/>`__
+
+versions:
+  - 1.0.0
+
+dependencies:
+  - apache-airflow>=2.5.1
+  - apache-airflow-providers-common-sql>=1.3.1
+  - attrs>=22.2
+  - openlineage-integration-common>=0.20.6
+  - openlineage-python>=0.20.6
+
+integrations:
+  - integration-name: OpenLineage
+    external-doc-url: https://openlineage.io
+    logo: /integration-logos/openlineage/openlineage.svg
+    tags: [apache]

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] ashb commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "ashb (via GitHub)" <gi...@apache.org>.
ashb commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1159777616


##########
airflow/providers/openlineage/utils/utils.py:
##########
@@ -0,0 +1,482 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from contextlib import suppress
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attrs
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def redacted_connection_uri(conn: Connection, filtered_params=None, filtered_prefixes=None):
+    """
+    Return the connection URI for the given Connection.
+    This method additionally filters URI by removing query parameters that are known to carry sensitive data
+    like username, password, access key.
+    """
+    if filtered_prefixes is None:
+        filtered_prefixes = []
+    if filtered_params is None:
+        filtered_params = []
+
+    def filter_key_params(k: str):
+        return k not in filtered_params and any(substr in k for substr in filtered_prefixes)
+
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not filter_key_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def get_normalized_postgres_connection_uri(conn):
+    """
+    URIs starting with postgresql:// and postgres:// are both valid
+    PostgreSQL connection strings. This function normalizes it to
+    postgres:// as canonical name according to OpenLineage spec.
+    """
+    uri = redacted_connection_uri(conn)
+    if uri.startswith("postgresql"):
+        uri = uri.replace("postgresql", "postgres", 1)
+    return uri
+
+
+def get_connection(conn_id) -> Connection | None:
+    from airflow.hooks.base import BaseHook
+
+    with suppress(Exception):
+        return BaseHook.get_connection(conn_id=conn_id)
+    return None
+
+
+def get_job_name(task):
+    return f"{task.dag_id}.{task.task_id}"
+
+
+def get_custom_facets(
+    dagrun, task, is_external_trigger: bool, task_instance: TaskInstance | None = None
+) -> dict[str, Any]:
+    custom_facets = {
+        "airflow_runArgs": AirflowRunArgsRunFacet(is_external_trigger),
+        "airflow_version": AirflowVersionRunFacet.from_dagrun_and_task(dagrun, task),
+    }
+    # check for -1 comes from SmartSensor compatibility with dynamic task mapping
+    # this comes from Airflow code
+    if hasattr(task_instance, "map_index") and getattr(task_instance, "map_index") != -1:
+        custom_facets["airflow_mappedTask"] = AirflowMappedTaskRunFacet.from_task_instance(task_instance)
+    return custom_facets
+
+
+class InfoJsonEncodable(dict):
+    """
+    Airflow objects might not be json-encodable overall.
+
+    The class provides additional attributes to control
+    what and how is encoded:
+    * renames: a dictionary of attribute name changes
+    * casts: a dictionary consisting of attribute names
+             and corresponding methods that should change
+             object value
+    * includes: list of attributes to be included in encoding
+    * excludes: list of attributes to be excluded from encoding
+
+    Don't use both includes and excludes.
+    """
+
+    renames: dict[str, str] = {}
+    casts: dict[str, Any] = {}
+    includes: list[str] = []
+    excludes: list[str] = []
+
+    def __init__(self, obj):
+        self.obj = obj
+        self._fields = []
+
+        self._cast_fields()
+        self._rename_fields()
+        self._include_fields()
+        dict.__init__(
+            self,
+            **{field: InfoJsonEncodable._cast_basic_types(getattr(self, field)) for field in self._fields},
+        )
+
+    @staticmethod
+    def _cast_basic_types(value):
+        if isinstance(value, datetime.datetime):
+            return value.isoformat()
+        if isinstance(value, (set, list, tuple)):
+            return str(list(value))
+        return value
+
+    def _rename_fields(self):
+        for field, renamed in self.renames.items():
+            if hasattr(self.obj, field):
+                setattr(self, renamed, getattr(self.obj, field))
+                self._fields.append(renamed)
+
+    def _cast_fields(self):
+        for field, func in self.casts.items():
+            setattr(self, field, func(self.obj))
+            self._fields.append(field)
+
+    def _include_fields(self):
+        if self.includes and self.excludes:
+            raise Exception("Don't use both includes and excludes.")
+        if self.includes:
+            for field in self.includes:
+                if field in self._fields or not hasattr(self.obj, field):
+                    continue
+                setattr(self, field, getattr(self.obj, field))
+                self._fields.append(field)
+        else:
+            for field, val in self.obj.__dict__.items():
+                if field in self._fields or field in self.excludes or field in self.renames:
+                    continue
+                setattr(self, field, val)
+                self._fields.append(field)
+
+
+class DagInfo(InfoJsonEncodable):
+    """Defines encoding DAG object to JSON."""
+
+    includes = ["dag_id", "schedule_interval", "tags", "start_date"]
+    casts = {"timetable": lambda dag: dag.timetable.serialize() if getattr(dag, "timetable", None) else None}
+    renames = {"_dag_id": "dag_id"}
+
+
+class DagRunInfo(InfoJsonEncodable):
+    """Defines encoding DagRun object to JSON."""
+
+    includes = [
+        "conf",
+        "dag_id",
+        "data_interval_start",
+        "data_interval_end",
+        "external_trigger",
+        "run_id",
+        "run_type",
+        "start_date",
+    ]
+
+
+class TaskInstanceInfo(InfoJsonEncodable):
+    """Defines encoding TaskInstance object to JSON."""
+
+    includes = ["duration", "try_number", "pool"]
+    casts = {
+        "map_index": lambda ti: ti.map_index
+        if hasattr(ti, "map_index") and getattr(ti, "map_index") != -1
+        else None
+    }
+
+
+class TaskInfo(InfoJsonEncodable):
+    """Defines encoding BaseOperator/AbstractOperator object to JSON."""
+
+    renames = {
+        "_BaseOperator__init_kwargs": "args",
+        "_BaseOperator__from_mapped": "mapped",
+        "_downstream_task_ids": "downstream_task_ids",
+        "_upstream_task_ids": "upstream_task_ids",
+    }
+    excludes = [
+        "_BaseOperator__instantiated",
+        "_dag",
+        "_hook",
+        "_log",
+        "_outlets",
+        "_inlets",
+        "_lock_for_execution",
+        "handler",
+        "params",
+        "python_callable",
+        "retry_delay",
+    ]
+    casts = {
+        "operator_class": lambda task: f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",  # noqa
+        "task_group": lambda task: TaskGroupInfo(task.task_group)
+        if hasattr(task, "task_group") and getattr(task.task_group, "_group_id", None)
+        else None,
+    }
+
+
+class TaskGroupInfo(InfoJsonEncodable):
+    """Defines encoding TaskGroup object to JSON."""
+
+    renames = {
+        "_group_id": "group_id",
+    }
+    includes = [
+        "downstream_group_ids",
+        "downstream_task_ids",
+        "prefix_group_id",
+        "tooltip",
+        "upstream_group_ids",
+        "upstream_task_ids",
+    ]
+
+
+def get_airflow_run_facet(
+    dag_run: DagRun,
+    dag: DAG,
+    task_instance: TaskInstance,
+    task: BaseOperator,
+    task_uuid: str,
+):
+    return {
+        "airflow": json.loads(
+            json.dumps(
+                asdict(
+                    AirflowRunFacet(
+                        dag=DagInfo(dag),
+                        dagRun=DagRunInfo(dag_run),
+                        taskInstance=TaskInstanceInfo(task_instance),
+                        task=TaskInfo(task),
+                        taskUuid=task_uuid,
+                    )
+                ),
+                default=str,
+            )
+        )
+    }
+
+
+def import_from_string(path: str):
+    try:
+        module_path, target = path.rsplit(".", 1)
+        module = importlib.import_module(module_path)
+        return getattr(module, target)
+    except Exception as e:
+        log.warning(e)
+        raise ImportError(f"Failed to import {path}") from e
+
+
+def try_import_from_string(path: str):
+    with suppress(ImportError):
+        return import_from_string(path)
+
+
+def redact_with_exclusions(source: Any):
+    """This function redacts sensitive data similar to SecretsMasker in Airflow logs.
+    The difference is that MAX_RECURSION_DEPTH is way higher - due to the structure of OL events
+    we need more depth.
+    Also, we allow data structures to specify data that needs not to be redacted by specifying
+    _skip_redact list.
+    """
+    try:
+        from airflow.utils.log.secrets_masker import (
+            _secrets_masker,
+            should_hide_value_for_key,
+        )
+    except ImportError:
+        return source
+
+    sm = _secrets_masker()
+    MAX_RECURSION_DEPTH = 20
+
+    def _redact(item, name: str | None, depth: int):
+        if depth > MAX_RECURSION_DEPTH:
+            return item

Review Comment:
   https://github.com/apache/airflow/pull/30505 for the depth path.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1160771045


##########
airflow/providers/openlineage/utils/utils.py:
##########
@@ -0,0 +1,482 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from contextlib import suppress
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attrs
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def redacted_connection_uri(conn: Connection, filtered_params=None, filtered_prefixes=None):
+    """
+    Return the connection URI for the given Connection.
+    This method additionally filters URI by removing query parameters that are known to carry sensitive data
+    like username, password, access key.
+    """
+    if filtered_prefixes is None:
+        filtered_prefixes = []
+    if filtered_params is None:
+        filtered_params = []
+
+    def filter_key_params(k: str):
+        return k not in filtered_params and any(substr in k for substr in filtered_prefixes)
+
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not filter_key_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def get_normalized_postgres_connection_uri(conn):
+    """
+    URIs starting with postgresql:// and postgres:// are both valid
+    PostgreSQL connection strings. This function normalizes it to
+    postgres:// as canonical name according to OpenLineage spec.
+    """
+    uri = redacted_connection_uri(conn)
+    if uri.startswith("postgresql"):
+        uri = uri.replace("postgresql", "postgres", 1)
+    return uri
+
+
+def get_connection(conn_id) -> Connection | None:
+    from airflow.hooks.base import BaseHook
+
+    with suppress(Exception):
+        return BaseHook.get_connection(conn_id=conn_id)
+    return None
+
+
+def get_job_name(task):
+    return f"{task.dag_id}.{task.task_id}"
+
+
+def get_custom_facets(
+    dagrun, task, is_external_trigger: bool, task_instance: TaskInstance | None = None
+) -> dict[str, Any]:
+    custom_facets = {
+        "airflow_runArgs": AirflowRunArgsRunFacet(is_external_trigger),
+        "airflow_version": AirflowVersionRunFacet.from_dagrun_and_task(dagrun, task),

Review Comment:
   Removed those facets.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] JDarDagran commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "JDarDagran (via GitHub)" <gi...@apache.org>.
JDarDagran commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1134147706


##########
airflow/providers/openlineage/plugins/macros.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.
+from __future__ import annotations
+
+import os
+import typing
+
+from airflow.providers.openlineage.plugins.adapter import OpenLineageAdapter
+
+if typing.TYPE_CHECKING:
+    from airflow.models import BaseOperator, TaskInstance
+
+_JOB_NAMESPACE = os.getenv("OPENLINEAGE_NAMESPACE", "default")
+
+
+def lineage_run_id(task: BaseOperator, task_instance: TaskInstance):
+    """
+    Macro function which returns the generated run id for a given task. This
+    can be used to forward the run id from a task to a child run so the job
+    hierarchy is preserved. Invoke as a jinja template, e.g.
+
+    PythonOperator(
+        task_id='render_template',
+        python_callable=my_task_function,
+        op_args=['{{ lineage_run_id(task, task_instance) }}'], # lineage_run_id macro invoked

Review Comment:
   You're right, `task_instance` should be enough in this case.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on PR #29940:
URL: https://github.com/apache/airflow/pull/29940#issuecomment-1473722541

   @bolkedebruin @potiuk Agreed, I'll prioritize documenting and explaining what happens here now. Generally I was thinking to add user-facing docs in a later PR, just as we have docs [here](https://openlineage.io/docs/integrations/airflow/) but I understand having more developer-focused docs earlier is better.  


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1156108369


##########
airflow/providers/openlineage/utils/utils.py:
##########
@@ -0,0 +1,482 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from contextlib import suppress
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attrs
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:

Review Comment:
   Removed, probably another 1.10 relic



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1156107624


##########
airflow/providers/openlineage/utils/utils.py:
##########
@@ -0,0 +1,482 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from contextlib import suppress
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attrs
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG

Review Comment:
   I made it just import `DAG` - it's another relic of 1.10 integration. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on PR #29940:
URL: https://github.com/apache/airflow/pull/29940#issuecomment-1494603822

   @ashb 
   
   >Can you explain to me what we are doing with all the git commands in ol/utils/utils.py? Where's that used and where does it end up?
   
   It ends up as `SourceCodeLocationJobFacet` https://openlineage.io/docs/spec/facets/job-facets/source-code-location
   
   >(I'm a little bit worried that some of the utils code "feels" like a lift-and-shift from the external repo without tying it in to Airflow's way of doing things. This isn't actionable feedback, mostly a comment at this point)
   
   Yeah - it's basically that. We wanted to keep Airflow specific changes - like adding Airflow config support - for later PRs, but given how long this is in review it was a bad idea. Maybe we should treat this instead as a rewrite from the beginning.
   
   There was also way more old cruft than I've thought... that's definitely on me.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1155878610


##########
airflow/providers/openlineage/utils/utils.py:
##########
@@ -0,0 +1,482 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from contextlib import suppress
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attrs
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__

Review Comment:
   Removed this as we're just using `task.task_type`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1157583751


##########
airflow/providers/openlineage/extractors/base.py:
##########
@@ -0,0 +1,111 @@
+# 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
+
+from abc import ABC, abstractmethod
+
+from attrs import Factory, define
+
+from airflow.utils.log.logging_mixin import LoggingMixin
+from openlineage.client.facet import BaseFacet
+from openlineage.client.run import Dataset
+
+
+@define
+class OperatorLineage:
+    """Structure returned from lineage extraction."""
+
+    inputs: list[Dataset] = Factory(list)
+    outputs: list[Dataset] = Factory(list)
+    run_facets: dict[str, BaseFacet] = Factory(dict)
+    job_facets: dict[str, BaseFacet] = Factory(dict)
+
+
+class BaseExtractor(ABC, LoggingMixin):
+    """
+    Abstract base extractor class.
+
+    This is used mostly to maintain support for custom extractors.
+    """
+
+    _allowed_query_params: list[str] = []
+
+    def __init__(self, operator):

Review Comment:
   We'd probably need to force extractors then to have some class-level member that shows what type `self.operator` is, due to mypy throwing huge list of extractor errors if they assume `BaseOperator`. Or just override `__init__`  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] eladkal commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "eladkal (via GitHub)" <gi...@apache.org>.
eladkal commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1160812862


##########
airflow/providers/openlineage/extractors/base.py:
##########
@@ -0,0 +1,106 @@
+# 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
+
+from abc import ABC, abstractmethod
+
+from attrs import Factory, define
+
+from airflow.utils.log.logging_mixin import LoggingMixin
+from openlineage.client.facet import BaseFacet
+from openlineage.client.run import Dataset
+
+
+@define
+class OperatorLineage:
+    """Structure returned from lineage extraction."""
+
+    inputs: list[Dataset] = Factory(list)
+    outputs: list[Dataset] = Factory(list)
+    run_facets: dict[str, BaseFacet] = Factory(dict)
+    job_facets: dict[str, BaseFacet] = Factory(dict)
+
+
+class BaseExtractor(ABC, LoggingMixin):
+    """
+    Abstract base extractor class.
+
+    This is used mostly to maintain support for custom extractors.
+    """
+
+    _allowed_query_params: list[str] = []
+
+    def __init__(self, operator):  # type: ignore
+        super().__init__()
+        self.operator = operator
+
+    @classmethod
+    @abstractmethod
+    def get_operator_classnames(cls) -> list[str]:
+        """
+        Implement this method returning list of operators that extractor works for.
+
+        Particularly, in Airflow 2 some operators are deprecated and simply subclass the new
+        implementation, for example airflow.contrib.operators.bigquery_operator.BigQueryOperator.
+        The BigQueryExtractor needs to work with both of them.

Review Comment:
   Please explain this.
   There is no `BigQueryOperator` in the code base.
   
   Deprecated operators are backward compatible. They should not have special treatment.



##########
docs/spelling_wordlist.txt:
##########
@@ -161,6 +161,7 @@ BestCandidate
 Bigquery
 bigquery
 BigQueryHook
+BigQueryOperator

Review Comment:
   This is not needed, you should mark the place where you use is as code then the spelling check will know to treat it as such.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] ashb merged pull request #29940: Add base OpenLineage provider implementation

Posted by "ashb (via GitHub)" <gi...@apache.org>.
ashb merged PR #29940:
URL: https://github.com/apache/airflow/pull/29940


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1135416785


##########
airflow/providers/openlineage/plugins/adapter.py:
##########
@@ -0,0 +1,302 @@
+# 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
+import uuid
+from typing import TYPE_CHECKING
+
+import requests.exceptions
+
+from airflow.providers.openlineage import version as OPENLINEAGE_PROVIDER_VERSION
+from airflow.providers.openlineage.extractors import OperatorLineage
+from airflow.providers.openlineage.utils import redact_with_exclusions
+from airflow.utils.log.logging_mixin import LoggingMixin
+from openlineage.client import OpenLineageClient, set_producer
+from openlineage.client.facet import (
+    BaseFacet,
+    DocumentationJobFacet,
+    ErrorMessageRunFacet,
+    NominalTimeRunFacet,
+    OwnershipJobFacet,
+    OwnershipJobFacetOwners,
+    ParentRunFacet,
+    ProcessingEngineRunFacet,
+    SourceCodeLocationJobFacet,
+)
+from openlineage.client.run import Job, Run, RunEvent, RunState
+
+if TYPE_CHECKING:
+    from airflow.models.dagrun import DagRun
+
+
+_DAG_DEFAULT_NAMESPACE = "default"
+
+_DAG_NAMESPACE = os.getenv("OPENLINEAGE_NAMESPACE", _DAG_DEFAULT_NAMESPACE)
+
+_PRODUCER = f"https://github.com/apache/airflow/tree/providers-openlineage/" f"{OPENLINEAGE_PROVIDER_VERSION}"
+
+set_producer(_PRODUCER)
+
+
+class OpenLineageAdapter(LoggingMixin):
+    """
+    Adapter for translating Airflow metadata to OpenLineage events,
+    instead of directly creating them from Airflow code.
+    """
+
+    def __init__(self, client=None):
+        super().__init__()
+        self._client = client
+
+    def get_or_create_openlineage_client(self) -> OpenLineageClient:
+        if not self._client:
+            self._client = OpenLineageClient.from_environment()
+        return self._client
+
+    def build_dag_run_id(self, dag_id, dag_run_id):
+        return str(uuid.uuid3(uuid.NAMESPACE_URL, f"{_DAG_NAMESPACE}.{dag_id}.{dag_run_id}"))
+
+    @staticmethod
+    def build_task_instance_run_id(task_id, execution_date, try_number):
+        return str(
+            uuid.uuid3(
+                uuid.NAMESPACE_URL,
+                f"{_DAG_NAMESPACE}.{task_id}.{execution_date}.{try_number}",
+            )
+        )
+
+    def emit(self, event: RunEvent):
+        event = redact_with_exclusions(event)
+        try:
+            return self.get_or_create_openlineage_client().emit(event)
+        except requests.exceptions.RequestException:
+            self.log.exception(f"Failed to emit OpenLineage event of id {event.run.runId}")
+
+    def start_task(
+        self,
+        run_id: str,
+        job_name: str,
+        job_description: str,
+        event_time: str,
+        parent_job_name: str | None,
+        parent_run_id: str | None,
+        code_location: str | None,
+        nominal_start_time: str,
+        nominal_end_time: str,
+        owners: list[str],
+        task: OperatorLineage | None,
+        run_facets: dict[str, type[BaseFacet]] | None = None,  # Custom run facets
+    ) -> str:
+        """
+        Emits openlineage event of type START

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1135360374


##########
airflow/providers/openlineage/extractors/manager.py:
##########
@@ -0,0 +1,119 @@
+# 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
+
+from airflow.providers.openlineage.extractors import BaseExtractor, Extractors, OperatorLineage
+from airflow.providers.openlineage.plugins.facets import (
+    UnknownOperatorAttributeRunFacet,
+    UnknownOperatorInstance,
+)
+from airflow.providers.openlineage.utils import get_operator_class
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class ExtractorManager(LoggingMixin):
+    """Class abstracting management of custom extractors."""
+
+    def __init__(self):
+        self.extractors = {}
+        self.task_to_extractor = Extractors()
+
+    def add_extractor(self, operator, extractor: type[BaseExtractor]):
+        self.task_to_extractor.add_extractor(operator, extractor)
+
+    def extract_metadata(self, dagrun, task, complete: bool = False, task_instance=None) -> OperatorLineage:
+        extractor = self._get_extractor(task)
+        task_info = (
+            f"task_type={get_operator_class(task).__name__} "
+            f"airflow_dag_id={task.dag_id} "
+            f"task_id={task.task_id} "
+            f"airflow_run_id={dagrun.run_id} "
+        )
+
+        if extractor:
+            # Extracting advanced metadata is only possible when extractor for particular operator
+            # is defined. Without it, we can't extract any input or output data.
+            try:
+                self.log.debug("Using extractor %s %s", extractor.__class__.__name__, str(task_info))
+                if complete:
+                    task_metadata = extractor.extract_on_complete(task_instance)
+                else:
+                    task_metadata = extractor.extract()
+
+                self.log.debug("Found task metadata for operation %s: %s", task.task_id, str(task_metadata))
+                if task_metadata:
+                    if (not task_metadata.inputs) and (not task_metadata.outputs):
+                        inlets = task.get_inlet_defs()
+                        outlets = task.get_outlet_defs()
+                        self.extract_inlets_and_outlets(task_metadata, inlets, outlets)
+
+                    return task_metadata
+
+            except Exception as e:
+                self.log.exception("Failed to extract metadata %s %s", e, task_info)
+        else:
+            self.log.warning("Unable to find an extractor %s", task_info)

Review Comment:
   Moved to `debug`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on pull request #29940: Add base OpenLineage provider implementation

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on PR #29940:
URL: https://github.com/apache/airflow/pull/29940#issuecomment-1473673118

   > Can I say that this integration needs extensive docs and architecture description (inside those docs)? It is quite opaque to me know how this works, why it needs workers at all, what it does to my running system to have those workers, how does it affect task runs, what if the code fails did my task fail? etc etc.
   
   Yeah. I think there are quite a number of things here to make decisions on. I am also going to have a closer look and make a more deep review after the presentation next week, I hope Iy will have much more context after seeing some of the decisions and reasoning for the open-lineage architecture there. I recall how useful it was to get a walkthrough by @amoshb of the new scheduler architecture and decisions back in the 2.0 days (and then seeing the "Deep dive" talk from the summit) - this allowed those who participated/watched (and paid attention :) ) to better reason in case of future issues/questions and be able to fix problem or diagnose issues or to propose improvements (even though some of the details there are a bit arcane). I wish for example we had something like that for the Celery Executor or K8S integration when it comes to stalling, adoption, log streaming etc.. 
   
   That's also note for  @mobuchowski and  @julienledem. The more of those decisions and contex will be explained, documented (including recording and publishing the meeting is a good idea - ideally followed up by a talk on the Summit) the more it will be a community effort.
   
   For example I am planning to submit a talk for the summit with a working title "Everything you even did not know you wanted to ask about the Airflow CI (or was terrified to ask)" to address at least part of the SPOF problem we have there and pave the way to get others at least being able to reason on where to fix when there are problems. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] julienledem commented on pull request #29940: Add base OpenLineage provider implementation

Posted by "julienledem (via GitHub)" <gi...@apache.org>.
julienledem commented on PR #29940:
URL: https://github.com/apache/airflow/pull/29940#issuecomment-1483267080

   > It is quite opaque to me know how this works, why it needs workers at all, what it does to my running system to have those workers, how does it affect task runs, what if the code fails did my task fail? etc etc.
   
   To make the documentation effort efficient, we need a list of the points that need documenting. 
   Starting from your comment, we have:
    - purpose of worker in this PR
    - impact of workers
    - how we isolate the running job from failures in the listener
    
   The purpose of the worker is to isolate the user task from failures or latency in the OL event sending. (I'll let @mobuchowski expand on that)
   
   @bolkedebruin could you elaborate on the other points that need explanation from your perspective?
   
   Since this code is being moved, a lot of the history is documented in the PRs that happened on the OL project: 
   https://github.com/OpenLineage/OpenLineage/pulls?q=is%3Apr+-label%3Adependabot+label%3Aintegration%2Fairflow+
   Incremental changes were documented and discussed over time there.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] ashb commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "ashb (via GitHub)" <gi...@apache.org>.
ashb commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1128111212


##########
airflow/providers/openlineage/extractors/base.py:
##########
@@ -0,0 +1,133 @@
+# 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 json
+from abc import ABC, abstractmethod
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attr

Review Comment:
   Nit/what do we do elsewhere: `attr` or `attrs`



##########
airflow/providers/openlineage/extractors/base.py:
##########
@@ -0,0 +1,133 @@
+# 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 json
+from abc import ABC, abstractmethod
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attr
+
+from airflow.utils.log.logging_mixin import LoggingMixin
+from openlineage.client.facet import BaseFacet
+from openlineage.client.run import Dataset
+
+
+@attr.s

Review Comment:
   This is the recommended approach by attrs for new code
   
   ```suggestion
   @attr.define
   ```
   
   (You might need to set `frozen=False` too)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] ashb commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "ashb (via GitHub)" <gi...@apache.org>.
ashb commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1128110636


##########
airflow/providers/openlineage/extractors/__init__.py:
##########
@@ -0,0 +1,24 @@
+# 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
+
+from airflow.providers.openlineage.extractors.base import BaseExtractor, OperatorLineage
+from airflow.providers.openlineage.extractors.extractors import Extractors
+from airflow.providers.openlineage.extractors.manager import ExtractorManager
+
+__all__ = [Extractors, BaseExtractor, OperatorLineage, ExtractorManager]  # type: ignore

Review Comment:
   These should be strings, not classes



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1156105490


##########
airflow/providers/openlineage/utils/utils.py:
##########
@@ -0,0 +1,482 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from contextlib import suppress
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attrs
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def redacted_connection_uri(conn: Connection, filtered_params=None, filtered_prefixes=None):
+    """
+    Return the connection URI for the given Connection.
+    This method additionally filters URI by removing query parameters that are known to carry sensitive data
+    like username, password, access key.
+    """
+    if filtered_prefixes is None:
+        filtered_prefixes = []
+    if filtered_params is None:
+        filtered_params = []
+
+    def filter_key_params(k: str):
+        return k not in filtered_params and any(substr in k for substr in filtered_prefixes)
+
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not filter_key_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def get_normalized_postgres_connection_uri(conn):
+    """
+    URIs starting with postgresql:// and postgres:// are both valid
+    PostgreSQL connection strings. This function normalizes it to
+    postgres:// as canonical name according to OpenLineage spec.
+    """
+    uri = redacted_connection_uri(conn)
+    if uri.startswith("postgresql"):
+        uri = uri.replace("postgresql", "postgres", 1)
+    return uri
+
+
+def get_connection(conn_id) -> Connection | None:
+    from airflow.hooks.base import BaseHook
+
+    with suppress(Exception):
+        return BaseHook.get_connection(conn_id=conn_id)
+    return None
+
+
+def get_job_name(task):
+    return f"{task.dag_id}.{task.task_id}"
+
+
+def get_custom_facets(
+    dagrun, task, is_external_trigger: bool, task_instance: TaskInstance | None = None
+) -> dict[str, Any]:
+    custom_facets = {
+        "airflow_runArgs": AirflowRunArgsRunFacet(is_external_trigger),
+        "airflow_version": AirflowVersionRunFacet.from_dagrun_and_task(dagrun, task),

Review Comment:
   At some point we want to remove those. The functionality is added in `AirflowRunFacet`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1157442420


##########
airflow/providers/openlineage/extractors/base.py:
##########
@@ -0,0 +1,111 @@
+# 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
+
+from abc import ABC, abstractmethod
+
+from attrs import Factory, define
+
+from airflow.utils.log.logging_mixin import LoggingMixin
+from openlineage.client.facet import BaseFacet
+from openlineage.client.run import Dataset
+
+
+@define
+class OperatorLineage:
+    """Structure returned from lineage extraction."""
+
+    inputs: list[Dataset] = Factory(list)
+    outputs: list[Dataset] = Factory(list)
+    run_facets: dict[str, BaseFacet] = Factory(dict)
+    job_facets: dict[str, BaseFacet] = Factory(dict)
+
+
+class BaseExtractor(ABC, LoggingMixin):
+    """
+    Abstract base extractor class.
+
+    This is used mostly to maintain support for custom extractors.
+    """
+
+    _allowed_query_params: list[str] = []
+
+    def __init__(self, operator):
+        super().__init__()
+        self.operator = operator
+        self.patch()
+
+    def patch(self):
+        # Extractor should register extension methods or patches to operator here
+        pass
+
+    @classmethod
+    @abstractmethod
+    def get_operator_classnames(cls) -> list[str]:
+        """
+        Implement this method returning list of operators that extractor works for.
+        Particularly, in Airflow 2 some operators are deprecated and simply subclass the new
+        implementation, for example BigQueryOperator:
+        https://github.com/apache/airflow/blob/main/airflow/contrib/operators/bigquery_operator.py

Review Comment:
   Removed, just added fully qualified class name.



##########
airflow/providers/openlineage/extractors/base.py:
##########
@@ -0,0 +1,111 @@
+# 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
+
+from abc import ABC, abstractmethod
+
+from attrs import Factory, define
+
+from airflow.utils.log.logging_mixin import LoggingMixin
+from openlineage.client.facet import BaseFacet
+from openlineage.client.run import Dataset
+
+
+@define
+class OperatorLineage:
+    """Structure returned from lineage extraction."""
+
+    inputs: list[Dataset] = Factory(list)
+    outputs: list[Dataset] = Factory(list)
+    run_facets: dict[str, BaseFacet] = Factory(dict)
+    job_facets: dict[str, BaseFacet] = Factory(dict)
+
+
+class BaseExtractor(ABC, LoggingMixin):
+    """
+    Abstract base extractor class.
+
+    This is used mostly to maintain support for custom extractors.
+    """
+
+    _allowed_query_params: list[str] = []
+
+    def __init__(self, operator):
+        super().__init__()
+        self.operator = operator
+        self.patch()
+
+    def patch(self):
+        # Extractor should register extension methods or patches to operator here
+        pass
+
+    @classmethod
+    @abstractmethod
+    def get_operator_classnames(cls) -> list[str]:
+        """
+        Implement this method returning list of operators that extractor works for.

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1156096828


##########
airflow/providers/openlineage/utils/utils.py:
##########
@@ -0,0 +1,482 @@
+# 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 datetime
+import importlib
+import json
+import logging
+import os
+import subprocess
+from contextlib import suppress
+from functools import wraps
+from typing import TYPE_CHECKING, Any
+from urllib.parse import parse_qsl, urlencode, urlparse, urlunparse
+
+import attrs
+from attrs import asdict
+
+from airflow.models import DAG as AIRFLOW_DAG
+from airflow.providers.openlineage.plugins.facets import (
+    AirflowMappedTaskRunFacet,
+    AirflowRunArgsRunFacet,
+    AirflowRunFacet,
+    AirflowVersionRunFacet,
+)
+
+# TODO: move this maybe to Airflow's logic?
+from openlineage.client.utils import RedactMixin
+
+if TYPE_CHECKING:
+    from airflow.models import DAG, BaseOperator, Connection, DagRun, TaskInstance
+
+
+log = logging.getLogger(__name__)
+_NOMINAL_TIME_FORMAT = "%Y-%m-%dT%H:%M:%S.%fZ"
+
+
+def openlineage_job_name(dag_id: str, task_id: str) -> str:
+    return f"{dag_id}.{task_id}"
+
+
+def get_operator_class(task: BaseOperator) -> type:
+    if task.__class__.__name__ in ("DecoratedMappedOperator", "MappedOperator"):
+        return task.operator_class
+    return task.__class__
+
+
+def to_json_encodable(task: BaseOperator) -> dict[str, object]:
+    def _task_encoder(obj):
+        if isinstance(obj, datetime.datetime):
+            return obj.isoformat()
+        elif isinstance(obj, AIRFLOW_DAG):
+            return {
+                "dag_id": obj.dag_id,
+                "tags": obj.tags,
+                "schedule_interval": obj.schedule_interval,
+            }
+        else:
+            return str(obj)
+
+    return json.loads(json.dumps(task.__dict__, default=_task_encoder))
+
+
+def url_to_https(url) -> str | None:
+    # Ensure URL exists
+    if not url:
+        return None
+
+    base_url = None
+    if url.startswith("git@"):
+        part = url.split("git@")[1:2]
+        if part:
+            base_url = f'https://{part[0].replace(":", "/", 1)}'
+    elif url.startswith("https://"):
+        base_url = url
+
+    if not base_url:
+        raise ValueError(f"Unable to extract location from: {url}")
+
+    if base_url.endswith(".git"):
+        base_url = base_url[:-4]
+    return base_url
+
+
+def get_location(file_path) -> str | None:
+    # Ensure file path exists
+    if not file_path:
+        return None
+
+    # move to the file directory
+    abs_path = os.path.abspath(file_path)
+    file_name = os.path.basename(file_path)
+    cwd = os.path.dirname(abs_path)
+
+    # get the repo url
+    repo_url = execute_git(cwd, ["config", "--get", "remote.origin.url"])
+
+    # get the repo relative path
+    repo_relative_path = execute_git(cwd, ["rev-parse", "--show-prefix"])
+
+    # get the commitId for the particular file
+    commit_id = execute_git(cwd, ["rev-list", "HEAD", "-1", "--", file_name])
+
+    # build the URL
+    base_url = url_to_https(repo_url)
+    if not base_url:
+        return None
+
+    return f"{base_url}/blob/{commit_id}/{repo_relative_path}{file_name}"
+
+
+def get_task_location(task):
+    try:
+        if hasattr(task, "file_path") and task.file_path:
+            return get_location(task.file_path)
+        else:
+            return get_location(task.dag.fileloc)
+    except Exception:
+        return None
+
+
+def execute_git(cwd, params):
+    p = subprocess.Popen(["git"] + params, cwd=cwd, stdout=subprocess.PIPE, stderr=None)
+    p.wait(timeout=0.5)
+    out, err = p.communicate()
+    return out.decode("utf8").strip()
+
+
+def redacted_connection_uri(conn: Connection, filtered_params=None, filtered_prefixes=None):
+    """
+    Return the connection URI for the given Connection.
+    This method additionally filters URI by removing query parameters that are known to carry sensitive data
+    like username, password, access key.
+    """
+    if filtered_prefixes is None:
+        filtered_prefixes = []
+    if filtered_params is None:
+        filtered_params = []
+
+    def filter_key_params(k: str):
+        return k not in filtered_params and any(substr in k for substr in filtered_prefixes)
+
+    conn_uri = conn.get_uri()
+    parsed = urlparse(conn_uri)
+
+    # Remove username and password
+    netloc = f"{parsed.hostname}" + (f":{parsed.port}" if parsed.port else "")
+    parsed = parsed._replace(netloc=netloc)
+    if parsed.query:
+        query_dict = dict(parse_qsl(parsed.query))
+        if conn.EXTRA_KEY in query_dict:
+            query_dict = json.loads(query_dict[conn.EXTRA_KEY])
+        filtered_qs = {k: v for k, v in query_dict.items() if not filter_key_params(k)}
+        parsed = parsed._replace(query=urlencode(filtered_qs))
+    return urlunparse(parsed)
+
+
+def get_normalized_postgres_connection_uri(conn):
+    """
+    URIs starting with postgresql:// and postgres:// are both valid
+    PostgreSQL connection strings. This function normalizes it to
+    postgres:// as canonical name according to OpenLineage spec.
+    """
+    uri = redacted_connection_uri(conn)
+    if uri.startswith("postgresql"):
+        uri = uri.replace("postgresql", "postgres", 1)
+    return uri
+
+
+def get_connection(conn_id) -> Connection | None:
+    from airflow.hooks.base import BaseHook
+
+    with suppress(Exception):

Review Comment:
   It's used in extractors (in followup PR https://github.com/apache/airflow/commit/2ba4ad57af4b0057ef73495515e886f40186125a#diff-27a950ed6d41d300271a5efb418010bc0e74284ca16dda079aab677357a66cffR169 - this won't be single PR, don't worry 🙂)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1156131429


##########
tests/providers/openlineage/conftest.py:
##########
@@ -0,0 +1,72 @@
+# 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.
+
+# Copyright 2018-2023 contributors to the OpenLineage project
+# SPDX-License-Identifier: Apache-2.0
+from __future__ import annotations
+
+import logging
+import os
+from unittest.mock import patch
+
+import pytest
+
+log = logging.getLogger(__name__)
+
+
+@pytest.fixture(scope="function")
+def remove_redshift_conn():
+    if "REDSHIFT_CONN" in os.environ:
+        del os.environ["REDSHIFT_CONN"]
+    if "WRITE_SCHEMA" in os.environ:
+        del os.environ["WRITE_SCHEMA"]
+
+
+@pytest.fixture(scope="function")
+def we_module_env():
+    os.environ["REDSHIFT_CONN"] = "postgresql://user:password@host.io:1234/db"
+    os.environ["WRITE_SCHEMA"] = "testing"
+
+
+@pytest.fixture(scope="function")
+def dagbag():
+    log.debug("dagbag()")
+    os.environ["AIRFLOW__CORE__SQL_ALCHEMY_CONN"] = "sqlite://"

Review Comment:
   Removed this too



##########
tests/providers/openlineage/conftest.py:
##########
@@ -0,0 +1,72 @@
+# 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.
+
+# Copyright 2018-2023 contributors to the OpenLineage project
+# SPDX-License-Identifier: Apache-2.0
+from __future__ import annotations
+
+import logging
+import os
+from unittest.mock import patch
+
+import pytest
+
+log = logging.getLogger(__name__)
+
+
+@pytest.fixture(scope="function")
+def remove_redshift_conn():
+    if "REDSHIFT_CONN" in os.environ:
+        del os.environ["REDSHIFT_CONN"]
+    if "WRITE_SCHEMA" in os.environ:
+        del os.environ["WRITE_SCHEMA"]
+
+
+@pytest.fixture(scope="function")
+def we_module_env():
+    os.environ["REDSHIFT_CONN"] = "postgresql://user:password@host.io:1234/db"
+    os.environ["WRITE_SCHEMA"] = "testing"
+
+
+@pytest.fixture(scope="function")
+def dagbag():
+    log.debug("dagbag()")
+    os.environ["AIRFLOW__CORE__SQL_ALCHEMY_CONN"] = "sqlite://"
+    os.environ["OPENLINEAGE_NAMESPACE"] = "test-marquez"
+
+    import airflow.utils.db as db_utils
+    from airflow import settings
+
+    db_utils.resetdb(settings.RBAC)

Review Comment:
   Removed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1156130939


##########
tests/providers/openlineage/conftest.py:
##########
@@ -0,0 +1,72 @@
+# 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.
+
+# Copyright 2018-2023 contributors to the OpenLineage project
+# SPDX-License-Identifier: Apache-2.0
+from __future__ import annotations
+
+import logging
+import os
+from unittest.mock import patch
+
+import pytest
+
+log = logging.getLogger(__name__)
+
+
+@pytest.fixture(scope="function")
+def remove_redshift_conn():
+    if "REDSHIFT_CONN" in os.environ:
+        del os.environ["REDSHIFT_CONN"]
+    if "WRITE_SCHEMA" in os.environ:
+        del os.environ["WRITE_SCHEMA"]
+
+
+@pytest.fixture(scope="function")
+def we_module_env():
+    os.environ["REDSHIFT_CONN"] = "postgresql://user:password@host.io:1234/db"
+    os.environ["WRITE_SCHEMA"] = "testing"

Review Comment:
   Removed this for now, will add when needed. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] mobuchowski commented on a diff in pull request #29940: Add base OpenLineage provider implementation

Posted by "mobuchowski (via GitHub)" <gi...@apache.org>.
mobuchowski commented on code in PR #29940:
URL: https://github.com/apache/airflow/pull/29940#discussion_r1157614375


##########
airflow/providers/openlineage/extractors/manager.py:
##########
@@ -0,0 +1,150 @@
+# 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 airflow.providers.openlineage.extractors import BaseExtractor, OperatorLineage
+from airflow.providers.openlineage.extractors.base import DefaultExtractor
+from airflow.providers.openlineage.plugins.facets import (
+    UnknownOperatorAttributeRunFacet,
+    UnknownOperatorInstance,
+)
+from airflow.providers.openlineage.utils.utils import get_operator_class
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class ExtractorManager(LoggingMixin):
+    """Class abstracting management of custom extractors."""
+
+    def __init__(self):
+        super().__init__()
+        self.extractors: dict[str, type[BaseExtractor]] = {}
+        self.default_extractor = DefaultExtractor
+
+        # Comma-separated extractors in OPENLINEAGE_EXTRACTORS variable.
+        # Extractors should implement BaseExtractor
+        from airflow.providers.openlineage.utils import import_from_string
+
+        # TODO: use airflow config with OL backup
+        env_extractors = os.getenv("OPENLINEAGE_EXTRACTORS")
+        if env_extractors is not None:
+            for extractor in env_extractors.split(";"):
+                extractor: type[BaseExtractor] = import_from_string(extractor.strip())
+                for operator_class in extractor.get_operator_classnames():
+                    self.extractors[operator_class] = extractor
+
+    def add_extractor(self, operator: str, extractor: type[BaseExtractor]):
+        self.extractors[operator] = extractor
+
+    def extract_metadata(self, dagrun, task, complete: bool = False, task_instance=None) -> OperatorLineage:
+        extractor = self._get_extractor(task)
+        task_info = (
+            f"task_type={task.task_type} "
+            f"airflow_dag_id={task.dag_id} "
+            f"task_id={task.task_id} "
+            f"airflow_run_id={dagrun.run_id} "
+        )
+
+        if extractor:
+            # Extracting advanced metadata is only possible when extractor for particular operator
+            # is defined. Without it, we can't extract any input or output data.
+            try:
+                self.log.debug("Using extractor %s %s", extractor.__class__.__name__, str(task_info))
+                if complete:
+                    task_metadata = extractor.extract_on_complete(task_instance)
+                else:
+                    task_metadata = extractor.extract()
+
+                self.log.debug("Found task metadata for operation %s: %s", task.task_id, str(task_metadata))
+                if task_metadata:
+                    if (not task_metadata.inputs) and (not task_metadata.outputs):
+                        inlets = task.get_inlet_defs()
+                        outlets = task.get_outlet_defs()
+                        self.extract_inlets_and_outlets(task_metadata, inlets, outlets)
+
+                    return task_metadata
+
+            except Exception as e:
+                self.log.exception(
+                    "Failed to extract metadata using found extractor %s - %s %s", extractor, e, task_info
+                )
+        else:
+            self.log.debug("Unable to find an extractor %s", task_info)
+
+            # Only include the unkonwnSourceAttribute facet if there is no extractor
+            task_metadata = OperatorLineage(
+                run_facets={
+                    "unknownSourceAttribute": UnknownOperatorAttributeRunFacet(
+                        unknownItems=[
+                            UnknownOperatorInstance(
+                                name=get_operator_class(task).__name__,
+                                properties={attr: value for attr, value in task.__dict__.items()},
+                            )
+                        ]
+                    )
+                },
+            )
+            inlets = task.get_inlet_defs()
+            outlets = task.get_outlet_defs()

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org