Fix #1604: Add Spline Pipeline Connector (#11562)

* Fix #1604: Add Spline Connector

* Add tests & grammer validation

* Spline UI Changes & Docs

* fix pipeline workflow doc

* chore: use common field for dbService name

* chore: use const for beta services

* chore: add service icon

* Update ingestion/src/metadata/ingestion/source/pipeline/spline/metadata.py

Co-authored-by: Onkar Ravgan <onkar.10r@gmail.com>

---------

Co-authored-by: Sachin Chaurasiya <sachinchaurasiyachotey87@gmail.com>
Co-authored-by: Sriharsha Chintalapani <harshach@users.noreply.github.com>
Co-authored-by: Onkar Ravgan <onkar.10r@gmail.com>
This commit is contained in:
Mayur Singal 2023-05-19 14:46:32 +05:30 committed by GitHub
parent fde1139764
commit e9992a52a8
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
33 changed files with 2013 additions and 29 deletions

View File

@ -0,0 +1,29 @@
source:
type: spline
serviceName: local_spline
serviceConnection:
config:
type: Spline
hostPort: http://localhost:8081/
uiHostPort: http://localhost:9090
sourceConfig:
config:
type: PipelineMetadata
dbServiceNames:
- local_databricks
- local_postgres_empty1
pipelineFilterPattern:
includes:
- .*jdbc.*
- .*databricks.*
sink:
type: metadata-rest
config: {}
workflowConfig:
loggerLevel: DEBUG
openMetadataServerConfig:
hostPort: http://localhost:8585/api
authProvider: openmetadata
securityConfig:
"jwtToken": "eyJraWQiOiJHYjM4OWEtOWY3Ni1nZGpzLWE5MmotMDI0MmJrOTQzNTYiLCJ0eXAiOiJKV1QiLCJhbGciOiJSUzI1NiJ9.eyJzdWIiOiJhZG1pbiIsImlzQm90IjpmYWxzZSwiaXNzIjoib3Blbi1tZXRhZGF0YS5vcmciLCJpYXQiOjE2NjM5Mzg0NjIsImVtYWlsIjoiYWRtaW5Ab3Blbm1ldGFkYXRhLm9yZyJ9.tS8um_5DKu7HgzGBzS1VTA5uUjKWOCU0B_j08WXBiEC0mr0zNREkqVfwFDD-d24HlNEbrqioLsBuFRiwIWKc1m_ZlVQbG7P36RUxhuv2vbSp80FKyNM-Tj93FDzq91jsyNmsQhyNv_fNr3TXfzzSPjHt8Go0FMMP66weoKMgW2PbXlhVKwEuXUHyakLLzewm9UMeQaEiRzhiTMU3UkLXcKbYEJJvfNFcLwSl9W8JCO_l0Yj3ud-qt_nQYEZwqW6u5nfdQllN133iikV4fM5QZsMCnm8Rq1mvLR0y9bmJiD7fwM1tmJ791TUWqmKaTnP49U493VanKpUAfzIiOiIbhg"

View File

@ -0,0 +1,94 @@
# Copyright 2021 Collate
# Licensed 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.
"""
Client to interact with Spline consumer apis
"""
import traceback
from typing import List
from metadata.generated.schema.entity.services.connections.pipeline.splineConnection import (
SplineConnection,
)
from metadata.ingestion.ometa.client import REST, APIError, ClientConfig
from metadata.ingestion.source.pipeline.spline.models import (
ExecutionDetail,
ExecutionEvents,
)
from metadata.utils.constants import AUTHORIZATION_HEADER, NO_ACCESS_TOKEN
from metadata.utils.logger import ingestion_logger
logger = ingestion_logger()
from metadata.utils.helpers import clean_uri
class SplineClient:
"""
Wrapper on top of Spline REST API
"""
# pylint: disable=too-many-arguments
def __init__(self, config: SplineConnection):
self.config = config
client_config: ClientConfig = ClientConfig(
base_url=clean_uri(self.config.hostPort),
api_version="consumer",
auth_header=AUTHORIZATION_HEADER,
auth_token=lambda: (NO_ACCESS_TOKEN, 0),
)
self.client = REST(client_config)
def _paginate_pipelines(self, pipelines: ExecutionEvents):
while pipelines.pageNum * pipelines.pageSize < pipelines.totalCount:
try:
response = self.client.get(
f"/execution-events?pageNum={pipelines.pageNum+1}"
)
pipelines = ExecutionEvents(**response)
yield pipelines
except Exception as exe:
pipelines.pageNum += 1
logger.debug(traceback.format_exc())
logger.error(f"failed to fetch pipeline list due to: {exe}")
def get_pipelines(self) -> List[dict]:
"""
Method returns the executions events as pipelines
"""
try:
response = self.client.get("/execution-events")
if response:
pipelines = ExecutionEvents(**response)
yield pipelines
yield from self._paginate_pipelines(pipelines)
except Exception as exe:
logger.debug(traceback.format_exc())
logger.error(f"failed to fetch pipeline list due to: {exe}")
def get_pipelines_test_connection(self) -> List[dict]:
"""
Method returns the executions events as pipelines
"""
response = self.client.get("/execution-events")
return ExecutionEvents(**response)
def get_lineage_details(self, pipeline_id: str) -> List[dict]:
"""
Method returns the executions events as pipelines
"""
try:
response = self.client.get(f"/lineage-detailed?execId={pipeline_id}")
if response:
return ExecutionDetail(**response)
except Exception as exe:
logger.debug(traceback.format_exc())
logger.error(f"failed to fetch pipeline list due to: {exe}")
return None

View File

@ -0,0 +1,54 @@
# Copyright 2021 Collate
# Licensed 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.
"""
Source connection handler
"""
from typing import Optional
from metadata.generated.schema.entity.automations.workflow import (
Workflow as AutomationWorkflow,
)
from metadata.generated.schema.entity.services.connections.pipeline.splineConnection import (
SplineConnection,
)
from metadata.ingestion.connections.test_connections import test_connection_steps
from metadata.ingestion.ometa.ometa_api import OpenMetadata
from metadata.ingestion.source.pipeline.spline.client import SplineClient
def get_connection(connection: SplineConnection) -> SplineClient:
"""
Create connection
"""
return SplineClient(config=connection)
def test_connection(
metadata: OpenMetadata,
client: SplineClient,
service_connection: SplineConnection,
automation_workflow: Optional[AutomationWorkflow] = None,
) -> None:
"""
Test connection. This can be executed either as part
of a metadata workflow or during an Automation Workflow
"""
test_fn = {"GetPipelines": client.get_pipelines_test_connection}
test_connection_steps(
metadata=metadata,
test_fn=test_fn,
service_fqn=service_connection.type.value,
automation_workflow=automation_workflow,
)

View File

