Skip to content

Commit

Permalink
First commit of OpenLineage provider. (#29940)
Browse files Browse the repository at this point in the history
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 and extracting them using DefaultExtractor.
  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.

Signed-off-by: Jakub Dardzinski <kuba0221@gmail.com>
  • Loading branch information
mobuchowski committed Apr 14, 2023
1 parent f01051a commit 55963de
Show file tree
Hide file tree
Showing 47 changed files with 2,806 additions and 284 deletions.
1 change: 1 addition & 0 deletions .github/ISSUE_TEMPLATE/airflow_providers_bug_report.yml
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,7 @@ body:
- neo4j
- odbc
- openfaas
- openlineage
- opsgenie
- oracle
- pagerduty
Expand Down
1 change: 1 addition & 0 deletions .github/workflows/ci.yml
Original file line number Diff line number Diff line change
Expand Up @@ -753,6 +753,7 @@ jobs:
run: |
rm -vf dist/apache_airflow-*.whl
# remove the provider packages that are not compatible with 2.3
rm -vf dist/apache_airflow_providers_openlineage*.whl
# rm -vf dist/apache_airflow_providers_docker*.whl
# pip download --no-deps --dest dist apache-airflow-providers-docker==3.1.0
if: matrix.package-format == 'wheel'
Expand Down
8 changes: 4 additions & 4 deletions CONTRIBUTING.rst
Original file line number Diff line number Diff line change
Expand Up @@ -618,10 +618,10 @@ devel_ci, devel_hadoop, dingding, discord, doc, doc_gen, docker, druid, elastics
facebook, ftp, gcp, gcp_api, github, github_enterprise, google, google_auth, grpc, hashicorp, hdfs,
hive, http, imap, influxdb, jdbc, jenkins, kerberos, kubernetes, ldap, leveldb, microsoft.azure,
microsoft.mssql, microsoft.psrp, microsoft.winrm, mongo, mssql, mysql, neo4j, odbc, openfaas,
opsgenie, oracle, otel, pagerduty, pandas, papermill, password, pinot, plexus, postgres, presto,
qds, qubole, rabbitmq, redis, s3, salesforce, samba, segment, sendgrid, sentry, sftp, singularity,
slack, smtp, snowflake, spark, sqlite, ssh, statsd, tableau, tabular, telegram, trino, vertica,
virtualenv, webhdfs, winrm, yandex, zendesk
openlineage, opsgenie, oracle, otel, pagerduty, pandas, papermill, password, pinot, plexus,
postgres, presto, qds, qubole, rabbitmq, redis, s3, salesforce, samba, segment, sendgrid, sentry,
sftp, singularity, slack, smtp, snowflake, spark, sqlite, ssh, statsd, tableau, tabular, telegram,
trino, vertica, virtualenv, webhdfs, winrm, yandex, zendesk
.. END EXTRAS HERE
Provider packages
Expand Down
8 changes: 4 additions & 4 deletions INSTALL
Original file line number Diff line number Diff line change
Expand Up @@ -103,10 +103,10 @@ devel_ci, devel_hadoop, dingding, discord, doc, doc_gen, docker, druid, elastics
facebook, ftp, gcp, gcp_api, github, github_enterprise, google, google_auth, grpc, hashicorp, hdfs,
hive, http, imap, influxdb, jdbc, jenkins, kerberos, kubernetes, ldap, leveldb, microsoft.azure,
microsoft.mssql, microsoft.psrp, microsoft.winrm, mongo, mssql, mysql, neo4j, odbc, openfaas,
opsgenie, oracle, otel, pagerduty, pandas, papermill, password, pinot, plexus, postgres, presto,
qds, qubole, rabbitmq, redis, s3, salesforce, samba, segment, sendgrid, sentry, sftp, singularity,
slack, smtp, snowflake, spark, sqlite, ssh, statsd, tableau, tabular, telegram, trino, vertica,
virtualenv, webhdfs, winrm, yandex, zendesk
openlineage, opsgenie, oracle, otel, pagerduty, pandas, papermill, password, pinot, plexus,
postgres, presto, qds, qubole, rabbitmq, redis, s3, salesforce, samba, segment, sendgrid, sentry,
sftp, singularity, slack, smtp, snowflake, spark, sqlite, ssh, statsd, tableau, tabular, telegram,
trino, vertica, virtualenv, webhdfs, winrm, yandex, zendesk
# END EXTRAS HERE

# For installing Airflow in development environments - see CONTRIBUTING.rst
Expand Down
2 changes: 1 addition & 1 deletion airflow/plugins_manager.py
Original file line number Diff line number Diff line change
Expand Up @@ -164,7 +164,7 @@ class AirflowPlugin:
# A list of timetable classes that can be used for DAG scheduling.
timetables: list[type[Timetable]] = []

listeners: list[ModuleType] = []
listeners: list[ModuleType | object] = []

@classmethod
def validate(cls):
Expand Down
25 changes: 25 additions & 0 deletions airflow/providers/openlineage/CHANGELOG.rst
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
.. 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.
Changelog
---------

1.0.0
.....

Initial version of the provider.
38 changes: 38 additions & 0 deletions airflow/providers/openlineage/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
# 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

__all__ = ["version"]

try:
import importlib_metadata as metadata
except ImportError:
from importlib import metadata # type: ignore[no-redef]

try:
version = metadata.version("apache-airflow-providers-openlineage")
except metadata.PackageNotFoundError:
import logging

log = logging.getLogger(__name__)
log.warning("Package metadata could not be found. Overriding it with version found in setup.py")
# TODO: What should be a proper fallback?
# If hardcoded version from provider version
# there's no point to use metadata above
version = "1.0.0.dev"

del metadata
23 changes: 23 additions & 0 deletions airflow/providers/openlineage/extractors/__init__.py
Original file line number Diff line number Diff line change
@@ -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"]
102 changes: 102 additions & 0 deletions airflow/providers/openlineage/extractors/base.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,102 @@
# 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.
There are operators which work very similarly and one extractor can cover both.
"""
raise NotImplementedError()

def validate(self):
assert self.operator.task_type in self.get_operator_classnames()

@abstractmethod
def extract(self) -> OperatorLineage | None:
pass

def extract_on_complete(self, task_instance) -> OperatorLineage | None:
return self.extract()


class DefaultExtractor(BaseExtractor):
"""Extractor that uses `get_openlineage_facets_on_start/complete/failure` methods."""

@classmethod
def get_operator_classnames(cls) -> list[str]:
"""
Default extractor is chosen not on the classname basis, but
by existence of get_openlineage_facets method on operator
"""
return []

def extract(self) -> OperatorLineage | None:
try:
return self._get_openlineage_facets(self.operator.get_openlineage_facets_on_start) # type: ignore
except AttributeError:
return None

def extract_on_complete(self, task_instance) -> OperatorLineage | None:
on_complete = getattr(self.operator, "get_openlineage_facets_on_complete", None)
if on_complete and callable(on_complete):
return self._get_openlineage_facets(on_complete, task_instance)
return self.extract()

def _get_openlineage_facets(self, get_facets_method, *args) -> OperatorLineage | None:
facets: OperatorLineage = get_facets_method(*args)
return OperatorLineage(
inputs=facets.inputs,
outputs=facets.outputs,
run_facets=facets.run_facets,
job_facets=facets.job_facets,
)
Loading

0 comments on commit 55963de

Please sign in to comment.