mirror of
https://github.com/open-metadata/OpenMetadata.git
synced 2025-11-09 15:32:25 +00:00
Fix airbyte pipeline lineage extraction (#21151)
This commit is contained in:
parent
f33cf42290
commit
4c0ce77756
@ -97,7 +97,7 @@ class ESMixin(Generic[T]):
|
|||||||
|
|
||||||
fqdn_search = (
|
fqdn_search = (
|
||||||
"/search/fieldQuery?fieldName={field_name}&fieldValue={field_value}&from={from_}"
|
"/search/fieldQuery?fieldName={field_name}&fieldValue={field_value}&from={from_}"
|
||||||
"&size={size}&index={index}"
|
"&size={size}&index={index}&deleted=false"
|
||||||
)
|
)
|
||||||
|
|
||||||
# sort_field needs to be unique for the pagination to work, so we can use the FQN
|
# sort_field needs to be unique for the pagination to work, so we can use the FQN
|
||||||
@ -125,14 +125,20 @@ class ESMixin(Generic[T]):
|
|||||||
if response:
|
if response:
|
||||||
if fields:
|
if fields:
|
||||||
fields = fields.split(",")
|
fields = fields.split(",")
|
||||||
return [
|
|
||||||
self.get_by_name(
|
entities = []
|
||||||
|
for hit in response["hits"]["hits"]:
|
||||||
|
entity = self.get_by_name(
|
||||||
entity=entity_type,
|
entity=entity_type,
|
||||||
fqn=hit["_source"]["fullyQualifiedName"],
|
fqn=hit["_source"]["fullyQualifiedName"],
|
||||||
fields=fields,
|
fields=fields,
|
||||||
)
|
)
|
||||||
for hit in response["hits"]["hits"]
|
if entity is None:
|
||||||
] or None
|
continue
|
||||||
|
|
||||||
|
entities.append(entity)
|
||||||
|
|
||||||
|
return entities or None
|
||||||
|
|
||||||
return None
|
return None
|
||||||
|
|
||||||
|
|||||||
@ -0,0 +1,29 @@
|
|||||||
|
# Copyright 2025 Collate
|
||||||
|
# Licensed under the Collate Community License, Version 1.0 (the "License");
|
||||||
|
# you may not use this file except in compliance with the License.
|
||||||
|
# You may obtain a copy of the License at
|
||||||
|
# https://github.com/open-metadata/OpenMetadata/blob/main/ingestion/LICENSE
|
||||||
|
# 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.
|
||||||
|
|
||||||
|
"""
|
||||||
|
Constants for Airbyte
|
||||||
|
"""
|
||||||
|
|
||||||
|
from enum import Enum
|
||||||
|
|
||||||
|
|
||||||
|
class AirbyteSource(Enum):
|
||||||
|
MYSQL = "MySQL"
|
||||||
|
POSTGRES = "Postgres"
|
||||||
|
MSSQL = "Microsoft SQL Server (MSSQL)"
|
||||||
|
MONGODB = "MongoDb"
|
||||||
|
|
||||||
|
|
||||||
|
class AirbyteDestination(Enum):
|
||||||
|
MYSQL = "MySQL"
|
||||||
|
POSTGRES = "Postgres"
|
||||||
|
MSSQL = "MS SQL Server"
|
||||||
@ -29,7 +29,6 @@ from metadata.generated.schema.entity.data.table import Table
|
|||||||
from metadata.generated.schema.entity.services.connections.pipeline.airbyteConnection import (
|
from metadata.generated.schema.entity.services.connections.pipeline.airbyteConnection import (
|
||||||
AirbyteConnection,
|
AirbyteConnection,
|
||||||
)
|
)
|
||||||
from metadata.generated.schema.entity.services.databaseService import DatabaseService
|
|
||||||
from metadata.generated.schema.metadataIngestion.workflow import (
|
from metadata.generated.schema.metadataIngestion.workflow import (
|
||||||
Source as WorkflowSource,
|
Source as WorkflowSource,
|
||||||
)
|
)
|
||||||
@ -46,12 +45,16 @@ from metadata.ingestion.api.models import Either
|
|||||||
from metadata.ingestion.api.steps import InvalidSourceException
|
from metadata.ingestion.api.steps import InvalidSourceException
|
||||||
from metadata.ingestion.models.pipeline_status import OMetaPipelineStatus
|
from metadata.ingestion.models.pipeline_status import OMetaPipelineStatus
|
||||||
from metadata.ingestion.ometa.ometa_api import OpenMetadata
|
from metadata.ingestion.ometa.ometa_api import OpenMetadata
|
||||||
|
from metadata.ingestion.source.pipeline.openlineage.models import TableDetails
|
||||||
|
from metadata.ingestion.source.pipeline.openlineage.utils import FQNNotFoundException
|
||||||
from metadata.ingestion.source.pipeline.pipeline_service import PipelineServiceSource
|
from metadata.ingestion.source.pipeline.pipeline_service import PipelineServiceSource
|
||||||
from metadata.utils import fqn
|
from metadata.utils import fqn
|
||||||
from metadata.utils.helpers import clean_uri
|
from metadata.utils.helpers import clean_uri
|
||||||
from metadata.utils.logger import ingestion_logger
|
from metadata.utils.logger import ingestion_logger
|
||||||
from metadata.utils.time_utils import convert_timestamp_to_milliseconds
|
from metadata.utils.time_utils import convert_timestamp_to_milliseconds
|
||||||
|
|
||||||
|
from .utils import get_destination_table_details, get_source_table_details
|
||||||
|
|
||||||
logger = ingestion_logger()
|
logger = ingestion_logger()
|
||||||
|
|
||||||
|
|
||||||
@ -189,6 +192,26 @@ class AirbyteSource(PipelineServiceSource):
|
|||||||
)
|
)
|
||||||
)
|
)
|
||||||
|
|
||||||
|
def _get_table_fqn(self, table_details: TableDetails) -> Optional[str]:
|
||||||
|
"""
|
||||||
|
Get the FQN of the table
|
||||||
|
"""
|
||||||
|
try:
|
||||||
|
if self.get_db_service_names():
|
||||||
|
return self._get_table_fqn_from_om(table_details)
|
||||||
|
|
||||||
|
return fqn.build(
|
||||||
|
metadata=self.metadata,
|
||||||
|
entity_type=Table,
|
||||||
|
service_name="*",
|
||||||
|
database_name=table_details.database,
|
||||||
|
schema_name=table_details.schema,
|
||||||
|
table_name=table_details.name,
|
||||||
|
)
|
||||||
|
except FQNNotFoundException:
|
||||||
|
return None
|
||||||
|
|
||||||
|
# pylint: disable=too-many-locals
|
||||||
def yield_pipeline_lineage_details(
|
def yield_pipeline_lineage_details(
|
||||||
self, pipeline_details: AirbytePipelineDetails
|
self, pipeline_details: AirbytePipelineDetails
|
||||||
) -> Iterable[Either[AddLineageRequest]]:
|
) -> Iterable[Either[AddLineageRequest]]:
|
||||||
@ -197,47 +220,65 @@ class AirbyteSource(PipelineServiceSource):
|
|||||||
:param pipeline_details: pipeline_details object from airbyte
|
:param pipeline_details: pipeline_details object from airbyte
|
||||||
:return: Lineage from inlets and outlets
|
:return: Lineage from inlets and outlets
|
||||||
"""
|
"""
|
||||||
|
pipeline_name = pipeline_details.connection.get("name")
|
||||||
source_connection = self.client.get_source(
|
source_connection = self.client.get_source(
|
||||||
pipeline_details.connection.get("sourceId")
|
pipeline_details.connection.get("sourceId")
|
||||||
)
|
)
|
||||||
destination_connection = self.client.get_destination(
|
destination_connection = self.client.get_destination(
|
||||||
pipeline_details.connection.get("destinationId")
|
pipeline_details.connection.get("destinationId")
|
||||||
)
|
)
|
||||||
source_service = self.metadata.get_by_name(
|
source_name = source_connection.get("sourceName")
|
||||||
entity=DatabaseService, fqn=source_connection.get("name")
|
destination_name = destination_connection.get("destinationName")
|
||||||
)
|
|
||||||
destination_service = self.metadata.get_by_name(
|
|
||||||
entity=DatabaseService, fqn=destination_connection.get("name")
|
|
||||||
)
|
|
||||||
if not source_service or not destination_service:
|
|
||||||
return
|
|
||||||
|
|
||||||
for task in (
|
for task in (
|
||||||
pipeline_details.connection.get("syncCatalog", {}).get("streams") or []
|
pipeline_details.connection.get("syncCatalog", {}).get("streams") or []
|
||||||
):
|
):
|
||||||
stream = task.get("stream")
|
stream = task.get("stream")
|
||||||
from_fqn = fqn.build(
|
|
||||||
self.metadata,
|
source_table_details = get_source_table_details(stream, source_connection)
|
||||||
Table,
|
destination_table_details = get_destination_table_details(
|
||||||
table_name=stream.get("name"),
|
stream, destination_connection
|
||||||
database_name=None,
|
|
||||||
schema_name=stream.get("namespace"),
|
|
||||||
service_name=source_connection.get("name"),
|
|
||||||
)
|
)
|
||||||
|
|
||||||
to_fqn = fqn.build(
|
if not source_table_details or not destination_table_details:
|
||||||
self.metadata,
|
continue
|
||||||
Table,
|
|
||||||
table_name=stream.get("name"),
|
from_fqn = self._get_table_fqn(source_table_details)
|
||||||
database_name=None,
|
to_fqn = self._get_table_fqn(destination_table_details)
|
||||||
schema_name=stream.get("namespace"),
|
|
||||||
service_name=destination_connection.get("name"),
|
if not from_fqn:
|
||||||
|
logger.warning(
|
||||||
|
f"While extracting lineage: [{pipeline_name}],"
|
||||||
|
f" source table: [{source_table_details.database or '*'}]"
|
||||||
|
f".[{source_table_details.schema}].[{source_table_details.name}]"
|
||||||
|
f" (type: {source_name}) not found in openmetadata"
|
||||||
)
|
)
|
||||||
|
continue
|
||||||
|
if not to_fqn:
|
||||||
|
logger.warning(
|
||||||
|
f"While extracting lineage: [{pipeline_name}],"
|
||||||
|
f" destination table: [{destination_table_details.database or '*'}]"
|
||||||
|
f".[{destination_table_details.schema}].[{destination_table_details.name}]"
|
||||||
|
f" (type: {destination_name}) not found in openmetadata"
|
||||||
|
)
|
||||||
|
continue
|
||||||
|
|
||||||
from_entity = self.metadata.get_by_name(entity=Table, fqn=from_fqn)
|
from_entity = self.metadata.get_by_name(entity=Table, fqn=from_fqn)
|
||||||
to_entity = self.metadata.get_by_name(entity=Table, fqn=to_fqn)
|
to_entity = self.metadata.get_by_name(entity=Table, fqn=to_fqn)
|
||||||
|
|
||||||
if not from_entity and not to_entity:
|
if not from_entity:
|
||||||
|
logger.warning(
|
||||||
|
f"While extracting lineage: [{pipeline_name}],"
|
||||||
|
f" source table (fqn: [{from_fqn}], type: {source_name}) not found"
|
||||||
|
" in openmetadata"
|
||||||
|
)
|
||||||
|
continue
|
||||||
|
if not to_entity:
|
||||||
|
logger.warning(
|
||||||
|
f"While extracting lineage: [{pipeline_name}],"
|
||||||
|
f" destination table (fqn: [{to_fqn}], type: {destination_name}) not found"
|
||||||
|
" in openmetadata"
|
||||||
|
)
|
||||||
continue
|
continue
|
||||||
|
|
||||||
pipeline_fqn = fqn.build(
|
pipeline_fqn = fqn.build(
|
||||||
@ -279,4 +320,4 @@ class AirbyteSource(PipelineServiceSource):
|
|||||||
"""
|
"""
|
||||||
Get Pipeline Name
|
Get Pipeline Name
|
||||||
"""
|
"""
|
||||||
return pipeline_details.connection.get("connectionId")
|
return pipeline_details.connection.get("name")
|
||||||
|
|||||||
@ -0,0 +1,99 @@
|
|||||||
|
# Copyright 2025 Collate
|
||||||
|
# Licensed under the Collate Community License, Version 1.0 (the "License");
|
||||||
|
# you may not use this file except in compliance with the License.
|
||||||
|
# You may obtain a copy of the License at
|
||||||
|
# https://github.com/open-metadata/OpenMetadata/blob/main/ingestion/LICENSE
|
||||||
|
# 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.
|
||||||
|
|
||||||
|
"""
|
||||||
|
Utils for Airbyte
|
||||||
|
"""
|
||||||
|
|
||||||
|
from metadata.ingestion.source.pipeline.openlineage.models import TableDetails
|
||||||
|
from metadata.utils.logger import ingestion_logger
|
||||||
|
|
||||||
|
from .constants import AirbyteDestination, AirbyteSource
|
||||||
|
|
||||||
|
logger = ingestion_logger()
|
||||||
|
|
||||||
|
|
||||||
|
def get_source_table_details(stream: dict, source_connection: dict) -> TableDetails:
|
||||||
|
"""
|
||||||
|
Get the source table details
|
||||||
|
"""
|
||||||
|
source_name = source_connection.get("sourceName")
|
||||||
|
source_database = source_connection.get("connectionConfiguration", {}).get(
|
||||||
|
"database"
|
||||||
|
)
|
||||||
|
source_schema = stream.get("namespace")
|
||||||
|
|
||||||
|
# Check if source is supported
|
||||||
|
if source_name not in [
|
||||||
|
AirbyteSource.POSTGRES.value,
|
||||||
|
AirbyteSource.MSSQL.value,
|
||||||
|
AirbyteSource.MYSQL.value,
|
||||||
|
AirbyteSource.MONGODB.value,
|
||||||
|
]:
|
||||||
|
logger.warning(
|
||||||
|
f"Lineage of airbyte pipeline with source [{source_name}] is not supported yet"
|
||||||
|
)
|
||||||
|
return None
|
||||||
|
|
||||||
|
# Handle specific database configurations
|
||||||
|
if source_name == AirbyteSource.MYSQL.value:
|
||||||
|
source_schema = source_database
|
||||||
|
source_database = None
|
||||||
|
elif source_name == AirbyteSource.MONGODB.value:
|
||||||
|
source_schema = (
|
||||||
|
source_connection.get("connectionConfiguration", {})
|
||||||
|
.get("database_config", {})
|
||||||
|
.get("database")
|
||||||
|
)
|
||||||
|
source_database = None
|
||||||
|
|
||||||
|
return TableDetails(
|
||||||
|
name=stream["name"],
|
||||||
|
schema=source_schema,
|
||||||
|
database=source_database,
|
||||||
|
)
|
||||||
|
|
||||||
|
|
||||||
|
def get_destination_table_details(
|
||||||
|
stream: dict, destination_connection: dict
|
||||||
|
) -> TableDetails:
|
||||||
|
"""
|
||||||
|
Get the destination table details
|
||||||
|
"""
|
||||||
|
destination_name = destination_connection.get("destinationName")
|
||||||
|
destination_database = destination_connection.get(
|
||||||
|
"connectionConfiguration", {}
|
||||||
|
).get("database")
|
||||||
|
destination_schema = destination_connection.get("connectionConfiguration", {}).get(
|
||||||
|
"schema"
|
||||||
|
)
|
||||||
|
|
||||||
|
# Check if destination is supported
|
||||||
|
if destination_name not in [
|
||||||
|
AirbyteDestination.POSTGRES.value,
|
||||||
|
AirbyteDestination.MSSQL.value,
|
||||||
|
AirbyteDestination.MYSQL.value,
|
||||||
|
]:
|
||||||
|
logger.warning(
|
||||||
|
f"Lineage of airbyte pipeline with destination [{destination_name}] is not supported yet"
|
||||||
|
)
|
||||||
|
return None
|
||||||
|
|
||||||
|
# Handle specific database configurations
|
||||||
|
if destination_name == AirbyteDestination.MYSQL.value:
|
||||||
|
destination_schema = destination_database
|
||||||
|
destination_database = None
|
||||||
|
|
||||||
|
return TableDetails(
|
||||||
|
name=stream["name"],
|
||||||
|
schema=destination_schema,
|
||||||
|
database=destination_database,
|
||||||
|
)
|
||||||
@ -14,7 +14,7 @@ Openlineage Source Model module
|
|||||||
|
|
||||||
from dataclasses import dataclass
|
from dataclasses import dataclass
|
||||||
from enum import Enum
|
from enum import Enum
|
||||||
from typing import Any, Dict, List
|
from typing import Any, Dict, List, Optional
|
||||||
|
|
||||||
|
|
||||||
@dataclass
|
@dataclass
|
||||||
@ -76,8 +76,9 @@ class TableDetails:
|
|||||||
Minimal table information.
|
Minimal table information.
|
||||||
"""
|
"""
|
||||||
|
|
||||||
schema: str
|
|
||||||
name: str
|
name: str
|
||||||
|
schema: str
|
||||||
|
database: Optional[str] = None
|
||||||
|
|
||||||
|
|
||||||
class EventType(str, Enum):
|
class EventType(str, Enum):
|
||||||
|
|||||||
@ -319,8 +319,7 @@ class PipelineServiceSource(TopologyRunnerMixin, Source, ABC):
|
|||||||
def _get_table_fqn_from_om(self, table_details: TableDetails) -> Optional[str]:
|
def _get_table_fqn_from_om(self, table_details: TableDetails) -> Optional[str]:
|
||||||
"""
|
"""
|
||||||
Based on partial schema and table names look for matching table object in open metadata.
|
Based on partial schema and table names look for matching table object in open metadata.
|
||||||
:param schema: schema name
|
:param table_details: TableDetails object containing table name, schema, database information
|
||||||
:param table: table name
|
|
||||||
:return: fully qualified name of a Table in Open Metadata
|
:return: fully qualified name of a Table in Open Metadata
|
||||||
"""
|
"""
|
||||||
result = None
|
result = None
|
||||||
@ -330,7 +329,7 @@ class PipelineServiceSource(TopologyRunnerMixin, Source, ABC):
|
|||||||
metadata=self.metadata,
|
metadata=self.metadata,
|
||||||
entity_type=Table,
|
entity_type=Table,
|
||||||
service_name=db_service,
|
service_name=db_service,
|
||||||
database_name=None,
|
database_name=table_details.database,
|
||||||
schema_name=table_details.schema,
|
schema_name=table_details.schema,
|
||||||
table_name=table_details.name,
|
table_name=table_details.name,
|
||||||
)
|
)
|
||||||
|
|||||||
@ -151,7 +151,7 @@ def build(
|
|||||||
def _(
|
def _(
|
||||||
metadata: Optional[OpenMetadata],
|
metadata: Optional[OpenMetadata],
|
||||||
*,
|
*,
|
||||||
service_name: str,
|
service_name: Optional[str],
|
||||||
database_name: Optional[str],
|
database_name: Optional[str],
|
||||||
schema_name: Optional[str],
|
schema_name: Optional[str],
|
||||||
table_name: str,
|
table_name: str,
|
||||||
@ -161,7 +161,7 @@ def _(
|
|||||||
"""
|
"""
|
||||||
Building logic for tables
|
Building logic for tables
|
||||||
:param metadata: OMeta client
|
:param metadata: OMeta client
|
||||||
:param service_name: Service Name to filter
|
:param service_name: Service Name to filter or None
|
||||||
:param database_name: DB name or None
|
:param database_name: DB name or None
|
||||||
:param schema_name: Schema name or None
|
:param schema_name: Schema name or None
|
||||||
:param table_name: Table name
|
:param table_name: Table name
|
||||||
@ -180,8 +180,8 @@ def _(
|
|||||||
service_name=service_name,
|
service_name=service_name,
|
||||||
)
|
)
|
||||||
|
|
||||||
# if entity not found in ES proceed to build FQN with database_name and schema_name
|
# if entity not found in ES proceed to build FQN with service_name, database_name and schema_name
|
||||||
if not entity and database_name and schema_name:
|
if not entity and service_name and database_name and schema_name:
|
||||||
fqn = _build(service_name, database_name, schema_name, table_name)
|
fqn = _build(service_name, database_name, schema_name, table_name)
|
||||||
return [fqn] if fetch_multiple_entities else fqn
|
return [fqn] if fetch_multiple_entities else fqn
|
||||||
if entity and fetch_multiple_entities:
|
if entity and fetch_multiple_entities:
|
||||||
|
|||||||
@ -18,6 +18,7 @@ from unittest import TestCase
|
|||||||
from unittest.mock import patch
|
from unittest.mock import patch
|
||||||
|
|
||||||
from metadata.generated.schema.api.data.createPipeline import CreatePipelineRequest
|
from metadata.generated.schema.api.data.createPipeline import CreatePipelineRequest
|
||||||
|
from metadata.generated.schema.api.lineage.addLineage import AddLineageRequest
|
||||||
from metadata.generated.schema.entity.data.pipeline import (
|
from metadata.generated.schema.entity.data.pipeline import (
|
||||||
Pipeline,
|
Pipeline,
|
||||||
PipelineStatus,
|
PipelineStatus,
|
||||||
@ -25,6 +26,7 @@ from metadata.generated.schema.entity.data.pipeline import (
|
|||||||
Task,
|
Task,
|
||||||
TaskStatus,
|
TaskStatus,
|
||||||
)
|
)
|
||||||
|
from metadata.generated.schema.entity.data.table import Table
|
||||||
from metadata.generated.schema.entity.services.pipelineService import (
|
from metadata.generated.schema.entity.services.pipelineService import (
|
||||||
PipelineConnection,
|
PipelineConnection,
|
||||||
PipelineService,
|
PipelineService,
|
||||||
@ -34,6 +36,8 @@ from metadata.generated.schema.metadataIngestion.workflow import (
|
|||||||
OpenMetadataWorkflowConfig,
|
OpenMetadataWorkflowConfig,
|
||||||
)
|
)
|
||||||
from metadata.generated.schema.type.basic import FullyQualifiedEntityName
|
from metadata.generated.schema.type.basic import FullyQualifiedEntityName
|
||||||
|
from metadata.generated.schema.type.entityLineage import EntitiesEdge, LineageDetails
|
||||||
|
from metadata.generated.schema.type.entityLineage import Source as LineageSource
|
||||||
from metadata.generated.schema.type.entityReference import EntityReference
|
from metadata.generated.schema.type.entityReference import EntityReference
|
||||||
from metadata.ingestion.models.pipeline_status import OMetaPipelineStatus
|
from metadata.ingestion.models.pipeline_status import OMetaPipelineStatus
|
||||||
from metadata.ingestion.source.pipeline.airbyte.metadata import (
|
from metadata.ingestion.source.pipeline.airbyte.metadata import (
|
||||||
@ -70,7 +74,7 @@ mock_airbyte_config = {
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
EXPECTED_ARIBYTE_DETAILS = AirbytePipelineDetails(
|
EXPECTED_AIRBYTE_DETAILS = AirbytePipelineDetails(
|
||||||
workspace=mock_data["workspace"][0], connection=mock_data["connection"][0]
|
workspace=mock_data["workspace"][0], connection=mock_data["connection"][0]
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -156,8 +160,79 @@ MOCK_PIPELINE = Pipeline(
|
|||||||
),
|
),
|
||||||
)
|
)
|
||||||
|
|
||||||
|
# Mock data for lineage testing
|
||||||
|
MOCK_POSTGRES_SOURCE_TABLE = Table(
|
||||||
|
id="69fc8906-4a4a-45ab-9a54-9cc2d399e10e",
|
||||||
|
name="mock_table_name",
|
||||||
|
fullyQualifiedName="mock_source_service.mock_source_db.mock_source_schema.mock_table_name",
|
||||||
|
columns=[{"name": "id", "dataType": "INT"}, {"name": "name", "dataType": "STRING"}],
|
||||||
|
)
|
||||||
|
|
||||||
|
MOCK_POSTGRES_DESTINATION_TABLE = Table(
|
||||||
|
id="59fc8906-4a4a-45ab-9a54-9cc2d399e10e",
|
||||||
|
name="mock_table_name",
|
||||||
|
fullyQualifiedName=(
|
||||||
|
"mock_destination_service.mock_destination_db"
|
||||||
|
".mock_destination_schema.mock_table_name"
|
||||||
|
),
|
||||||
|
columns=[{"name": "id", "dataType": "INT"}, {"name": "name", "dataType": "STRING"}],
|
||||||
|
)
|
||||||
|
|
||||||
|
EXPECTED_LINEAGE = AddLineageRequest(
|
||||||
|
edge=EntitiesEdge(
|
||||||
|
fromEntity=EntityReference(
|
||||||
|
id="69fc8906-4a4a-45ab-9a54-9cc2d399e10e", type="table"
|
||||||
|
),
|
||||||
|
toEntity=EntityReference(
|
||||||
|
id="59fc8906-4a4a-45ab-9a54-9cc2d399e10e", type="table"
|
||||||
|
),
|
||||||
|
lineageDetails=LineageDetails(
|
||||||
|
pipeline=EntityReference(
|
||||||
|
id="2aaa012e-099a-11ed-861d-0242ac120002", type="pipeline"
|
||||||
|
),
|
||||||
|
source=LineageSource.PipelineLineage,
|
||||||
|
),
|
||||||
|
)
|
||||||
|
)
|
||||||
|
|
||||||
|
MOCK_SOURCE_TABLE_FQN = (
|
||||||
|
"mock_source_service.mock_source_db.mock_source_schema.mock_table_name"
|
||||||
|
)
|
||||||
|
MOCK_DESTINATION_TABLE_FQN = "mock_destination_service.mock_destination_db.mock_destination_schema.mock_table_name"
|
||||||
|
|
||||||
|
|
||||||
|
# Configure mock for _get_table_fqn to return FQNs for source and destination tables
|
||||||
|
def mock_get_table_fqn(self, table_details): # pylint: disable=unused-argument
|
||||||
|
if table_details.name != "mock_table_name":
|
||||||
|
return None
|
||||||
|
|
||||||
|
if table_details.schema == "mock_source_schema":
|
||||||
|
return MOCK_SOURCE_TABLE_FQN
|
||||||
|
if table_details.schema == "mock_destination_schema":
|
||||||
|
return MOCK_DESTINATION_TABLE_FQN
|
||||||
|
if table_details.schema == "mock_source_db":
|
||||||
|
return MOCK_SOURCE_TABLE_FQN
|
||||||
|
if table_details.schema == "mock_destination_db":
|
||||||
|
return MOCK_DESTINATION_TABLE_FQN
|
||||||
|
|
||||||
|
return None
|
||||||
|
|
||||||
|
|
||||||
|
# Configure the mock to return our test tables and pipeline
|
||||||
|
def mock_get_by_name(entity, fqn):
|
||||||
|
if entity == Table:
|
||||||
|
if fqn == MOCK_SOURCE_TABLE_FQN:
|
||||||
|
return MOCK_POSTGRES_SOURCE_TABLE
|
||||||
|
if fqn == MOCK_DESTINATION_TABLE_FQN:
|
||||||
|
return MOCK_POSTGRES_DESTINATION_TABLE
|
||||||
|
if entity == Pipeline:
|
||||||
|
return MOCK_PIPELINE
|
||||||
|
return None
|
||||||
|
|
||||||
|
|
||||||
class AirbyteUnitTest(TestCase):
|
class AirbyteUnitTest(TestCase):
|
||||||
|
"""Test class for Airbyte source module."""
|
||||||
|
|
||||||
@patch(
|
@patch(
|
||||||
"metadata.ingestion.source.pipeline.pipeline_service.PipelineServiceSource.test_connection"
|
"metadata.ingestion.source.pipeline.pipeline_service.PipelineServiceSource.test_connection"
|
||||||
)
|
)
|
||||||
@ -180,20 +255,86 @@ class AirbyteUnitTest(TestCase):
|
|||||||
self.client.list_connections.return_value = mock_data.get("connection")
|
self.client.list_connections.return_value = mock_data.get("connection")
|
||||||
|
|
||||||
def test_pipeline_list(self):
|
def test_pipeline_list(self):
|
||||||
assert list(self.airbyte.get_pipelines_list())[0] == EXPECTED_ARIBYTE_DETAILS
|
assert list(self.airbyte.get_pipelines_list())[0] == EXPECTED_AIRBYTE_DETAILS
|
||||||
|
|
||||||
def test_pipeline_name(self):
|
def test_pipeline_name(self):
|
||||||
assert self.airbyte.get_pipeline_name(
|
assert self.airbyte.get_pipeline_name(
|
||||||
EXPECTED_ARIBYTE_DETAILS
|
EXPECTED_AIRBYTE_DETAILS
|
||||||
) == mock_data.get("connection")[0].get("connectionId")
|
) == mock_data.get("connection")[0].get("name")
|
||||||
|
|
||||||
def test_pipelines(self):
|
def test_pipelines(self):
|
||||||
pipline = list(self.airbyte.yield_pipeline(EXPECTED_ARIBYTE_DETAILS))[0].right
|
pipeline = list(self.airbyte.yield_pipeline(EXPECTED_AIRBYTE_DETAILS))[0].right
|
||||||
assert pipline == EXPECTED_CREATED_PIPELINES
|
assert pipeline == EXPECTED_CREATED_PIPELINES
|
||||||
|
|
||||||
def test_pipeline_status(self):
|
def test_pipeline_status(self):
|
||||||
status = [
|
status = [
|
||||||
either.right
|
either.right
|
||||||
for either in self.airbyte.yield_pipeline_status(EXPECTED_ARIBYTE_DETAILS)
|
for either in self.airbyte.yield_pipeline_status(EXPECTED_AIRBYTE_DETAILS)
|
||||||
]
|
]
|
||||||
assert status == EXPECTED_PIPELINE_STATUS
|
assert status == EXPECTED_PIPELINE_STATUS
|
||||||
|
|
||||||
|
@patch.object(AirbyteSource, "_get_table_fqn", mock_get_table_fqn)
|
||||||
|
def test_yield_pipeline_lineage_details(self):
|
||||||
|
"""Test the Airbyte lineage generation functionality."""
|
||||||
|
# Mock the client methods needed for lineage with supported source and destination types
|
||||||
|
self.client.get_source.return_value = {
|
||||||
|
"sourceName": "Postgres",
|
||||||
|
"connectionConfiguration": {
|
||||||
|
"database": "mock_source_db",
|
||||||
|
"schema": "mock_source_schema",
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
self.client.get_destination.return_value = {
|
||||||
|
"destinationName": "Postgres",
|
||||||
|
"connectionConfiguration": {
|
||||||
|
"database": "mock_destination_db",
|
||||||
|
"schema": "mock_destination_schema",
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
# Mock connection with stream data for lineage test
|
||||||
|
test_connection = {
|
||||||
|
"connectionId": "test-connection-id",
|
||||||
|
"sourceId": "test-source-id",
|
||||||
|
"destinationId": "test-destination-id",
|
||||||
|
"name": "Test Connection",
|
||||||
|
"syncCatalog": {
|
||||||
|
"streams": [
|
||||||
|
{
|
||||||
|
"stream": {
|
||||||
|
"name": "mock_table_name",
|
||||||
|
"namespace": "mock_source_schema",
|
||||||
|
"jsonSchema": {},
|
||||||
|
}
|
||||||
|
}
|
||||||
|
]
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
test_workspace = {"workspaceId": "test-workspace-id"}
|
||||||
|
test_pipeline_details = AirbytePipelineDetails(
|
||||||
|
workspace=test_workspace, connection=test_connection
|
||||||
|
)
|
||||||
|
|
||||||
|
# Mock the metadata object directly in the Airbyte source
|
||||||
|
with patch.object(self.airbyte, "metadata") as mock_metadata:
|
||||||
|
mock_metadata.get_by_name.side_effect = mock_get_by_name
|
||||||
|
|
||||||
|
# Test yield_pipeline_lineage_details
|
||||||
|
lineage_results = list(
|
||||||
|
self.airbyte.yield_pipeline_lineage_details(test_pipeline_details)
|
||||||
|
)
|
||||||
|
|
||||||
|
# Check that we get at least one lineage result
|
||||||
|
assert len(lineage_results) > 0
|
||||||
|
|
||||||
|
# Extract the lineage details
|
||||||
|
lineage = lineage_results[0].right
|
||||||
|
|
||||||
|
# Verify the lineage structure
|
||||||
|
assert lineage.edge.fromEntity.id == MOCK_POSTGRES_SOURCE_TABLE.id
|
||||||
|
assert lineage.edge.toEntity.id == MOCK_POSTGRES_DESTINATION_TABLE.id
|
||||||
|
# Compare just the UUID string value from both sides
|
||||||
|
assert lineage.edge.lineageDetails.pipeline.id.root == MOCK_PIPELINE.id.root
|
||||||
|
assert lineage.edge.lineageDetails.source == LineageSource.PipelineLineage
|
||||||
|
|||||||
@ -409,10 +409,14 @@ public class SearchResource {
|
|||||||
@Parameter(description = "Search Index name, defaults to table_search_index")
|
@Parameter(description = "Search Index name, defaults to table_search_index")
|
||||||
@DefaultValue("table_search_index")
|
@DefaultValue("table_search_index")
|
||||||
@QueryParam("index")
|
@QueryParam("index")
|
||||||
String index)
|
String index,
|
||||||
|
@Parameter(description = "Filter documents by deleted param. By default deleted is false")
|
||||||
|
@DefaultValue("false")
|
||||||
|
@QueryParam("deleted")
|
||||||
|
boolean deleted)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
|
||||||
return searchRepository.searchByField(fieldName, fieldValue, index);
|
return searchRepository.searchByField(fieldName, fieldValue, index, deleted);
|
||||||
}
|
}
|
||||||
|
|
||||||
@GET
|
@GET
|
||||||
|
|||||||
@ -257,7 +257,8 @@ public interface SearchClient {
|
|||||||
Response.Status.NOT_IMPLEMENTED, NOT_IMPLEMENTED_ERROR_TYPE, NOT_IMPLEMENTED_METHOD);
|
Response.Status.NOT_IMPLEMENTED, NOT_IMPLEMENTED_ERROR_TYPE, NOT_IMPLEMENTED_METHOD);
|
||||||
}
|
}
|
||||||
|
|
||||||
Response searchByField(String fieldName, String fieldValue, String index) throws IOException;
|
Response searchByField(String fieldName, String fieldValue, String index, Boolean deleted)
|
||||||
|
throws IOException;
|
||||||
|
|
||||||
Response aggregate(AggregationRequest request) throws IOException;
|
Response aggregate(AggregationRequest request) throws IOException;
|
||||||
|
|
||||||
|
|||||||
@ -1160,9 +1160,9 @@ public class SearchRepository {
|
|||||||
.withIsConnectedVia(isConnectedVia(entityType)));
|
.withIsConnectedVia(isConnectedVia(entityType)));
|
||||||
}
|
}
|
||||||
|
|
||||||
public Response searchByField(String fieldName, String fieldValue, String index)
|
public Response searchByField(String fieldName, String fieldValue, String index, Boolean deleted)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return searchClient.searchByField(fieldName, fieldValue, index);
|
return searchClient.searchByField(fieldName, fieldValue, index, deleted);
|
||||||
}
|
}
|
||||||
|
|
||||||
public Response aggregate(AggregationRequest request) throws IOException {
|
public Response aggregate(AggregationRequest request) throws IOException {
|
||||||
|
|||||||
@ -1310,13 +1310,17 @@ public class ElasticSearchClient implements SearchClient {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Response searchByField(String fieldName, String fieldValue, String index)
|
public Response searchByField(String fieldName, String fieldValue, String index, Boolean deleted)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
es.org.elasticsearch.action.search.SearchRequest searchRequest =
|
es.org.elasticsearch.action.search.SearchRequest searchRequest =
|
||||||
new es.org.elasticsearch.action.search.SearchRequest(
|
new es.org.elasticsearch.action.search.SearchRequest(
|
||||||
Entity.getSearchRepository().getIndexOrAliasName(index));
|
Entity.getSearchRepository().getIndexOrAliasName(index));
|
||||||
SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder();
|
SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder();
|
||||||
searchSourceBuilder.query(QueryBuilders.wildcardQuery(fieldName, fieldValue));
|
BoolQueryBuilder query =
|
||||||
|
QueryBuilders.boolQuery()
|
||||||
|
.must(QueryBuilders.wildcardQuery(fieldName, fieldValue))
|
||||||
|
.filter(QueryBuilders.termQuery("deleted", deleted));
|
||||||
|
searchSourceBuilder.query(query);
|
||||||
searchRequest.source(searchSourceBuilder);
|
searchRequest.source(searchSourceBuilder);
|
||||||
String response = client.search(searchRequest, RequestOptions.DEFAULT).toString();
|
String response = client.search(searchRequest, RequestOptions.DEFAULT).toString();
|
||||||
return Response.status(OK).entity(response).build();
|
return Response.status(OK).entity(response).build();
|
||||||
|
|||||||
@ -1381,13 +1381,17 @@ public class OpenSearchClient implements SearchClient {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Response searchByField(String fieldName, String fieldValue, String index)
|
public Response searchByField(String fieldName, String fieldValue, String index, Boolean deleted)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
os.org.opensearch.action.search.SearchRequest searchRequest =
|
os.org.opensearch.action.search.SearchRequest searchRequest =
|
||||||
new os.org.opensearch.action.search.SearchRequest(
|
new os.org.opensearch.action.search.SearchRequest(
|
||||||
Entity.getSearchRepository().getIndexOrAliasName(index));
|
Entity.getSearchRepository().getIndexOrAliasName(index));
|
||||||
SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder();
|
SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder();
|
||||||
searchSourceBuilder.query(QueryBuilders.wildcardQuery(fieldName, fieldValue));
|
BoolQueryBuilder query =
|
||||||
|
QueryBuilders.boolQuery()
|
||||||
|
.must(QueryBuilders.wildcardQuery(fieldName, fieldValue))
|
||||||
|
.filter(QueryBuilders.termQuery("deleted", deleted));
|
||||||
|
searchSourceBuilder.query(query);
|
||||||
searchRequest.source(searchSourceBuilder);
|
searchRequest.source(searchSourceBuilder);
|
||||||
String response = client.search(searchRequest, RequestOptions.DEFAULT).toString();
|
String response = client.search(searchRequest, RequestOptions.DEFAULT).toString();
|
||||||
return Response.status(OK).entity(response).build();
|
return Response.status(OK).entity(response).build();
|
||||||
|
|||||||
Loading…
x
Reference in New Issue
Block a user