@ -0,0 +1,213 @@
# Copyright 2021 Collate
# Licensed 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.
"""
Spline source to extract metadata
"""
import traceback
from typing import Iterable, Optional
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 Task
from metadata.generated.schema.entity.data.table import Table
from metadata.generated.schema.entity.services.connections.metadata.openMetadataConnection import (
OpenMetadataConnection,
)
from metadata.generated.schema.entity.services.connections.pipeline.splineConnection import (
SplineConnection,
)
from metadata.generated.schema.metadataIngestion.workflow import (
Source as WorkflowSource,
)
from metadata.generated.schema.type.entityLineage import EntitiesEdge, LineageDetails
from metadata.generated.schema.type.entityReference import EntityReference
from metadata.ingestion.api.source import InvalidSourceException
from metadata.ingestion.models.pipeline_status import OMetaPipelineStatus
from metadata.ingestion.source.pipeline.pipeline_service import PipelineServiceSource
from metadata.ingestion.source.pipeline.spline.models import ExecutionEvent
from metadata.ingestion.source.pipeline.spline.utils import (
parse_dbfs_path,
parse_jdbc_url,
)
from metadata.utils import fqn
from metadata.utils.helpers import clean_uri
from metadata.utils.logger import ingestion_logger
logger = ingestion_logger()
class SplineSource(PipelineServiceSource):
"""
Implements the necessary methods ot extract
Pipeline metadata from Airflow's metadata db
"""
@classmethod
def create(cls, config_dict, metadata_config: OpenMetadataConnection):
config: WorkflowSource = WorkflowSource.parse_obj(config_dict)
connection: SplineConnection = config.serviceConnection.__root__.config
if not isinstance(connection, SplineConnection):
raise InvalidSourceException(
f"Expected SplineConnection, but got {connection}"
)
return cls(config, metadata_config)
def get_connections_jobs(
self, pipeline_details: ExecutionEvent, connection_url: str
):
"""
Returns the list of tasks linked to connection
"""
return [
Task(
name=pipeline_details.executionEventId,
displayName=pipeline_details.applicationName,
description="",
taskUrl=connection_url,
)
]
def yield_pipeline(
self, pipeline_details: ExecutionEvent
) -> Iterable[CreatePipelineRequest]:
"""
Convert a Connection into a Pipeline Entity
:param pipeline_details: pipeline_details object from airbyte
:return: Create Pipeline request with tasks
"""
connection_url = None
if self.service_connection.uiHostPort:
connection_url = (
f"{clean_uri(self.service_connection.uiHostPort)}/app/events/"
f"overview/{pipeline_details.executionEventId}"
)
pipeline_request = CreatePipelineRequest(
name=pipeline_details.executionEventId,
displayName=pipeline_details.applicationName,
description="",
pipelineUrl=connection_url,
tasks=self.get_connections_jobs(pipeline_details, connection_url),
service=self.context.pipeline_service.fullyQualifiedName.__root__,
)
yield pipeline_request
self.register_record(pipeline_request=pipeline_request)
def yield_pipeline_status(
self, pipeline_details: ExecutionEvent
) -> Optional[OMetaPipelineStatus]:
"""
pipeline status not supported for spline connector
"""
def _get_table_entity(
self, database_name: str, schema_name: str, table_name: str
) -> Table:
if not table_name:
return
for service_name in self.source_config.dbServiceNames:
table_fqn = fqn.build(
metadata=self.metadata,
entity_type=Table,
table_name=table_name,
service_name=service_name,
schema_name=schema_name,
database_name=database_name,
)
if table_fqn:
table_entity = self.metadata.get_by_name(entity=Table, fqn=table_fqn)
if table_entity:
return table_entity
return None
def _get_table_from_datasource_name(self, datasource: str):
if (
not datasource
and not datasource.startswith("dbfs")
and not datasource.startswith("jdbc")
):
return
try:
schema_name = None
database_name = None
table_name = None
if datasource.startswith("dbfs") and "/" in datasource:
table_name = parse_dbfs_path(datasource)
if datasource.startswith("jdbc"):
database_name, schema_name, table_name = parse_jdbc_url(datasource)
return self._get_table_entity(database_name, schema_name, table_name)
except Exception as exc:
logger.debug(traceback.format_exc())
logger.warning(f"failed to parse datasource details due to: {exc}")
def yield_pipeline_lineage_details(
self, pipeline_details: ExecutionEvent
) -> Optional[Iterable[AddLineageRequest]]:
"""
Parse all the executions available and create lineage
"""
if not self.source_config.dbServiceNames:
return
lineage_details = self.client.get_lineage_details(
pipeline_details.executionPlanId
)
if (
lineage_details
and lineage_details.executionPlan
and lineage_details.executionPlan.inputs
and lineage_details.executionPlan.output
):
from_entities = lineage_details.executionPlan.inputs
to_entity = lineage_details.executionPlan.output
for from_entity in from_entities:
from_table = (
self._get_table_from_datasource_name(from_entity.source)
if from_entity
else None
)
to_table = (
self._get_table_from_datasource_name(to_entity.source)
if to_entity
else None
)
if from_table and to_table:
yield AddLineageRequest(
edge=EntitiesEdge(
lineageDetails=LineageDetails(
pipeline=EntityReference(
id=self.context.pipeline.id.__root__,
type="pipeline",
)
),
fromEntity=EntityReference(id=from_table.id, type="table"),
toEntity=EntityReference(id=to_table.id, type="table"),
)
)
def get_pipelines_list(self) -> Iterable[ExecutionEvent]:
"""
Get List of all pipelines
"""
for pipelines in self.client.get_pipelines() or []:
for pipeline in pipelines.items or []:
yield pipeline
def get_pipeline_name(self, pipeline_details: ExecutionEvent) -> str:
"""
Get Pipeline Name for filtering
"""
return pipeline_details.applicationName

View File

@ -0,0 +1,46 @@
# Copyright 2021 Collate
# Licensed 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 typing import List, Optional
from pydantic import BaseModel
class ExecutionEvent(BaseModel):
executionEventId: Optional[str]
executionPlanId: Optional[str]
applicationName: Optional[str]
class ExecutionEvents(BaseModel):
items: Optional[List[ExecutionEvent]] = []
totalCount: Optional[int] = 0
pageNum: Optional[int] = 0
pageSize: Optional[int] = 0
class Inputs(BaseModel):
source: Optional[str]
class Output(BaseModel):
source: Optional[str]
class ExecutionPlan(BaseModel):
_id: Optional[str]
name: Optional[str]
inputs: Optional[List[Inputs]] = []
output: Optional[Output]
class ExecutionDetail(BaseModel):
executionPlan: Optional[ExecutionPlan]

View File

@ -0,0 +1,82 @@
# Copyright 2021 Collate
# Licensed 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.
import traceback
from typing import Optional, Tuple
from antlr4.CommonTokenStream import CommonTokenStream
from antlr4.error.ErrorStrategy import BailErrorStrategy
from antlr4.InputStream import InputStream
from metadata.generated.antlr.JdbcUriLexer import JdbcUriLexer
from metadata.generated.antlr.JdbcUriParser import JdbcUriParser
from metadata.utils.constants import DEFAULT_DATABASE
from metadata.utils.logger import ingestion_logger
logger = ingestion_logger()
MULTI_DB_SOURCE = {"postgresql", "oracle:thin", "vertica", "redshift"}
def parse_dbfs_path(path: str) -> Optional[str]:
try:
return path.split("/")[-1]
except Exception as exc:
logger.warning(f"Failed to parse dbfs: {exc}")
logger.error(traceback.format_exc())
return None
def clean_name(name: str) -> str:
"""
replace empty string with None
"""
if name:
return name
def parse_jdbc_url(url: str) -> Tuple[Optional[str], Optional[str], Optional[str]]:
"""
Handle parsing of jdbc url to extract table, schema and database name
"""
try:
lexer = JdbcUriLexer(InputStream(url))
stream = CommonTokenStream(lexer)
parser = JdbcUriParser(stream)
parser._errHandler = BailErrorStrategy() # pylint: disable=protected-acc ess
tree = parser.jdbcUrl()
schema_table = tree.schemaTable()
if schema_table:
table = (
clean_name(schema_table.tableName().getText())
if schema_table.tableName()
else None
)
schema = (
clean_name(schema_table.schemaName().getText())
if schema_table.schemaName()
else None
)
else:
table, schema = None, None
database = (
clean_name(tree.databaseName().getText()) if tree.databaseName() else None
)
if tree.DATABASE_TYPE() and tree.DATABASE_TYPE().getText() in MULTI_DB_SOURCE:
return database, schema, table
return DEFAULT_DATABASE, database, table
except Exception as exc:
logger.warning(f"Failed to parse jdbc url: {exc}")
logger.error(traceback.format_exc())
return None, None, None

View File

@ -0,0 +1,187 @@
{
"execution-events": {
"items": [
{
"executionEventId": "3f784e72-5bf7-5704-8828-ae8464fe915b:lhq160w0",
"executionPlanId": "3f784e72-5bf7-5704-8828-ae8464fe915b",
"frameworkName": "spark 3.3.2",
"applicationName": "jdbc postgres ssl app",
"applicationId": "local-1684226801003",
"timestamp": 1684226827008,
"durationNs": 8.101352417E9,
"dataSourceName": "postgres?sslmode=disable:spline_test.filter",
"dataSourceUri": "jdbc:postgresql://localhost:5432/postgres?sslmode=disable:spline_test.filter",
"dataSourceType": "jdbc",
"append": true
},
{
"executionEventId": "96976f56-9fe3-5bcc-a3e7-ac28781842c9:lhpygcji",
"executionPlanId": "96976f56-9fe3-5bcc-a3e7-ac28781842c9",
"frameworkName": "spark 3.3.2",
"applicationName": "databricks shell",
"applicationId": "local-1684222240845",
"timestamp": 1684222269822,
"durationNs": 1.0835231042E10,
"dataSourceName": "table3",
"dataSourceUri": "dbfs:/user/hive/warehouse/table3",
"dataSourceType": "jdbc",
"append": true
}
],
"totalCount": 2,
"pageNum": 1,
"pageSize": 10,
"totalDateRange": []
},
"lineage-detailed": {
"executionPlan": {
"_id": "3f784e72-5bf7-5704-8828-ae8464fe915b",
"name": "jdbc postgres ssl app",
"systemInfo": {
"name": "spark",
"version": "3.3.2"
},
"agentInfo": {
"name": "spline",
"version": "1.1.0"
},
"extra": {
"appName": "jdbc postgres ssl app",
"attributes": [
{
"id": "3f784e72-5bf7-5704-8828-ae8464fe915b:attr-1",
"name": "sample",
"dataTypeId": "75fe27b9-9a00-5c7d-966f-33ba32333133"
},
{
"id": "3f784e72-5bf7-5704-8828-ae8464fe915b:attr-0",
"name": "id",
"dataTypeId": "e63adadc-648a-56a0-9424-3289858cf0bb"
},
{
"id": "3f784e72-5bf7-5704-8828-ae8464fe915b:attr-3",
"name": "sample",
"dataTypeId": "75fe27b9-9a00-5c7d-966f-33ba32333133"
},
{
"id": "3f784e72-5bf7-5704-8828-ae8464fe915b:attr-2",
"name": "id",
"dataTypeId": "e63adadc-648a-56a0-9424-3289858cf0bb"
},
{
"id": "3f784e72-5bf7-5704-8828-ae8464fe915b:attr-5",
"name": "sample",
"dataTypeId": "75fe27b9-9a00-5c7d-966f-33ba32333133"
},
{
"id": "3f784e72-5bf7-5704-8828-ae8464fe915b:attr-4",
"name": "id",
"dataTypeId": "e63adadc-648a-56a0-9424-3289858cf0bb"
}
],
"dataTypes": [
{
"_typeHint": "dt.Simple",
"id": "e63adadc-648a-56a0-9424-3289858cf0bb",
"name": "int",
"nullable": true
},
{
"_typeHint": "dt.Simple",
"id": "75fe27b9-9a00-5c7d-966f-33ba32333133",
"name": "string",
"nullable": true
},
{
"_typeHint": "dt.Simple",
"id": "a155e715-56ab-59c4-a94b-ed1851a6984a",
"name": "boolean",
"nullable": true
},
{
"_typeHint": "dt.Simple",
"id": "ab4da308-91fb-550a-a5e4-beddecff2a2b",
"name": "int",
"nullable": false
}
]
},
"inputs": [
{
"sourceType": "jdbc",
"source": "jdbc:postgresql://localhost:5432/postgres?sslmode=disable:spline_demo.start"
},
{
"sourceType": "jdbc",
"source": "jdbc:postgresql://localhost:5432/postgres?sslmode=disable:spline_demo.destination"
}
],
"output": {
"sourceType": "jdbc",
"source": "jdbc:postgresql://localhost:5432/postgres?sslmode=disable:spline_test.filter"
}
},
"graph": {
"nodes": [
{
"_id": "3f784e72-5bf7-5704-8828-ae8464fe915b:op-4",
"_type": "Read",
"name": "LogicalRelation",
"properties": null
},
{
"_id": "3f784e72-5bf7-5704-8828-ae8464fe915b:op-5",
"_type": "Read",
"name": "LogicalRelation",
"properties": null
},
{
"_id": "3f784e72-5bf7-5704-8828-ae8464fe915b:op-3",
"_type": "Transformation",
"name": "Union",
"properties": null
},
{
"_id": "3f784e72-5bf7-5704-8828-ae8464fe915b:op-2",
"_type": "Transformation",
"name": "Filter",
"properties": null
},
{
"_id": "3f784e72-5bf7-5704-8828-ae8464fe915b:op-1",
"_type": "Transformation",
"name": "Project",
"properties": null
},
{
"_id": "3f784e72-5bf7-5704-8828-ae8464fe915b:op-0",
"_type": "Write",
"name": "SaveIntoDataSourceCommand",
"properties": null
}
],
"edges": [
{
"source": "3f784e72-5bf7-5704-8828-ae8464fe915b:op-4",
"target": "3f784e72-5bf7-5704-8828-ae8464fe915b:op-3"
},
{
"source": "3f784e72-5bf7-5704-8828-ae8464fe915b:op-5",
"target": "3f784e72-5bf7-5704-8828-ae8464fe915b:op-3"
},
{
"source": "3f784e72-5bf7-5704-8828-ae8464fe915b:op-3",
"target": "3f784e72-5bf7-5704-8828-ae8464fe915b:op-2"
},
{
"source": "3f784e72-5bf7-5704-8828-ae8464fe915b:op-2",
"target": "3f784e72-5bf7-5704-8828-ae8464fe915b:op-1"
},
{
"source": "3f784e72-5bf7-5704-8828-ae8464fe915b:op-1",
"target": "3f784e72-5bf7-5704-8828-ae8464fe915b:op-0"
}
]
}
}
}

View File

@ -0,0 +1,269 @@
# Copyright 2021 Collate
# Licensed 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.
"""
Test Spline using the topology
"""
# pylint: disable=line-too-long
import json
from pathlib import Path
from unittest import TestCase
from unittest.mock import patch
from metadata.generated.schema.api.data.createPipeline import CreatePipelineRequest
from metadata.generated.schema.entity.data.pipeline import Pipeline, Task
from metadata.generated.schema.entity.services.pipelineService import (
PipelineConnection,
PipelineService,
PipelineServiceType,
)
from metadata.generated.schema.metadataIngestion.workflow import (
OpenMetadataWorkflowConfig,
)
from metadata.generated.schema.type.basic import FullyQualifiedEntityName
from metadata.generated.schema.type.entityReference import EntityReference
from metadata.ingestion.ometa.client import REST
from metadata.ingestion.source.pipeline.spline.metadata import SplineSource
from metadata.ingestion.source.pipeline.spline.models import (
ExecutionDetail,
ExecutionEvent,
ExecutionEvents,
ExecutionPlan,
Inputs,
Output,
)
from metadata.ingestion.source.pipeline.spline.utils import parse_jdbc_url
from metadata.utils.constants import DEFAULT_DATABASE, UTF_8
mock_file_path = (
Path(__file__).parent.parent.parent / "resources/datasets/spline_dataset.json"
)
with open(mock_file_path, encoding=UTF_8) as file:
mock_data: dict = json.load(file)
MOCK_SPLINE_UI_URL = "http://localhost:9090"
mock_spline_config = {
"source": {
"type": "spline",
"serviceName": "spline_source",
"serviceConnection": {
"config": {
"type": "Spline",
"hostPort": "http://localhost:8080",
"uiHostPort": MOCK_SPLINE_UI_URL,
}
},
"sourceConfig": {"config": {"type": "PipelineMetadata"}},
},
"sink": {"type": "metadata-rest", "config": {}},
"workflowConfig": {
"openMetadataServerConfig": {
"hostPort": "http://localhost:8585/api",
"authProvider": "openmetadata",
"securityConfig": {
"jwtToken": "eyJraWQiOiJHYjM4OWEtOWY3Ni1nZGpzLWE5MmotMDI0MmJrOTQzNTYiLCJ0eXAiOiJKV1QiLCJhbGciOiJSUzI1NiJ9.eyJzdWIiOiJhZG1pbiIsImlzQm90IjpmYWxzZSwiaXNzIjoib3Blbi1tZXRhZGF0YS5vcmciLCJpYXQiOjE2NjM5Mzg0NjIsImVtYWlsIjoiYWRtaW5Ab3Blbm1ldGFkYXRhLm9yZyJ9.tS8um_5DKu7HgzGBzS1VTA5uUjKWOCU0B_j08WXBiEC0mr0zNREkqVfwFDD-d24HlNEbrqioLsBuFRiwIWKc1m_ZlVQbG7P36RUxhuv2vbSp80FKyNM-Tj93FDzq91jsyNmsQhyNv_fNr3TXfzzSPjHt8Go0FMMP66weoKMgW2PbXlhVKwEuXUHyakLLzewm9UMeQaEiRzhiTMU3UkLXcKbYEJJvfNFcLwSl9W8JCO_l0Yj3ud-qt_nQYEZwqW6u5nfdQllN133iikV4fM5QZsMCnm8Rq1mvLR0y9bmJiD7fwM1tmJ791TUWqmKaTnP49U493VanKpUAfzIiOiIbhg"
},
}
},
}
PIPELINE_ID = "3f784e72-5bf7-5704-8828-ae8464fe915b:lhq160w0"
MOCK_PIPELINE_URL = f"{MOCK_SPLINE_UI_URL}/app/events/overview/{PIPELINE_ID}"
EXPECTED_CREATED_PIPELINES = CreatePipelineRequest(
name=PIPELINE_ID,
displayName="jdbc postgres ssl app",
description="",
pipelineUrl=MOCK_PIPELINE_URL,
tasks=[
Task(
name=PIPELINE_ID,
displayName="jdbc postgres ssl app",
description="",
taskUrl=MOCK_PIPELINE_URL,
)
],
service=FullyQualifiedEntityName(__root__="spline_source"),
)
MOCK_PIPELINE_SERVICE = PipelineService(
id="85811038-099a-11ed-861d-0242ac120002",
name="spline_source",
fullyQualifiedName=FullyQualifiedEntityName(__root__="spline_source"),
connection=PipelineConnection(),
serviceType=PipelineServiceType.Airbyte,
)
MOCK_PIPELINE = Pipeline(
id="2aaa012e-099a-11ed-861d-0242ac120002",
name=PIPELINE_ID,
fullyQualifiedName=f"spline_source.{PIPELINE_ID}",
displayName="MSSQL <> Postgres",
description="",
pipelineUrl=MOCK_PIPELINE_URL,
tasks=[
Task(
name=PIPELINE_ID,
displayName="jdbc postgres ssl app",
description="",
taskUrl=MOCK_PIPELINE_URL,
)
],
service=EntityReference(
id="85811038-099a-11ed-861d-0242ac120002", type="pipelineService"
),
)
EXPECTED_SPLINE_PIPELINES = ExecutionEvents(
items=[
ExecutionEvent(
executionEventId="3f784e72-5bf7-5704-8828-ae8464fe915b:lhq160w0",
executionPlanId="3f784e72-5bf7-5704-8828-ae8464fe915b",
applicationName="jdbc postgres ssl app",
),
ExecutionEvent(
executionEventId="96976f56-9fe3-5bcc-a3e7-ac28781842c9:lhpygcji",
executionPlanId="96976f56-9fe3-5bcc-a3e7-ac28781842c9",
applicationName="databricks shell",
),
],
totalCount=2,
pageNum=1,
pageSize=10,
)
EXPECTED_LINEAGE_DETAILS = ExecutionDetail(
executionPlan=ExecutionPlan(
_id="3f784e72-5bf7-5704-8828-ae8464fe915b",
name="jdbc postgres ssl app",
inputs=[
Inputs(
source="jdbc:postgresql://localhost:5432/postgres?sslmode=disable:spline_demo.start"
),
Inputs(
source="jdbc:postgresql://localhost:5432/postgres?sslmode=disable:spline_demo.destination"
),
],
output=Output(
source="jdbc:postgresql://localhost:5432/postgres?sslmode=disable:spline_test.filter"
),
)
)
JDBC_PARSING_EXAMPLES = [
(
"jdbc:postgresql://localhost:5432/postgres?sslmode=disable:spline_demo.start",
"postgres",
"spline_demo",
"start",
),
(
"jdbc:postgresql://localhost:5432/postgres?sslmode=disable:spline_demo.destination",
"postgres",
"spline_demo",
"destination",
),
(
"jdbc:postgresql://localhost:5432/postgres?sslmode=disable:spline_test.filter",
"postgres",
"spline_test",
"filter",
),
(
"jdbc:postgresql://localhost:5432/postgres?sslmode=disable:filter",
"postgres",
None,
"filter",
),
("jdbc:oracle:thin:@localhost:1521:test_table", None, None, "test_table"),
(
"jdbc:oracle:thin:@localhost:1521:test_schema.test_table",
None,
"test_schema",
"test_table",
),
(
"jdbc:oracle:thin:@localhost:1521/TESTDB:test_schema.test_table",
"TESTDB",
"test_schema",
"test_table",
),
(
"jdbc:mysql://localhost:3306/openmetadata_db?allowPublicKeyRetrieval=true&useSSL=false&serverTimezone=UTC",
DEFAULT_DATABASE,
"openmetadata_db",
None,
),
(
"jdbc:mysql://localhost:3306/openmetadata_db?allowPublicKeyRetrieval=true&useSSL=false&serverTimezone=UTC:filter",
DEFAULT_DATABASE,
"openmetadata_db",
"filter",
),
("jdbc:hive2://localhost:10000/:test_tbl", DEFAULT_DATABASE, None, "test_tbl"),
("jdbc:vertica://localhost:5432/custom_db:test_tbl", "custom_db", None, "test_tbl"),
(
"jdbc:vertica://localhost:5432/custom_db:schem.test_tbl",
"custom_db",
"schem",
"test_tbl",
),
(
"jdbc:redshift://ec2-192-168-29-1.us-east-2.compute.amazonaws.com:5432/custom_db:test_tbl",
"custom_db",
None,
"test_tbl",
),
]
class SplineUnitTest(TestCase):
@patch(
"metadata.ingestion.source.pipeline.pipeline_service.PipelineServiceSource.test_connection"
)
def __init__(self, methodName, test_connection) -> None:
super().__init__(methodName)
test_connection.return_value = False
config = OpenMetadataWorkflowConfig.parse_obj(mock_spline_config)
self.spline = SplineSource.create(
mock_spline_config["source"],
config.workflowConfig.openMetadataServerConfig,
)
self.spline.context.__dict__["pipeline"] = MOCK_PIPELINE
self.spline.context.__dict__["pipeline_service"] = MOCK_PIPELINE_SERVICE
def test_client(self):
with patch.object(REST, "get", return_value=mock_data.get("execution-events")):
self.assertEqual(
list(self.spline.client.get_pipelines()), [EXPECTED_SPLINE_PIPELINES]
)
with patch.object(REST, "get", return_value=mock_data.get("lineage-detailed")):
self.assertEqual(
self.spline.client.get_lineage_details(PIPELINE_ID),
EXPECTED_LINEAGE_DETAILS,
)
def test_pipeline_name(self):
assert (
self.spline.get_pipeline_name(EXPECTED_SPLINE_PIPELINES.items[0])
== EXPECTED_SPLINE_PIPELINES.items[0].applicationName
)
def test_pipelines(self):
pipline = list(self.spline.yield_pipeline(EXPECTED_SPLINE_PIPELINES.items[0]))[
0
]
assert pipline == EXPECTED_CREATED_PIPELINES
def test_jdbc_parsing(self):
for example in JDBC_PARSING_EXAMPLES:
result = parse_jdbc_url(example[0])
self.assertEqual(result, example[1:])

View File

@ -0,0 +1,296 @@
---
title: Run Spline Connector using Airflow SDK
slug: /connectors/pipeline/spline/airflow
---
# Run Spline using the Airflow SDK
In this section, we provide guides and references to use the Spline connector.
Configure and schedule Spline metadata and profiler workflows from the OpenMetadata UI:
- [Requirements](#requirements)
- [Metadata Ingestion](#metadata-ingestion)
## Requirements
{%inlineCallout icon="description" bold="OpenMetadata 0.12 or later" href="/deployment"%}
To deploy OpenMetadata, check the Deployment guides.
{% /inlineCallout %}
To run the Ingestion via the UI you'll need to use the OpenMetadata Ingestion Container, which comes shipped with
custom Airflow plugins to handle the workflow deployment.
### Python Requirements
To run the Spline ingestion, you will need to install:
```bash
pip3 install "openmetadata-ingestion"
```
## Metadata Ingestion
All connectors are defined as JSON Schemas.
[Here](https://github.com/open-metadata/OpenMetadata/blob/main/openmetadata-spec/src/main/resources/json/schema/entity/services/connections/pipeline/splineConnection.json)
you can find the structure to create a connection to Spline.
In order to create and run a Metadata Ingestion workflow, we will follow
the steps to create a YAML configuration able to connect to the source,
process the Entities if needed, and reach the OpenMetadata server.
The workflow is modeled around the following
[JSON Schema](https://github.com/open-metadata/OpenMetadata/blob/main/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/workflow.json)
### 1. Define the YAML Config
This is a sample config for Spline:
{% codePreview %}
{% codeInfoContainer %}
#### Source Configuration - Service Connection
{% codeInfo srNumber=1 %}
**hostPort**: Spline REST Server API Host & Port, OpenMetadata uses Spline REST Server APIs to extract the execution details from spline to generate lineage. This should be specified as a URI string in the format `scheme://hostname:port`. E.g., `http://localhost:8080`, `http://host.docker.internal:8080`.
**uiHostPort**: Spline UI Host & Port is an optional field which is used for generating redirection URL from OpenMetadata to Spline Portal. This should be specified as a URI string in the format `scheme://hostname:port`. E.g., `http://localhost:9090`, `http://host.docker.internal:9090`.
{% /codeInfo %}
#### Source Configuration - Source Config
{% codeInfo srNumber=2 %}
The `sourceConfig` is defined [here](https://github.com/open-metadata/OpenMetadata/blob/main/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/pipelineServiceMetadataPipeline.json):
**dbServiceNames**: Database Service Name for the creation of lineage, if the source supports it.
**includeTags**: Set the 'Include Tags' toggle to control whether to include tags as part of metadata ingestion.
**markDeletedPipelines**: Set the Mark Deleted Pipelines toggle to flag pipelines as soft-deleted if they are not present anymore in the source system.
**pipelineFilterPattern** and **chartFilterPattern**: Note that the `pipelineFilterPattern` and `chartFilterPattern` both support regex as include or exclude.
{% /codeInfo %}
#### Sink Configuration
{% codeInfo srNumber=3 %}
To send the metadata to OpenMetadata, it needs to be specified as `type: metadata-rest`.
{% /codeInfo %}
#### Workflow Configuration
{% codeInfo srNumber=4 %}
The main property here is the `openMetadataServerConfig`, where you can define the host and security provider of your OpenMetadata installation.
For a simple, local installation using our docker containers, this looks like:
{% /codeInfo %}
{% /codeInfoContainer %}
{% codeBlock fileName="filename.yaml" %}
```yaml
source:
type: spline
serviceName: spline_source
serviceConnection:
config:
type: Spline
```
```yaml {% srNumber=1 %}
hostPort: http://localhost:8080
uiHostPort: http://localhost:9090
```
```yaml {% srNumber=2 %}
sourceConfig:
config:
type: PipelineMetadata
# markDeletedPipelines: True
# includeTags: True
# includeLineage: true
# dbServiceNames:
# - local_hive
# pipelineFilterPattern:
# includes:
# - pipeline1
# - pipeline2
# excludes:
# - pipeline3
# - pipeline4
```
```yaml {% srNumber=3 %}
sink:
type: metadata-rest
config: {}
```
```yaml {% srNumber=4 %}
workflowConfig:
openMetadataServerConfig:
hostPort: "http://localhost:8585/api"
authProvider: openmetadata
securityConfig:
jwtToken: "{bot_jwt_token}"
```
{% /codeBlock %}
{% /codePreview %}
### Workflow Configs for Security Provider
We support different security providers. You can find their definitions [here](https://github.com/open-metadata/OpenMetadata/tree/main/openmetadata-spec/src/main/resources/json/schema/security/client).
## Openmetadata JWT Auth
- JWT tokens will allow your clients to authenticate against the OpenMetadata server. To enable JWT Tokens, you will get more details [here](/deployment/security/enable-jwt-tokens).
```yaml
workflowConfig:
openMetadataServerConfig:
hostPort: "http://localhost:8585/api"
authProvider: openmetadata
securityConfig:
jwtToken: "{bot_jwt_token}"
```
- You can refer to the JWT Troubleshooting section [link](/deployment/security/jwt-troubleshooting) for any issues in your JWT configuration. If you need information on configuring the ingestion with other security providers in your bots, you can follow this doc [link](/deployment/security/workflow-config-auth).
### 2. Prepare the Ingestion DAG
Create a Python file in your Airflow DAGs directory with the following contents:
{% codePreview %}
{% codeInfoContainer %}
{% codeInfo srNumber=5 %}
#### Import necessary modules
The `Workflow` class that is being imported is a part of a metadata ingestion framework, which defines a process of getting data from different sources and ingesting it into a central metadata repository.
Here we are also importing all the basic requirements to parse YAMLs, handle dates and build our DAG.
{% /codeInfo %}
{% codeInfo srNumber=6 %}
**Default arguments for all tasks in the Airflow DAG.**
- Default arguments dictionary contains default arguments for tasks in the DAG, including the owner's name, email address, number of retries, retry delay, and execution timeout.
{% /codeInfo %}
{% codeInfo srNumber=7 %}
- **config**: Specifies config for the metadata ingestion as we prepare above.
{% /codeInfo %}
{% codeInfo srNumber=8 %}
- **metadata_ingestion_workflow()**: This code defines a function `metadata_ingestion_workflow()` that loads a YAML configuration, creates a `Workflow` object, executes the workflow, checks its status, prints the status to the console, and stops the workflow.
{% /codeInfo %}
{% codeInfo srNumber=9 %}
- **DAG**: creates a DAG using the Airflow framework, and tune the DAG configurations to whatever fits with your requirements
- For more Airflow DAGs creation details visit [here](https://airflow.apache.org/docs/apache-airflow/stable/core-concepts/dags.html#declaring-a-dag).
{% /codeInfo %}
Note that from connector to connector, this recipe will always be the same.
By updating the `YAML configuration`, you will be able to extract metadata from different sources.
{% /codeInfoContainer %}
{% codeBlock fileName="filename.py" %}
```python {% srNumber=5 %}
import pathlib
import yaml
from datetime import timedelta
from airflow import DAG
from metadata.config.common import load_config_file
from metadata.ingestion.api.workflow import Workflow
from airflow.utils.dates import days_ago
try:
from airflow.operators.python import PythonOperator
except ModuleNotFoundError:
from airflow.operators.python_operator import PythonOperator
```
```python {% srNumber=6 %}
default_args = {
"owner": "user_name",
"email": ["username@org.com"],
"email_on_failure": False,
"retries": 3,
"retry_delay": timedelta(minutes=5),
"execution_timeout": timedelta(minutes=60)
}
```
```python {% srNumber=7 %}
config = """
<your YAML configuration>
"""
```
```python {% srNumber=8 %}
def metadata_ingestion_workflow():
workflow_config = yaml.safe_load(config)
workflow = Workflow.create(workflow_config)
workflow.execute()
workflow.raise_from_status()
workflow.print_status()
workflow.stop()
```
```python {% srNumber=9 %}
with DAG(
"sample_data",
default_args=default_args,
description="An example DAG which runs a OpenMetadata ingestion workflow",
start_date=days_ago(1),
is_paused_upon_creation=False,
schedule_interval='*/5 * * * *',
catchup=False,
) as dag:
ingest_task = PythonOperator(
task_id="ingest_using_recipe",
python_callable=metadata_ingestion_workflow,
)
```
{% /codeBlock %}
{% /codePreview %}

View File

@ -0,0 +1,182 @@
---
title: Run Spline Connector using the CLI
slug: /connectors/pipeline/spline/cli
---
# Run Spline using the metadata CLI
In this section, we provide guides and references to use the Spline connector.
Configure and schedule Spline metadata and profiler workflows from the OpenMetadata UI:
- [Requirements](#requirements)
- [Metadata Ingestion](#metadata-ingestion)
## Requirements
{%inlineCallout icon="description" bold="OpenMetadata 0.12 or later" href="/deployment"%}
To deploy OpenMetadata, check the Deployment guides.
{% /inlineCallout %}
To run the Ingestion via the UI you'll need to use the OpenMetadata Ingestion Container, which comes shipped with
custom Airflow plugins to handle the workflow deployment.
### Python Requirements
To run the Spline ingestion, you will need to install:
```bash
pip3 install "openmetadata-ingestion"
```
## Metadata Ingestion
All connectors are defined as JSON Schemas.
[Here](https://github.com/open-metadata/OpenMetadata/blob/main/openmetadata-spec/src/main/resources/json/schema/entity/services/connections/pipeline/splineConnection.json)
you can find the structure to create a connection to Spline.
In order to create and run a Metadata Ingestion workflow, we will follow
the steps to create a YAML configuration able to connect to the source,
process the Entities if needed, and reach the OpenMetadata server.
The workflow is modeled around the following
[JSON Schema](https://github.com/open-metadata/OpenMetadata/blob/main/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/workflow.json)
### 1. Define the YAML Config
This is a sample config for Spline:
{% codePreview %}
{% codeInfoContainer %}
#### Source Configuration - Service Connection
{% codeInfo srNumber=1 %}
**hostPort**: Spline REST Server API Host & Port, OpenMetadata uses Spline REST Server APIs to extract the execution details from spline to generate lineage. This should be specified as a URI string in the format `scheme://hostname:port`. E.g., `http://localhost:8080`, `http://host.docker.internal:8080`.
**uiHostPort**: Spline UI Host & Port is an optional field which is used for generating redirection URL from OpenMetadata to Spline Portal. This should be specified as a URI string in the format `scheme://hostname:port`. E.g., `http://localhost:9090`, `http://host.docker.internal:9090`.
{% /codeInfo %}
#### Source Configuration - Source Config
{% codeInfo srNumber=2 %}
The `sourceConfig` is defined [here](https://github.com/open-metadata/OpenMetadata/blob/main/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/pipelineServiceMetadataPipeline.json):
**dbServiceNames**: Database Service Name for the creation of lineage, if the source supports it.
**includeTags**: Set the 'Include Tags' toggle to control whether to include tags as part of metadata ingestion.
**markDeletedPipelines**: Set the Mark Deleted Pipelines toggle to flag pipelines as soft-deleted if they are not present anymore in the source system.
**pipelineFilterPattern** and **chartFilterPattern**: Note that the `pipelineFilterPattern` and `chartFilterPattern` both support regex as include or exclude.
{% /codeInfo %}
#### Sink Configuration
{% codeInfo srNumber=3 %}
To send the metadata to OpenMetadata, it needs to be specified as `type: metadata-rest`.
{% /codeInfo %}
#### Workflow Configuration
{% codeInfo srNumber=4 %}
The main property here is the `openMetadataServerConfig`, where you can define the host and security provider of your OpenMetadata installation.
For a simple, local installation using our docker containers, this looks like:
{% /codeInfo %}
{% /codeInfoContainer %}
{% codeBlock fileName="filename.yaml" %}
```yaml
source:
type: spline
serviceName: spline_source
serviceConnection:
config:
type: Spline
```
```yaml {% srNumber=1 %}
hostPort: http://localhost:8080
uiHostPort: http://localhost:9090
```
```yaml {% srNumber=2 %}
sourceConfig:
config:
type: PipelineMetadata
# markDeletedPipelines: True
# includeTags: True
# includeLineage: true
# dbServiceNames:
# - local_hive
# pipelineFilterPattern:
# includes:
# - pipeline1
# - pipeline2
# excludes:
# - pipeline3
# - pipeline4
```
```yaml {% srNumber=3 %}
sink:
type: metadata-rest
config: {}
```
```yaml {% srNumber=4 %}
workflowConfig:
openMetadataServerConfig:
hostPort: "http://localhost:8585/api"
authProvider: openmetadata
securityConfig:
jwtToken: "{bot_jwt_token}"
```
{% /codeBlock %}
{% /codePreview %}
### Workflow Configs for Security Provider
We support different security providers. You can find their definitions [here](https://github.com/open-metadata/OpenMetadata/tree/main/openmetadata-spec/src/main/resources/json/schema/security/client).
## Openmetadata JWT Auth
- JWT tokens will allow your clients to authenticate against the OpenMetadata server. To enable JWT Tokens, you will get more details [here](/deployment/security/enable-jwt-tokens).
```yaml
workflowConfig:
openMetadataServerConfig:
hostPort: "http://localhost:8585/api"
authProvider: openmetadata
securityConfig:
jwtToken: "{bot_jwt_token}"
```
- You can refer to the JWT Troubleshooting section [link](/deployment/security/jwt-troubleshooting) for any issues in your JWT configuration. If you need information on configuring the ingestion with other security providers in your bots, you can follow this doc [link](/deployment/security/workflow-config-auth).
### 2. Run with the CLI
First, we will need to save the YAML file. Afterward, and with all requirements installed, we can run:
```bash
metadata ingest -c <path-to-yaml>
```
Note that from connector to connector, this recipe will always be the same. By updating the YAML configuration,
you will be able to extract metadata from different sources.

View File

@ -0,0 +1,291 @@
---
title: Spline
slug: /connectors/pipeline/spline
---
# Spline
In this section, we provide guides and references to use the Spline connector.
Configure and schedule Spline metadata and profiler workflows from the OpenMetadata UI:
- [Requirements](#requirements)
- [Metadata Ingestion](#metadata-ingestion)
If you don't want to use the OpenMetadata Ingestion container to configure the workflows via the UI, then you can check
the following docs to connect using Airflow SDK or with the CLI.
{% tilesContainer %}
{% tile
title="Ingest with Airflow"
description="Configure the ingestion using Airflow SDK"
link="/connectors/pipeline/spline/airflow"
/ %}
{% tile
title="Ingest with the CLI"
description="Run a one-time ingestion using the metadata CLI"
link="/connectors/pipeline/spline/cli"
/ %}
{% /tilesContainer %}
## Requirements
{%inlineCallout icon="description" bold="OpenMetadata 0.12 or later" href="/deployment"%}
To deploy OpenMetadata, check the Deployment guides.
{% /inlineCallout %}
To run the Ingestion via the UI you'll need to use the OpenMetadata Ingestion Container, which comes shipped with
custom Airflow plugins to handle the workflow deployment.
## Metadata Ingestion
{% stepsContainer %}
{% step srNumber=1 %}
{% stepDescription title="1. Visit the Services Page" %}
The first step is ingesting the metadata from your sources. Under
Settings, you will find a Services link an external source system to
OpenMetadata. Once a service is created, it can be used to configure
metadata, usage, and profiler workflows.
To visit the Services page, select Services from the Settings menu.
{% /stepDescription %}
{% stepVisualInfo %}
{% image
src="/images/v1.0.0/connectors/visit-services.png"
alt="Visit Services Page"
caption="Find Dashboard option on left panel of the settings page" /%}
{% /stepVisualInfo %}
{% /step %}
{% step srNumber=2 %}
{% stepDescription title="2. Create a New Service" %}
Click on the 'Add New Service' button to start the Service creation.
{% /stepDescription %}
{% stepVisualInfo %}
{% image
src="/images/v1.0.0/connectors/create-service.png"
alt="Create a new service"
caption="Add a new Service from the Dashboard Services page" /%}
{% /stepVisualInfo %}
{% /step %}
{% step srNumber=3 %}
{% stepDescription title="3. Select the Service Type" %}
Select Spline as the service type and click Next.
{% /stepDescription %}
{% stepVisualInfo %}
{% image
src="/images/v1.1.0/connectors/spline/select-service.png"
alt="Select Service"
caption="Select your service from the list" /%}
{% /stepVisualInfo %}
{% /step %}
{% step srNumber=4 %}
{% stepDescription title="4. Name and Describe your Service" %}
Provide a name and description for your service as illustrated below.
#### Service Name
OpenMetadata uniquely identifies services by their Service Name. Provide
a name that distinguishes your deployment from other services, including
the other {connector} services that you might be ingesting metadata
from.
{% /stepDescription %}
{% stepVisualInfo %}
{% image
src="/images/v1.1.0/connectors/spline/add-new-service.png"
alt="Add New Service"
caption="Provide a Name and description for your Service" /%}
{% /stepVisualInfo %}
{% /step %}
{% step srNumber=5 %}
{% stepDescription title="5. Configure the Service Connection" %}
In this step, we will configure the connection settings required for
this connector. Please follow the instructions below to ensure that
you've configured the connector to read from your spline service as
desired.
{% /stepDescription %}
{% stepVisualInfo %}
{% image
src="/images/v1.1.0/connectors/spline/service-connection.png"
alt="Configure service connection"
caption="Configure the service connection by filling the form" /%}
{% /stepVisualInfo %}
{% /step %}
{% extraContent parentTagName="stepsContainer" %}
#### Connection Options
- **Spline REST Server Host & Port**: OpenMetadata uses Spline REST Server APIs to extract the execution details from spline to generate lineage. This should be specified as a URI string in the format `scheme://hostname:port`. E.g., `http://localhost:8080`, `http://host.docker.internal:8080`.
- **Spline UI Host & Port**: Spline UI Host & Port is an optional field which is used for generating redirection URL from OpenMetadata to Spline Portal. This should be specified as a URI string in the format `scheme://hostname:port`. E.g., `http://localhost:9090`, `http://host.docker.internal:9090`.
{% /extraContent %}
{% step srNumber=6 %}
{% stepDescription title="6. Test the Connection" %}
Once the credentials have been added, click on `Test Connection` and Save
the changes.
{% /stepDescription %}
{% stepVisualInfo %}
{% image
src="/images/v1.0.0/connectors/test-connection.png"
alt="Test Connection"
caption="Test the connection and save the Service" /%}
{% /stepVisualInfo %}
{% /step %}
{% step srNumber=7 %}
{% stepDescription title="7. Configure Metadata Ingestion" %}
In this step we will configure the metadata ingestion pipeline,
Please follow the instructions below
{% /stepDescription %}
{% stepVisualInfo %}
{% image
src="/images/v1.1.0/connectors/configure-metadata-ingestion-pipeline.png"
alt="Configure Metadata Ingestion"
caption="Configure Metadata Ingestion Page" /%}
{% /stepVisualInfo %}
{% /step %}
{% extraContent parentTagName="stepsContainer" %}
#### Metadata Ingestion Options
- **Name**: This field refers to the name of ingestion pipeline, you can customize the name or use the generated name.
- **Pipeline Filter Pattern (Optional)**: Use to pipeline filter patterns to control whether or not to include pipeline as part of metadata ingestion.
- **Include**: Explicitly include pipeline by adding a list of comma-separated regular expressions to the Include field. OpenMetadata will include all pipeline with names matching one or more of the supplied regular expressions. All other schemas will be excluded.
- **Exclude**: Explicitly exclude pipeline by adding a list of comma-separated regular expressions to the Exclude field. OpenMetadata will exclude all pipeline with names matching one or more of the supplied regular expressions. All other schemas will be included.
- **Database Service Name (Optional)**: Enter the name of Database Service which is already ingested in OpenMetadata to create lineage between dashboards and database tables.
- **Include lineage (toggle)**: Set the Include lineage toggle to control whether or not to include lineage between pipelines and data sources as part of metadata ingestion.
- **Enable Debug Log (toggle)**: Set the Enable Debug Log toggle to set the default log level to debug, these logs can be viewed later in Airflow.
- **Mark Deleted Pipelines (toggle)**: Set the Mark Deleted Pipelines toggle to flag pipelines as soft-deleted if they are not present anymore in the source system.
{% /extraContent %}
{% step srNumber=8 %}
{% stepDescription title="8. Schedule the Ingestion and Deploy" %}
Scheduling can be set up at an hourly, daily, weekly, or manual cadence. The
timezone is in UTC. Select a Start Date to schedule for ingestion. It is
optional to add an End Date.
Review your configuration settings. If they match what you intended,
click Deploy to create the service and schedule metadata ingestion.
If something doesn't look right, click the Back button to return to the
appropriate step and change the settings as needed.
After configuring the workflow, you can click on Deploy to create the
pipeline.
{% /stepDescription %}
{% stepVisualInfo %}
{% image
src="/images/v1.0.0/connectors/schedule.png"
alt="Schedule the Workflow"
caption="Schedule the Ingestion Pipeline and Deploy" /%}
{% /stepVisualInfo %}
{% /step %}
{% step srNumber=9 %}
{% stepDescription title="9. View the Ingestion Pipeline" %}
Once the workflow has been successfully deployed, you can view the
Ingestion Pipeline running from the Service Page.
{% /stepDescription %}
{% stepVisualInfo %}
{% image
src="/images/v1.0.0/connectors/view-ingestion-pipeline.png"
alt="View Ingestion Pipeline"
caption="View the Ingestion Pipeline from the Service Page" /%}
{% /stepVisualInfo %}
{% /step %}
{% /stepsContainer %}
## Troubleshooting
### Workflow Deployment Error
If there were any errors during the workflow deployment process, the
Ingestion Pipeline Entity will still be created, but no workflow will be
present in the Ingestion container.
- You can then edit the Ingestion Pipeline and Deploy it again.
- From the Connection tab, you can also Edit the Service if needed.
{% image
src="/images/v1.0.0/connectors/workflow-deployment-error.png"
alt="Workflow Deployment Error"
caption="Edit and Deploy the Ingestion Pipeline" /%}

Binary file not shown.

After

Width:  |  Height:  |  Size: 110 KiB

Binary file not shown.

After

Width:  |  Height:  |  Size: 152 KiB

Binary file not shown.

After

Width:  |  Height:  |  Size: 93 KiB

View File

@ -64,6 +64,7 @@ OpenMetadata can extract metadata from the following list of 55 connectors:
- [Glue](/connectors/pipeline/glue-pipeline)
- [Domo Pipeline](/connectors/pipeline/domo-pipeline)
- [NiFi](/connectors/pipeline/nifi)
- [Spline](/connectors/pipeline/spline)
## ML Model Services

View File

@ -15,6 +15,7 @@ This is the supported list of connectors for Pipeline Services:
- [Domo Pipeline](/connectors/pipeline/domo-pipeline)
- [Databricks Pipeline](/connectors/pipeline/databricks-pipeline)
- [NiFi](/connectors/pipeline/nifi)
- [Spline](/connectors/pipeline/spline)
If you have a request for a new connector, don't hesitate to reach out in [Slack](https://slack.open-metadata.org/) or
open a [feature request](https://github.com/open-metadata/OpenMetadata/issues/new/choose) in our GitHub repo.

View File

@ -525,6 +525,12 @@ site_menu:
url: /connectors/pipeline/domo-pipeline/airflow
- category: Connectors / Pipeline / Domo Pipeline / CLI
url: /connectors/pipeline/domo-pipeline/cli
- category: Connectors / Pipeline / Spline
url: /connectors/pipeline/spline
- category: Connectors / Pipeline / Spline / Airflow
url: /connectors/pipeline/spline/airflow
- category: Connectors / Pipeline / Spline / CLI
url: /connectors/pipeline/spline/cli
- category: Connectors / ML Model
url: /connectors/ml-model

Binary file not shown.

Before

Width:  |  Height:  |  Size: 161 KiB

After

Width:  |  Height:  |  Size: 117 KiB

View File

@ -0,0 +1,15 @@
{
"name": "Spline",
"fullyQualifiedName": "Spline",
"displayName": "Spline Test Connection",
"description": "This Test Connection validates the access against the server and basic metadata extraction of pipelines.",
"steps": [
{
"name": "GetPipelines",
"description": "List all execution events",
"errorMessage": "Failed to fetch pipelines, please validate the credentials",
"shortCircuit": true,
"mandatory": true
}
]
}

View File

@ -0,0 +1,99 @@
grammar JdbcUri;
jdbcUrl
: 'jdbc:' DATABASE_TYPE serverName? PORT_NUMBER? ('/' databaseName)? (CONNECTION_ARG_INIT CONNECTION_ARG (AMP CONNECTION_ARG)*)? schemaTable?
;
schemaTable
: COLON (schemaName PERIOD)? tableName
;
databaseName
: IDENTIFIER?
;
schemaName
: IDENTIFIER
;
tableName
: IDENTIFIER
;
DATABASE_TYPE
: 'mysql'
| 'postgresql'
| 'oracle:thin'
| 'clickhouse'
| 'trino'
| 'presto'
| 'vertica'
| 'hive2'
| 'redshift'
;
URI_SEPARATOR
: '://'
| ':@//'
| ':@'
;
serverName
: HOST_NAME
| IPV4_ADDRESS
| IPV6_ADDRESS
| URI_SEPARATOR IDENTIFIER
| URI_SEPARATOR
;
PORT_NUMBER
: COLON [0-9]+
;
IDENTIFIER
: [a-zA-Z][a-zA-Z0-9_]*
;
HOST_NAME
: URI_SEPARATOR [a-zA-Z][a-zA-Z0-9.-]*[a-zA-Z0-9]
;
IPV4_ADDRESS
: URI_SEPARATOR [0-9]+ PERIOD [0-9]+ PERIOD [0-9]+ PERIOD [0-9]+
;
IPV6_ADDRESS
: URI_SEPARATOR '[' HEXDIGIT+ (COLON HEXDIGIT+)* (COLON IPV4_ADDRESS)? ']'
;
HEXDIGIT
: [0-9a-fA-F]
;
CONNECTION_ARG
: IDENTIFIER '=' IDENTIFIER
;
CONNECTION_ARG_INIT
: '?'
;
PERIOD
: '.'
;
COLON
: ':'
;
AMP
: '&'
;
WS
: [ \t\r\n]+ -> skip
;

View File

@ -0,0 +1,45 @@
{
"$id": "https://open-metadata.org/schema/entity/services/connections/pipeline/splineConnection.json",
"$schema": "http://json-schema.org/draft-07/schema#",
"title": "SplineConnection",
"description": "Spline Metadata Database Connection Config",
"type": "object",
"javaType": "org.openmetadata.schema.services.connections.pipeline.SplineConnection",
"definitions": {
"SplineType": {
"description": "Service type.",
"type": "string",
"enum": ["Spline"],
"default": "Spline"
}
},
"properties": {
"type": {
"title": "Service Type",
"description": "Service Type",
"$ref": "#/definitions/SplineType",
"default": "Spline"
},
"hostPort": {
"title": "Spline REST Server Host & Port",
"expose": true,
"description": "Spline REST Server Host & Port.",
"type": "string",
"format": "uri"
},
"uiHostPort": {
"title": "Spline UI Host & Port",
"expose": true,
"description": "Spline UI Host & Port.",
"type": "string",
"format": "uri"
},
"supportsMetadataExtraction": {
"title": "Supports Metadata Extraction",
"$ref": "../connectionBasicType.json#/definitions/supportsMetadataExtraction"
}
},
"additionalProperties": false,
"required": ["hostPort"]
}

View File

@ -25,7 +25,8 @@
"Nifi",
"DomoPipeline",
"CustomPipeline",
"DatabricksPipeline"
"DatabricksPipeline",
"Spline"
],
"javaEnums": [
{
@ -54,6 +55,9 @@
},
{
"name": "DatabricksPipeline"
},
{
"name": "Spline"
}
]
},
@ -94,6 +98,9 @@
},
{
"$ref": "./connections/pipeline/databricksPipelineConnection.json"
},
{
"$ref": "./connections/pipeline/splineConnection.json"
}
]
}

View File

@ -26,6 +26,11 @@
"description": "Regex exclude pipelines.",
"$ref": "../type/filterPattern.json#/definitions/filterPattern"
},
"dbServiceNames": {
"title": "Database Service Names List",
"description": "List of Database Service Names for creation of lineage",
"type": "array"
},
"markDeletedPipelines": {
"description": "Optional configuration to soft delete Pipelines in OpenMetadata if the source Pipelines are deleted. Also, if the Pipeline is deleted, all the associated entities like lineage, etc., with that Pipeline will be deleted",
"type": "boolean",

View File

@ -0,0 +1,27 @@
# Spline
In this section, we provide guides and references to use the Spline connector. You can view the full documentation for Spline [here](https://docs.open-metadata.org/connectors/pipeline/spline).
## Requirements
We extract Spline's metadata by using its [API](https://absaoss.github.io/spline/).
## Connection Details
$$section
### Spline REST Server Host & Port $(id="hostPort")
OpenMetadata uses Spline REST Server APIs to extract the execution details from spline to generate lineage.
This should be specified as a URI string in the format `scheme://hostname:port`. E.g., `http://localhost:8080`, `http://host.docker.internal:8080`.
$$
$$section
### Spline UI Host & Port $(id="uiHostPort")
Spline UI Host & Port is an optional field which is used for generating redirection URL from OpenMetadata to Spline Portal.
This should be specified as a URI string in the format `scheme://hostname:port`. E.g., `http://localhost:9090`, `http://host.docker.internal:9090`.
$$

View File

@ -21,6 +21,16 @@ For example, to exclude all Pipelines with the name containing the word `demo`,
Checkout [this](https://docs.open-metadata.org/connectors/ingestion/workflows/metadata/filter-patterns/database#database-filter-pattern) document for further examples on filter patterns.
$$
$$section
### Database Service Name $(id="dbServiceNames")
When processing Pipelines we can extract information about the inlet and the outlet tables.
In order to create the lineage between the inlet and the outlet tables, we need to know where to look for such tables.
You can enter a list of Database Services that are hosting the inlet and the outlet tables.
$$
$$section
### Include Lineage $(id="includeLineage")

Binary file not shown.

After

Width:  |  Height:  |  Size: 57 KiB

View File

@ -456,6 +456,7 @@ const AddIngestion = ({
pipelineFilterPattern,
showPipelineFilter
),
dbServiceNames: databaseServiceNames,
type: ConfigType.PipelineMetadata,
markDeletedPipelines,
includeTags,

View File

@ -429,6 +429,29 @@ const ConfigureIngestion = ({
},
};
const dbServiceNamesField: FieldProp = {
name: 'dbServiceNames',
label: t('label.database-service-name'),
type: FieldTypes.SELECT,
required: false,
id: 'root/dbServiceNames',
hasSeparator: true,
props: {
allowClear: true,
'data-testid': 'name',
mode: 'tags',
placeholder: t('label.add-entity', {
entity: t('label.database-service-name'),
}),
style: { width: '100%' },
value: databaseServiceNames,
onChange: handleDashBoardServiceNames,
},
formItemProps: {
initialValue: databaseServiceNames,
},
};
const databaseMetadataFields: FieldProp[] = [
...databaseServiceFilterPatternFields,
{
@ -569,28 +592,7 @@ const ConfigureIngestion = ({
id: 'root/dataModelFilterPattern',
hasSeparator: true,
},
{
name: 'dbServiceNames',
label: t('label.database-service-name'),
type: FieldTypes.SELECT,
required: false,
id: 'root/dbServiceNames',
hasSeparator: true,
props: {
allowClear: true,
'data-testid': 'name',
mode: 'tags',
placeholder: t('label.add-entity', {
entity: t('label.database-service-name'),
}),
style: { width: '100%' },
value: databaseServiceNames,
onChange: handleDashBoardServiceNames,
},
formItemProps: {
initialValue: databaseServiceNames,
},
},
dbServiceNamesField,
loggerLevelField,
{
name: 'includeOwners',
@ -705,6 +707,7 @@ const ConfigureIngestion = ({
id: 'root/pipelineFilterPattern',
hasSeparator: true,
},
dbServiceNamesField,
{
name: 'includeLineage',
label: t('label.include-entity', {

View File

@ -13,11 +13,13 @@
import { Badge, Button, Col, Row, Select, Space } from 'antd';
import classNames from 'classnames';
import { DatabaseServiceType } from 'generated/entity/data/table';
import { DatabaseServiceType } from 'generated/entity/data/database';
import { PipelineServiceType } from 'generated/entity/services/pipelineService';
import { startCase } from 'lodash';
import React, { useEffect, useMemo, useState } from 'react';
import { useTranslation } from 'react-i18next';
import {
BETA_SERVICES,
excludedService,
serviceTypes,
SERVICE_CATEGORY_OPTIONS,
@ -134,15 +136,15 @@ const SelectServiceType = ({
</div>
<p className="break-word text-center">
{type.includes('Custom') ? startCase(type) : type}
{type === DatabaseServiceType.Impala ? (
{BETA_SERVICES.includes(
type as DatabaseServiceType | PipelineServiceType
) ? (
<Badge
className="service-beta-tag"
color="#7147E8"
count={t('label.beta')}
/>
) : (
''
)}
) : null}
</p>
</Space>
</Col>

View File

@ -250,7 +250,9 @@ const ServiceConnectionDetails = ({
<div
className="d-flex flex-wrap p-xss"
data-testid="service-connection-details">
<Row gutter={[8, 8]}>{data}</Row>
<Row className="w-full" gutter={[8, 8]}>
{data}
</Row>
</div>
</Card>
);

View File

@ -68,6 +68,7 @@ import salesforce from '../assets/img/service-icon-salesforce.png';
import scikit from '../assets/img/service-icon-scikit.png';
import singlestore from '../assets/img/service-icon-singlestore.png';
import snowflakes from '../assets/img/service-icon-snowflakes.png';
import spline from '../assets/img/service-icon-spline.png';
import mysql from '../assets/img/service-icon-sql.png';
import sqlite from '../assets/img/service-icon-sqlite.png';
import superset from '../assets/img/service-icon-superset.png';
@ -165,6 +166,7 @@ export const SAGEMAKER = sagemaker;
export const AMAZON_S3 = amazonS3;
export const GCS = gcs;
export const MS_AZURE = msAzure;
export const SPLINE = spline;
export const PLUS = plus;
export const NOSERVICE = noService;
@ -339,3 +341,8 @@ export const SERVICE_TYPE_MAP = {
[ServiceCategory.STORAGE_SERVICES]: ServiceType.Storage,
[ServiceCategory.PIPELINE_SERVICES]: ServiceType.Pipeline,
};
export const BETA_SERVICES = [
DatabaseServiceType.Impala,
PipelineServiceType.Spline,
];

View File

@ -25,6 +25,7 @@ import domoPipelineConnection from '../jsons/connectionSchemas/connections/pipel
import fivetranConnection from '../jsons/connectionSchemas/connections/pipeline/fivetranConnection.json';
import gluePipelineConnection from '../jsons/connectionSchemas/connections/pipeline/gluePipelineConnection.json';
import nifiConnection from '../jsons/connectionSchemas/connections/pipeline/nifiConnection.json';
import splineConnection from '../jsons/connectionSchemas/connections/pipeline/splineConnection.json';
export const getPipelineConfig = (type: PipelineServiceType) => {
let schema = {};
@ -76,6 +77,11 @@ export const getPipelineConfig = (type: PipelineServiceType) => {
break;
}
case PipelineServiceType.Spline: {
schema = splineConnection;
break;
}
default:
break;

View File

@ -92,6 +92,7 @@ import {
SERVICE_TYPE_MAP,
SINGLESTORE,
SNOWFLAKE,
SPLINE,
SQLITE,
SUPERSET,
TABLEAU,
@ -261,6 +262,9 @@ export const serviceTypeLogo = (type: string) => {
case PipelineServiceType.GluePipeline:
return GLUE;
case PipelineServiceType.Spline:
return SPLINE;
case PipelineServiceType.Nifi:
return NIFI;