mirror of
https://github.com/open-metadata/OpenMetadata.git
synced 2025-08-26 09:55:52 +00:00
Issue 5592: Add Databricks Pipeline Connector (#9554)
* feat: update databricks_client * feat: create pipelineconnection & connection * test: add unit test * feat: add examples file * fix: remove unused imports * style: fix python style * fix: code review adjustments * fix: remove unneeded schema ref * fix: format requirements * fix: change context calls * test: add pipeline_status test * format: pylint format change Co-authored-by: Pere Miquel Brull <peremiquelbrull@gmail.com>
This commit is contained in:
parent
9f1f91f393
commit
3a3f248edd
@ -0,0 +1,20 @@
|
|||||||
|
source:
|
||||||
|
type: databrickspipeline
|
||||||
|
serviceName: DatabricksPipeline
|
||||||
|
serviceConnection:
|
||||||
|
config:
|
||||||
|
type: DatabricksPipeline
|
||||||
|
token: <databricks token>
|
||||||
|
hostPort: localhost:443
|
||||||
|
connectionArguments:
|
||||||
|
http_path: <http path of databricks cluster>
|
||||||
|
sourceConfig:
|
||||||
|
config:
|
||||||
|
type: PipelineMetadata
|
||||||
|
sink:
|
||||||
|
type: metadata-rest
|
||||||
|
config: {}
|
||||||
|
workflowConfig:
|
||||||
|
openMetadataServerConfig:
|
||||||
|
hostPort: http://localhost:8585/api
|
||||||
|
authProvider: no-auth
|
@ -36,10 +36,14 @@ class DatabricksClient:
|
|||||||
|
|
||||||
def __init__(self, config: DatabricksConnection):
|
def __init__(self, config: DatabricksConnection):
|
||||||
self.config = config
|
self.config = config
|
||||||
base_url, _ = self.config.hostPort.split(":")
|
base_url, *_ = self.config.hostPort.split(":")
|
||||||
api_version = "/api/2.0"
|
api_version = "/api/2.0"
|
||||||
|
job_api_version = "/api/2.1"
|
||||||
auth_token = self.config.token.get_secret_value()
|
auth_token = self.config.token.get_secret_value()
|
||||||
self.base_url = f"https://{base_url}{api_version}/sql/history/queries"
|
self.base_query_url = f"https://{base_url}{api_version}/sql/history/queries"
|
||||||
|
self.base_job_url = f"https://{base_url}{job_api_version}/jobs"
|
||||||
|
self.jobs_list_url = f"{self.base_job_url}/list"
|
||||||
|
self.jobs_run_list_url = f"{self.base_job_url}/runs/list"
|
||||||
self.headers = {
|
self.headers = {
|
||||||
"Authorization": f"Bearer {auth_token}",
|
"Authorization": f"Bearer {auth_token}",
|
||||||
"Content-Type": "application/json",
|
"Content-Type": "application/json",
|
||||||
@ -75,7 +79,7 @@ class DatabricksClient:
|
|||||||
}
|
}
|
||||||
|
|
||||||
response = self.client.get(
|
response = self.client.get(
|
||||||
self.base_url,
|
self.base_query_url,
|
||||||
data=json.dumps(data),
|
data=json.dumps(data),
|
||||||
headers=self.headers,
|
headers=self.headers,
|
||||||
timeout=10,
|
timeout=10,
|
||||||
@ -103,7 +107,7 @@ class DatabricksClient:
|
|||||||
if result[-1]["execution_end_time_ms"] <= end_time:
|
if result[-1]["execution_end_time_ms"] <= end_time:
|
||||||
|
|
||||||
response = self.client.get(
|
response = self.client.get(
|
||||||
self.base_url,
|
self.base_query_url,
|
||||||
data=json.dumps(data),
|
data=json.dumps(data),
|
||||||
headers=self.headers,
|
headers=self.headers,
|
||||||
timeout=10,
|
timeout=10,
|
||||||
@ -122,3 +126,81 @@ class DatabricksClient:
|
|||||||
query_text.startswith(QUERY_WITH_DBT)
|
query_text.startswith(QUERY_WITH_DBT)
|
||||||
or query_text.startswith(QUERY_WITH_OM_VERSION)
|
or query_text.startswith(QUERY_WITH_OM_VERSION)
|
||||||
)
|
)
|
||||||
|
|
||||||
|
def list_jobs(self) -> List[dict]:
|
||||||
|
"""
|
||||||
|
Method returns List all the created jobs in a Databricks Workspace
|
||||||
|
"""
|
||||||
|
job_list = []
|
||||||
|
try:
|
||||||
|
data = {"limit": 25, "expand_tasks": True, "offset": 0}
|
||||||
|
|
||||||
|
response = self.client.get(
|
||||||
|
self.jobs_list_url,
|
||||||
|
data=json.dumps(data),
|
||||||
|
headers=self.headers,
|
||||||
|
timeout=10,
|
||||||
|
).json()
|
||||||
|
|
||||||
|
job_list.extend(response["jobs"])
|
||||||
|
|
||||||
|
while response["has_more"]:
|
||||||
|
|
||||||
|
data["offset"] = len(response["jobs"])
|
||||||
|
|
||||||
|
response = self.client.get(
|
||||||
|
self.jobs_list_url,
|
||||||
|
data=json.dumps(data),
|
||||||
|
headers=self.headers,
|
||||||
|
timeout=10,
|
||||||
|
).json()
|
||||||
|
|
||||||
|
job_list.extend(response["jobs"])
|
||||||
|
|
||||||
|
except Exception as exc:
|
||||||
|
logger.debug(traceback.format_exc())
|
||||||
|
logger.error(exc)
|
||||||
|
|
||||||
|
return job_list
|
||||||
|
|
||||||
|
def get_job_runs(self, job_id) -> List[dict]:
|
||||||
|
"""
|
||||||
|
Method returns List of all runs for a job by the specified job_id
|
||||||
|
"""
|
||||||
|
job_runs = []
|
||||||
|
try:
|
||||||
|
params = {
|
||||||
|
"job_id": job_id,
|
||||||
|
"active_only": "false",
|
||||||
|
"completed_only": "true",
|
||||||
|
"run_type": "JOB_RUN",
|
||||||
|
"expand_tasks": "true",
|
||||||
|
}
|
||||||
|
|
||||||
|
response = self.client.get(
|
||||||
|
self.jobs_run_list_url,
|
||||||
|
params=params,
|
||||||
|
headers=self.headers,
|
||||||
|
timeout=10,
|
||||||
|
).json()
|
||||||
|
|
||||||
|
job_runs.extend(response["runs"])
|
||||||
|
|
||||||
|
while response["has_more"]:
|
||||||
|
|
||||||
|
params.update({"start_time_to": response["runs"][-1]["start_time"]})
|
||||||
|
|
||||||
|
response = self.client.get(
|
||||||
|
self.jobs_run_list_url,
|
||||||
|
params=params,
|
||||||
|
headers=self.headers,
|
||||||
|
timeout=10,
|
||||||
|
).json()
|
||||||
|
|
||||||
|
job_runs.extend(response["runs"])
|
||||||
|
|
||||||
|
except Exception as exc:
|
||||||
|
logger.debug(traceback.format_exc())
|
||||||
|
logger.error(exc)
|
||||||
|
|
||||||
|
return job_runs
|
||||||
|
@ -0,0 +1,37 @@
|
|||||||
|
# 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 metadata.generated.schema.entity.services.connections.pipeline.databricksPipelineConnection import (
|
||||||
|
DatabricksPipelineConnection,
|
||||||
|
)
|
||||||
|
from metadata.ingestion.connections.test_connections import SourceConnectionException
|
||||||
|
from metadata.ingestion.source.database.databricks.client import DatabricksClient
|
||||||
|
|
||||||
|
|
||||||
|
def get_connection(connection: DatabricksPipelineConnection) -> DatabricksClient:
|
||||||
|
"""
|
||||||
|
Create connection
|
||||||
|
"""
|
||||||
|
return DatabricksClient(connection)
|
||||||
|
|
||||||
|
|
||||||
|
def test_connection(client: DatabricksClient) -> None:
|
||||||
|
"""
|
||||||
|
Test connection
|
||||||
|
"""
|
||||||
|
try:
|
||||||
|
client.list_jobs()
|
||||||
|
except Exception as exc:
|
||||||
|
msg = f"Unknown error connecting with {client}: {exc}."
|
||||||
|
raise SourceConnectionException(msg) from exc
|
@ -0,0 +1,231 @@
|
|||||||
|
# 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.
|
||||||
|
|
||||||
|
"""
|
||||||
|
Databricks pipeline source to extract metadata
|
||||||
|
"""
|
||||||
|
|
||||||
|
import traceback
|
||||||
|
from typing import Any, Iterable, List, Optional
|
||||||
|
|
||||||
|
from pydantic import ValidationError
|
||||||
|
|
||||||
|
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 (
|
||||||
|
PipelineStatus,
|
||||||
|
StatusType,
|
||||||
|
Task,
|
||||||
|
TaskStatus,
|
||||||
|
)
|
||||||
|
from metadata.generated.schema.entity.services.connections.metadata.openMetadataConnection import (
|
||||||
|
OpenMetadataConnection,
|
||||||
|
)
|
||||||
|
from metadata.generated.schema.entity.services.connections.pipeline.databricksPipelineConnection import (
|
||||||
|
DatabricksPipelineConnection,
|
||||||
|
)
|
||||||
|
from metadata.generated.schema.metadataIngestion.workflow import (
|
||||||
|
Source as WorkflowSource,
|
||||||
|
)
|
||||||
|
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.database.databricks.client import DatabricksClient
|
||||||
|
from metadata.ingestion.source.pipeline.pipeline_service import PipelineServiceSource
|
||||||
|
from metadata.utils.logger import ingestion_logger
|
||||||
|
|
||||||
|
logger = ingestion_logger()
|
||||||
|
|
||||||
|
|
||||||
|
STATUS_MAP = {
|
||||||
|
"SUCCESS": StatusType.Successful,
|
||||||
|
"FAILED": StatusType.Failed,
|
||||||
|
"TIMEOUT": StatusType.Failed,
|
||||||
|
"CANCELED": StatusType.Failed,
|
||||||
|
"PENDING": StatusType.Pending,
|
||||||
|
"RUNNING": StatusType.Pending,
|
||||||
|
"TERMINATING": StatusType.Pending,
|
||||||
|
"SKIPPED": StatusType.Failed,
|
||||||
|
"INTERNAL_ERROR": StatusType.Failed,
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
class DatabrickspipelineSource(PipelineServiceSource):
|
||||||
|
"""
|
||||||
|
Implements the necessary methods ot extract
|
||||||
|
Pipeline metadata from Databricks Jobs API
|
||||||
|
"""
|
||||||
|
|
||||||
|
def __init__(self, config: WorkflowSource, metadata_config: OpenMetadataConnection):
|
||||||
|
super().__init__(config, metadata_config)
|
||||||
|
self.connection = self.config.serviceConnection.__root__.config
|
||||||
|
self.client = DatabricksClient(self.connection)
|
||||||
|
|
||||||
|
@classmethod
|
||||||
|
def create(cls, config_dict, metadata_config: OpenMetadataConnection):
|
||||||
|
"""Create class instance"""
|
||||||
|
config: WorkflowSource = WorkflowSource.parse_obj(config_dict)
|
||||||
|
connection: DatabricksPipelineConnection = (
|
||||||
|
config.serviceConnection.__root__.config
|
||||||
|
)
|
||||||
|
if not isinstance(connection, DatabricksPipelineConnection):
|
||||||
|
raise InvalidSourceException(
|
||||||
|
f"Expected DatabricksPipelineConnection, but got {connection}"
|
||||||
|
)
|
||||||
|
return cls(config, metadata_config)
|
||||||
|
|
||||||
|
def get_pipelines_list(self) -> Iterable[dict]:
|
||||||
|
"""
|
||||||
|
Get List of all pipelines
|
||||||
|
"""
|
||||||
|
for workflow in self.client.list_jobs():
|
||||||
|
yield workflow
|
||||||
|
|
||||||
|
def get_pipeline_name(self, pipeline_details: dict) -> str:
|
||||||
|
"""
|
||||||
|
Get Pipeline Name
|
||||||
|
"""
|
||||||
|
return pipeline_details["settings"]["name"]
|
||||||
|
|
||||||
|
def yield_pipeline(self, pipeline_details: Any) -> Iterable[CreatePipelineRequest]:
|
||||||
|
"""
|
||||||
|
Method to Get Pipeline Entity
|
||||||
|
"""
|
||||||
|
self.context.job_id_list = []
|
||||||
|
try:
|
||||||
|
yield CreatePipelineRequest(
|
||||||
|
name=pipeline_details["job_id"],
|
||||||
|
displayName=pipeline_details["settings"]["name"],
|
||||||
|
description=pipeline_details["settings"]["name"],
|
||||||
|
tasks=self.get_tasks(pipeline_details),
|
||||||
|
service=EntityReference(
|
||||||
|
id=self.context.pipeline_service.id.__root__, type="pipelineService"
|
||||||
|
),
|
||||||
|
)
|
||||||
|
|
||||||
|
except TypeError as err:
|
||||||
|
logger.debug(traceback.format_exc())
|
||||||
|
logger.warning(
|
||||||
|
f"Error building Databricks Pipeline information from {pipeline_details}."
|
||||||
|
f" There might be Databricks Jobs API version incompatibilities - {err}"
|
||||||
|
)
|
||||||
|
except ValidationError as err:
|
||||||
|
logger.debug(traceback.format_exc())
|
||||||
|
logger.warning(
|
||||||
|
f"Error building pydantic model for {pipeline_details} - {err}"
|
||||||
|
)
|
||||||
|
except Exception as err:
|
||||||
|
logger.debug(traceback.format_exc())
|
||||||
|
logger.warning(f"Wild error ingesting pipeline {pipeline_details} - {err}")
|
||||||
|
|
||||||
|
def get_tasks(self, pipeline_details: dict) -> List[Task]:
|
||||||
|
task_list = []
|
||||||
|
self.append_context(key="job_id_list", value=pipeline_details["job_id"])
|
||||||
|
|
||||||
|
downstream_tasks = self.get_downstream_tasks(
|
||||||
|
pipeline_details["settings"]["tasks"]
|
||||||
|
)
|
||||||
|
for task in pipeline_details["settings"]["tasks"]:
|
||||||
|
task_list.append(
|
||||||
|
Task(
|
||||||
|
name=task["task_key"],
|
||||||
|
displayName=task["task_key"],
|
||||||
|
taskType=self.get_task_type(task),
|
||||||
|
downstreamTasks=downstream_tasks.get(task["task_key"], []),
|
||||||
|
)
|
||||||
|
)
|
||||||
|
|
||||||
|
return task_list
|
||||||
|
|
||||||
|
def get_task_type(self, task):
|
||||||
|
task_key = "undefined_task_type"
|
||||||
|
for key in task.keys():
|
||||||
|
if key.endswith("_task"):
|
||||||
|
task_key = key
|
||||||
|
|
||||||
|
return task_key
|
||||||
|
|
||||||
|
def get_downstream_tasks(self, workflow):
|
||||||
|
task_key_list = [task["task_key"] for task in workflow]
|
||||||
|
|
||||||
|
dependent_tasks = self.get_dependent_tasks(workflow)
|
||||||
|
|
||||||
|
downstream_tasks = {}
|
||||||
|
|
||||||
|
for task_key, task_depend_ons in dependent_tasks.items():
|
||||||
|
if task_depend_ons:
|
||||||
|
for task in task_depend_ons:
|
||||||
|
if task in downstream_tasks:
|
||||||
|
downstream_tasks[task].append(task_key)
|
||||||
|
else:
|
||||||
|
downstream_tasks[task] = [task_key]
|
||||||
|
|
||||||
|
for task in task_key_list:
|
||||||
|
if task not in downstream_tasks:
|
||||||
|
downstream_tasks[task] = []
|
||||||
|
|
||||||
|
return downstream_tasks
|
||||||
|
|
||||||
|
def get_dependent_tasks(self, workflow):
|
||||||
|
dependent_tasks = {}
|
||||||
|
|
||||||
|
for task in workflow:
|
||||||
|
depends_on = task.get("depends_on")
|
||||||
|
if depends_on:
|
||||||
|
dependent_tasks[task["task_key"]] = [v["task_key"] for v in depends_on]
|
||||||
|
else:
|
||||||
|
dependent_tasks[task["task_key"]] = None
|
||||||
|
|
||||||
|
return dependent_tasks
|
||||||
|
|
||||||
|
def yield_pipeline_status(self, pipeline_details) -> Iterable[OMetaPipelineStatus]:
|
||||||
|
|
||||||
|
for job_id in self.context.job_id_list:
|
||||||
|
try:
|
||||||
|
runs = self.client.get_job_runs(job_id=job_id)
|
||||||
|
for attempt in runs:
|
||||||
|
for task_run in attempt["tasks"]:
|
||||||
|
task_status = []
|
||||||
|
task_status.append(
|
||||||
|
TaskStatus(
|
||||||
|
name=task_run["task_key"],
|
||||||
|
executionStatus=STATUS_MAP.get(
|
||||||
|
task_run["state"].get("result_state"),
|
||||||
|
StatusType.Failed,
|
||||||
|
),
|
||||||
|
startTime=task_run["start_time"],
|
||||||
|
endTime=task_run["end_time"],
|
||||||
|
logLink=task_run["run_page_url"],
|
||||||
|
)
|
||||||
|
)
|
||||||
|
pipeline_status = PipelineStatus(
|
||||||
|
taskStatus=task_status,
|
||||||
|
timestamp=attempt["start_time"],
|
||||||
|
executionStatus=STATUS_MAP.get(
|
||||||
|
attempt["state"].get("result_state"),
|
||||||
|
StatusType.Failed,
|
||||||
|
),
|
||||||
|
)
|
||||||
|
yield OMetaPipelineStatus(
|
||||||
|
pipeline_fqn=self.context.pipeline.fullyQualifiedName.__root__,
|
||||||
|
pipeline_status=pipeline_status,
|
||||||
|
)
|
||||||
|
except Exception as exc:
|
||||||
|
logger.debug(traceback.format_exc())
|
||||||
|
logger.error(f"Failed to yield pipeline status: {exc}")
|
||||||
|
|
||||||
|
def yield_pipeline_lineage_details(
|
||||||
|
self, pipeline_details: Any
|
||||||
|
) -> Optional[Iterable[AddLineageRequest]]:
|
||||||
|
"""
|
||||||
|
Get lineage between pipeline and data sources
|
||||||
|
"""
|
||||||
|
logger.info("Lineage is not yet supported on Databicks Pipelines")
|
@ -0,0 +1,174 @@
|
|||||||
|
[
|
||||||
|
{
|
||||||
|
"job_id": 325697581681107,
|
||||||
|
"run_id": 820960,
|
||||||
|
"creator_user_name": "user_name",
|
||||||
|
"number_in_job": 820960,
|
||||||
|
"original_attempt_run_id": 820960,
|
||||||
|
"state": {
|
||||||
|
"life_cycle_state": "TERMINATED",
|
||||||
|
"result_state": "SUCCESS",
|
||||||
|
"state_message": "",
|
||||||
|
"user_cancelled_or_timedout": false
|
||||||
|
},
|
||||||
|
"schedule": {
|
||||||
|
"quartz_cron_expression": "30 1/2 * * * ?",
|
||||||
|
"timezone_id": "UTC",
|
||||||
|
"pause_status": "UNPAUSED"
|
||||||
|
},
|
||||||
|
"start_time": 1672691730552,
|
||||||
|
"setup_duration": 51000,
|
||||||
|
"execution_duration": 11000,
|
||||||
|
"cleanup_duration": 0,
|
||||||
|
"end_time": 1672691793780,
|
||||||
|
"trigger": "PERIODIC",
|
||||||
|
"run_name": "OpenMetadata Databricks Workflow",
|
||||||
|
"run_page_url": "https://workspace.azuredatabricks.net/?o=workspace_id#job/325697581681107/run/820960",
|
||||||
|
"run_type": "JOB_RUN",
|
||||||
|
"tasks": [
|
||||||
|
{
|
||||||
|
"run_id": 821029,
|
||||||
|
"task_key": "one_task",
|
||||||
|
"notebook_task": {
|
||||||
|
"notebook_path": "/Users/user_name/notebook",
|
||||||
|
"source": "WORKSPACE"
|
||||||
|
},
|
||||||
|
"job_cluster_key": "Shared_job_cluster",
|
||||||
|
"state": {
|
||||||
|
"life_cycle_state": "TERMINATED",
|
||||||
|
"result_state": "SUCCESS",
|
||||||
|
"state_message": "",
|
||||||
|
"user_cancelled_or_timedout": false
|
||||||
|
},
|
||||||
|
"run_page_url": "https://workspace.azuredatabricks.net/?o=workspace_id#job/325697581681107/run/821029",
|
||||||
|
"start_time": 1672691730568,
|
||||||
|
"setup_duration": 51000,
|
||||||
|
"execution_duration": 11000,
|
||||||
|
"cleanup_duration": 0,
|
||||||
|
"end_time": 1672691793559,
|
||||||
|
"cluster_instance": {
|
||||||
|
"cluster_id": "0102-203531-uh2t6efy",
|
||||||
|
"spark_context_id": "3018173046193599096"
|
||||||
|
},
|
||||||
|
"attempt_number": 0
|
||||||
|
}
|
||||||
|
],
|
||||||
|
"job_clusters": [
|
||||||
|
{
|
||||||
|
"job_cluster_key": "Shared_job_cluster",
|
||||||
|
"new_cluster": {
|
||||||
|
"cluster_name": "",
|
||||||
|
"spark_version": "11.3.x-scala2.12",
|
||||||
|
"spark_conf": {
|
||||||
|
"spark.databricks.delta.preview.enabled": "true",
|
||||||
|
"spark.master": "local[*, 4]",
|
||||||
|
"spark.databricks.cluster.profile": "singleNode"
|
||||||
|
},
|
||||||
|
"azure_attributes": {
|
||||||
|
"first_on_demand": 1,
|
||||||
|
"availability": "ON_DEMAND_AZURE",
|
||||||
|
"spot_bid_max_price": -1.0
|
||||||
|
},
|
||||||
|
"node_type_id": "Standard_DS3_v2",
|
||||||
|
"custom_tags": {
|
||||||
|
"ResourceClass": "SingleNode"
|
||||||
|
},
|
||||||
|
"spark_env_vars": {
|
||||||
|
"PYSPARK_PYTHON": "/databricks/python3/bin/python3"
|
||||||
|
},
|
||||||
|
"enable_elastic_disk": true,
|
||||||
|
"data_security_mode": "SINGLE_USER",
|
||||||
|
"runtime_engine": "STANDARD",
|
||||||
|
"num_workers": 0
|
||||||
|
}
|
||||||
|
}
|
||||||
|
],
|
||||||
|
"format": "MULTI_TASK"
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"job_id": 325697581681107,
|
||||||
|
"run_id": 820788,
|
||||||
|
"creator_user_name": "user_name",
|
||||||
|
"number_in_job": 820788,
|
||||||
|
"original_attempt_run_id": 820788,
|
||||||
|
"state": {
|
||||||
|
"life_cycle_state": "TERMINATED",
|
||||||
|
"result_state": "FAILED ",
|
||||||
|
"state_message": "",
|
||||||
|
"user_cancelled_or_timedout": false
|
||||||
|
},
|
||||||
|
"schedule": {
|
||||||
|
"quartz_cron_expression": "30 1/2 * * * ?",
|
||||||
|
"timezone_id": "UTC",
|
||||||
|
"pause_status": "UNPAUSED"
|
||||||
|
},
|
||||||
|
"start_time": 1672691610525,
|
||||||
|
"setup_duration": 55000,
|
||||||
|
"execution_duration": 12000,
|
||||||
|
"cleanup_duration": 0,
|
||||||
|
"end_time": 1672691677852,
|
||||||
|
"trigger": "PERIODIC",
|
||||||
|
"run_name": "OpenMetadata Databricks Workflow",
|
||||||
|
"run_page_url": "https://workspace.azuredatabricks.net/?o=workspace_id#job/325697581681107/run/820788",
|
||||||
|
"run_type": "JOB_RUN",
|
||||||
|
"tasks": [
|
||||||
|
{
|
||||||
|
"run_id": 820956,
|
||||||
|
"task_key": "one_task",
|
||||||
|
"notebook_task": {
|
||||||
|
"notebook_path": "/Users/user_name/notebook",
|
||||||
|
"source": "WORKSPACE"
|
||||||
|
},
|
||||||
|
"job_cluster_key": "Shared_job_cluster",
|
||||||
|
"state": {
|
||||||
|
"life_cycle_state": "TERMINATED",
|
||||||
|
"result_state": "FAILED",
|
||||||
|
"state_message": "",
|
||||||
|
"user_cancelled_or_timedout": false
|
||||||
|
},
|
||||||
|
"run_page_url": "https://workspace.azuredatabricks.net/?o=workspace_id#job/325697581681107/run/820956",
|
||||||
|
"start_time": 1672691610544,
|
||||||
|
"setup_duration": 55000,
|
||||||
|
"execution_duration": 12000,
|
||||||
|
"cleanup_duration": 0,
|
||||||
|
"end_time": 1672691677696,
|
||||||
|
"cluster_instance": {
|
||||||
|
"cluster_id": "0102-203335-kfts9a0p",
|
||||||
|
"spark_context_id": "3081931546313825347"
|
||||||
|
},
|
||||||
|
"attempt_number": 0
|
||||||
|
}
|
||||||
|
],
|
||||||
|
"job_clusters": [
|
||||||
|
{
|
||||||
|
"job_cluster_key": "Shared_job_cluster",
|
||||||
|
"new_cluster": {
|
||||||
|
"cluster_name": "",
|
||||||
|
"spark_version": "11.3.x-scala2.12",
|
||||||
|
"spark_conf": {
|
||||||
|
"spark.databricks.delta.preview.enabled": "true",
|
||||||
|
"spark.master": "local[*, 4]",
|
||||||
|
"spark.databricks.cluster.profile": "singleNode"
|
||||||
|
},
|
||||||
|
"azure_attributes": {
|
||||||
|
"first_on_demand": 1,
|
||||||
|
"availability": "ON_DEMAND_AZURE",
|
||||||
|
"spot_bid_max_price": -1.0
|
||||||
|
},
|
||||||
|
"node_type_id": "Standard_DS3_v2",
|
||||||
|
"custom_tags": {
|
||||||
|
"ResourceClass": "SingleNode"
|
||||||
|
},
|
||||||
|
"spark_env_vars": {
|
||||||
|
"PYSPARK_PYTHON": "/databricks/python3/bin/python3"
|
||||||
|
},
|
||||||
|
"enable_elastic_disk": true,
|
||||||
|
"data_security_mode": "SINGLE_USER",
|
||||||
|
"runtime_engine": "STANDARD",
|
||||||
|
"num_workers": 0
|
||||||
|
}
|
||||||
|
}
|
||||||
|
],
|
||||||
|
"format": "MULTI_TASK"
|
||||||
|
}
|
||||||
|
]
|
@ -0,0 +1,120 @@
|
|||||||
|
[
|
||||||
|
{
|
||||||
|
"job_id": 606358633757175,
|
||||||
|
"creator_user_name": "user_name",
|
||||||
|
"settings": {
|
||||||
|
"name": "OpenMetadata Databricks Workflow",
|
||||||
|
"email_notifications": {
|
||||||
|
"no_alert_for_skipped_runs": false
|
||||||
|
},
|
||||||
|
"timeout_seconds": 0,
|
||||||
|
"schedule": {
|
||||||
|
"quartz_cron_expression": "16 1/2 * * * ?",
|
||||||
|
"timezone_id": "UTC",
|
||||||
|
"pause_status": "PAUSED"
|
||||||
|
},
|
||||||
|
"max_concurrent_runs": 3,
|
||||||
|
"tasks": [
|
||||||
|
{
|
||||||
|
"task_key": "task_1",
|
||||||
|
"notebook_task": {
|
||||||
|
"notebook_path": "/Users/user_name/notebook",
|
||||||
|
"source": "WORKSPACE"
|
||||||
|
},
|
||||||
|
"job_cluster_key": "cluster_name",
|
||||||
|
"timeout_seconds": 0,
|
||||||
|
"email_notifications": {}
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"task_key": "task_2",
|
||||||
|
"depends_on": [
|
||||||
|
{
|
||||||
|
"task_key": "task_1"
|
||||||
|
}
|
||||||
|
],
|
||||||
|
"spark_python_task": {
|
||||||
|
"python_file": "dbfs:/user/python_script"
|
||||||
|
},
|
||||||
|
"job_cluster_key": "cluster_name",
|
||||||
|
"timeout_seconds": 0,
|
||||||
|
"email_notifications": {}
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"task_key": "task_3",
|
||||||
|
"depends_on": [
|
||||||
|
{
|
||||||
|
"task_key": "task_1"
|
||||||
|
}
|
||||||
|
],
|
||||||
|
"python_wheel_task": {
|
||||||
|
"package_name": "my_package",
|
||||||
|
"entry_point": "run_job"
|
||||||
|
},
|
||||||
|
"job_cluster_key": "cluster_name",
|
||||||
|
"timeout_seconds": 0,
|
||||||
|
"email_notifications": {}
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"task_key": "task_4",
|
||||||
|
"depends_on": [
|
||||||
|
{
|
||||||
|
"task_key": "task_1"
|
||||||
|
}
|
||||||
|
],
|
||||||
|
"pipeline_task": {
|
||||||
|
"pipeline_id": "4068beca-753c-46ed-9a64-370108ffcd8f",
|
||||||
|
"full_refresh": false
|
||||||
|
},
|
||||||
|
"timeout_seconds": 0,
|
||||||
|
"email_notifications": {}
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"task_key": "task_5",
|
||||||
|
"depends_on": [
|
||||||
|
{
|
||||||
|
"task_key": "task_3"
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"task_key": "task_4"
|
||||||
|
}
|
||||||
|
],
|
||||||
|
"sql_task": {
|
||||||
|
"query": {
|
||||||
|
"query_id": "032a8806-bd62-4594-b222-43a973290210"
|
||||||
|
},
|
||||||
|
"warehouse_id": "21e75a911f658d1b"
|
||||||
|
},
|
||||||
|
"timeout_seconds": 0,
|
||||||
|
"email_notifications": {}
|
||||||
|
}
|
||||||
|
],
|
||||||
|
"job_clusters": [
|
||||||
|
{
|
||||||
|
"job_cluster_key": "cluster_name",
|
||||||
|
"new_cluster": {
|
||||||
|
"cluster_name": "",
|
||||||
|
"spark_version": "11.3.x-scala2.12",
|
||||||
|
"spark_conf": {
|
||||||
|
"spark.databricks.delta.preview.enabled": "true"
|
||||||
|
},
|
||||||
|
"azure_attributes": {
|
||||||
|
"first_on_demand": 1,
|
||||||
|
"availability": "ON_DEMAND_AZURE",
|
||||||
|
"spot_bid_max_price": -1.0
|
||||||
|
},
|
||||||
|
"node_type_id": "Standard_DS3_v2",
|
||||||
|
"spark_env_vars": {
|
||||||
|
"PYSPARK_PYTHON": "/databricks/python3/bin/python3"
|
||||||
|
},
|
||||||
|
"enable_elastic_disk": true,
|
||||||
|
"data_security_mode": "SINGLE_USER",
|
||||||
|
"runtime_engine": "STANDARD",
|
||||||
|
"num_workers": 8
|
||||||
|
}
|
||||||
|
}
|
||||||
|
],
|
||||||
|
"format": "MULTI_TASK"
|
||||||
|
},
|
||||||
|
"created_time": 1663513966931
|
||||||
|
}
|
||||||
|
]
|
@ -0,0 +1,265 @@
|
|||||||
|
# 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.
|
||||||
|
"""
|
||||||
|
Databricks Pipeline utils tests
|
||||||
|
"""
|
||||||
|
|
||||||
|
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,
|
||||||
|
PipelineStatus,
|
||||||
|
StatusType,
|
||||||
|
Task,
|
||||||
|
TaskStatus,
|
||||||
|
)
|
||||||
|
from metadata.generated.schema.entity.services.pipelineService import (
|
||||||
|
PipelineConnection,
|
||||||
|
PipelineService,
|
||||||
|
PipelineServiceType,
|
||||||
|
)
|
||||||
|
from metadata.generated.schema.metadataIngestion.workflow import (
|
||||||
|
OpenMetadataWorkflowConfig,
|
||||||
|
)
|
||||||
|
from metadata.generated.schema.type.entityReference import EntityReference
|
||||||
|
from metadata.ingestion.models.pipeline_status import OMetaPipelineStatus
|
||||||
|
from metadata.ingestion.source.pipeline.databrickspipeline.metadata import (
|
||||||
|
DatabrickspipelineSource,
|
||||||
|
)
|
||||||
|
from metadata.utils.ansi import print_ansi_encoded_string
|
||||||
|
|
||||||
|
mock_file_path = (
|
||||||
|
Path(__file__).parent.parent.parent
|
||||||
|
/ "resources/datasets/databricks_pipeline_resource.json"
|
||||||
|
)
|
||||||
|
with open(mock_file_path) as file:
|
||||||
|
mock_data: dict = json.load(file)
|
||||||
|
|
||||||
|
mock_file_path = (
|
||||||
|
Path(__file__).parent.parent.parent
|
||||||
|
/ "resources/datasets/databricks_pipeline_history.json"
|
||||||
|
)
|
||||||
|
with open(mock_file_path) as file:
|
||||||
|
mock_history_data: dict = json.load(file)
|
||||||
|
|
||||||
|
|
||||||
|
mock_databricks_config = {
|
||||||
|
"source": {
|
||||||
|
"type": "DatabricksPipeline",
|
||||||
|
"serviceName": "DatabricksPipeline",
|
||||||
|
"serviceConnection": {
|
||||||
|
"config": {
|
||||||
|
"type": "DatabricksPipeline",
|
||||||
|
"token": "random_token",
|
||||||
|
"hostPort": "localhost:443",
|
||||||
|
"connectionArguments": {
|
||||||
|
"http_path": "sql/1.0/endpoints/path",
|
||||||
|
},
|
||||||
|
}
|
||||||
|
},
|
||||||
|
"sourceConfig": {"config": {"type": "PipelineMetadata"}},
|
||||||
|
},
|
||||||
|
"sink": {"type": "metadata-rest", "config": {}},
|
||||||
|
"workflowConfig": {
|
||||||
|
"openMetadataServerConfig": {
|
||||||
|
"hostPort": "http://localhost:8585/api",
|
||||||
|
"authProvider": "openmetadata",
|
||||||
|
"securityConfig": {
|
||||||
|
"jwtToken": "eyJraWQiOiJHYjM4OWEtOWY3Ni1nZGpzLWE5MmotMDI0MmJrOTQzNTYiLCJ0eXAiOiJKV1QiLCJhbGc"
|
||||||
|
"iOiJSUzI1NiJ9.eyJzdWIiOiJhZG1pbiIsImlzQm90IjpmYWxzZSwiaXNzIjoib3Blbi1tZXRhZGF0YS5vcmciLCJpYXQiOjE"
|
||||||
|
"2NjM5Mzg0NjIsImVtYWlsIjoiYWRtaW5Ab3Blbm1ldGFkYXRhLm9yZyJ9.tS8um_5DKu7HgzGBzS1VTA5uUjKWOCU0B_j08WXB"
|
||||||
|
"iEC0mr0zNREkqVfwFDD-d24HlNEbrqioLsBuFRiwIWKc1m_ZlVQbG7P36RUxhuv2vbSp80FKyNM-Tj93FDzq91jsyNmsQhyNv_fN"
|
||||||
|
"r3TXfzzSPjHt8Go0FMMP66weoKMgW2PbXlhVKwEuXUHyakLLzewm9UMeQaEiRzhiTMU3UkLXcKbYEJJvfNFcLwSl9W8JCO_l0Yj3u"
|
||||||
|
"d-qt_nQYEZwqW6u5nfdQllN133iikV4fM5QZsMCnm8Rq1mvLR0y9bmJiD7fwM1tmJ791TUWqmKaTnP49U493VanKpUAfzIiOiIbhg"
|
||||||
|
},
|
||||||
|
}
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
MOCK_PIPELINE_SERVICE = PipelineService(
|
||||||
|
id="85811038-099a-11ed-861d-0242ac120002",
|
||||||
|
name="databricks_pipeline_test",
|
||||||
|
connection=PipelineConnection(),
|
||||||
|
serviceType=PipelineServiceType.DatabricksPipeline,
|
||||||
|
)
|
||||||
|
|
||||||
|
MOCK_PIPELINE = Pipeline(
|
||||||
|
id="2aaa012e-099a-11ed-861d-0242ac120002",
|
||||||
|
name="606358633757175",
|
||||||
|
fullyQualifiedName="databricks_pipeline_source.606358633757175",
|
||||||
|
displayName="OpenMetadata Databricks Workflow",
|
||||||
|
tasks=[
|
||||||
|
Task(
|
||||||
|
name="task_1",
|
||||||
|
displayName="task_1",
|
||||||
|
taskType="notebook_task",
|
||||||
|
downstreamTasks=["task_2", "task_3", "task_4"],
|
||||||
|
),
|
||||||
|
Task(
|
||||||
|
name="task_2",
|
||||||
|
displayName="task_2",
|
||||||
|
taskType="spark_python_task",
|
||||||
|
downstreamTasks=[],
|
||||||
|
),
|
||||||
|
Task(
|
||||||
|
name="task_3",
|
||||||
|
displayName="task_3",
|
||||||
|
taskType="python_wheel_task",
|
||||||
|
downstreamTasks=["task_5"],
|
||||||
|
),
|
||||||
|
Task(
|
||||||
|
name="task_4",
|
||||||
|
displayName="task_4",
|
||||||
|
taskType="pipeline_task",
|
||||||
|
downstreamTasks=["task_5"],
|
||||||
|
),
|
||||||
|
Task(
|
||||||
|
name="task_5",
|
||||||
|
displayName="task_5",
|
||||||
|
taskType="sql_task",
|
||||||
|
downstreamTasks=[],
|
||||||
|
),
|
||||||
|
],
|
||||||
|
service=EntityReference(
|
||||||
|
id="85811038-099a-11ed-861d-0242ac120002", type="pipelineService"
|
||||||
|
),
|
||||||
|
)
|
||||||
|
|
||||||
|
EXPECTED_CREATED_PIPELINES = CreatePipelineRequest(
|
||||||
|
name="606358633757175",
|
||||||
|
displayName="OpenMetadata Databricks Workflow",
|
||||||
|
description="OpenMetadata Databricks Workflow",
|
||||||
|
tasks=[
|
||||||
|
Task(
|
||||||
|
name="task_1",
|
||||||
|
displayName="task_1",
|
||||||
|
taskType="notebook_task",
|
||||||
|
downstreamTasks=["task_2", "task_3", "task_4"],
|
||||||
|
),
|
||||||
|
Task(
|
||||||
|
name="task_2",
|
||||||
|
displayName="task_2",
|
||||||
|
taskType="spark_python_task",
|
||||||
|
downstreamTasks=[],
|
||||||
|
),
|
||||||
|
Task(
|
||||||
|
name="task_3",
|
||||||
|
displayName="task_3",
|
||||||
|
taskType="python_wheel_task",
|
||||||
|
downstreamTasks=["task_5"],
|
||||||
|
),
|
||||||
|
Task(
|
||||||
|
name="task_4",
|
||||||
|
displayName="task_4",
|
||||||
|
taskType="pipeline_task",
|
||||||
|
downstreamTasks=["task_5"],
|
||||||
|
),
|
||||||
|
Task(
|
||||||
|
name="task_5",
|
||||||
|
displayName="task_5",
|
||||||
|
taskType="sql_task",
|
||||||
|
downstreamTasks=[],
|
||||||
|
),
|
||||||
|
],
|
||||||
|
service=EntityReference(
|
||||||
|
id="85811038-099a-11ed-861d-0242ac120002", type="pipelineService"
|
||||||
|
),
|
||||||
|
)
|
||||||
|
|
||||||
|
|
||||||
|
EXPECTED_PIPELINE_STATUS = [
|
||||||
|
OMetaPipelineStatus(
|
||||||
|
pipeline_fqn="databricks_pipeline_source.606358633757175",
|
||||||
|
pipeline_status=PipelineStatus(
|
||||||
|
executionStatus=StatusType.Successful.value,
|
||||||
|
taskStatus=[
|
||||||
|
TaskStatus(
|
||||||
|
name="one_task",
|
||||||
|
executionStatus=StatusType.Successful.value,
|
||||||
|
startTime=1672691730568,
|
||||||
|
endTime=1672691793559,
|
||||||
|
logLink="https://workspace.azuredatabricks.net/?o=workspace_id#job/325697581681107/run/821029",
|
||||||
|
)
|
||||||
|
],
|
||||||
|
timestamp=1672691730552,
|
||||||
|
),
|
||||||
|
),
|
||||||
|
OMetaPipelineStatus(
|
||||||
|
pipeline_fqn="databricks_pipeline_source.606358633757175",
|
||||||
|
pipeline_status=PipelineStatus(
|
||||||
|
executionStatus=StatusType.Failed.value,
|
||||||
|
taskStatus=[
|
||||||
|
TaskStatus(
|
||||||
|
name="one_task",
|
||||||
|
executionStatus=StatusType.Failed.value,
|
||||||
|
startTime=1672691610544,
|
||||||
|
endTime=1672691677696,
|
||||||
|
logLink="https://workspace.azuredatabricks.net/?o=workspace_id#job/325697581681107/run/820956",
|
||||||
|
)
|
||||||
|
],
|
||||||
|
timestamp=1672691610525,
|
||||||
|
),
|
||||||
|
),
|
||||||
|
]
|
||||||
|
|
||||||
|
|
||||||
|
class DatabricksPipelineTests(TestCase):
|
||||||
|
"""
|
||||||
|
Implements the necessary methods to extract
|
||||||
|
Databricks Pipeline test
|
||||||
|
"""
|
||||||
|
|
||||||
|
maxDiff = None
|
||||||
|
|
||||||
|
@patch(
|
||||||
|
"metadata.ingestion.source.pipeline.pipeline_service.PipelineServiceSource.test_connection"
|
||||||
|
)
|
||||||
|
def __init__(self, methodName, test_connection) -> None:
|
||||||
|
super().__init__(methodName)
|
||||||
|
print_ansi_encoded_string(message="init")
|
||||||
|
test_connection.return_value = False
|
||||||
|
config = OpenMetadataWorkflowConfig.parse_obj(mock_databricks_config)
|
||||||
|
|
||||||
|
self.databricks = DatabrickspipelineSource.create(
|
||||||
|
mock_databricks_config["source"],
|
||||||
|
config.workflowConfig.openMetadataServerConfig,
|
||||||
|
)
|
||||||
|
self.databricks.context.__dict__["pipeline"] = MOCK_PIPELINE
|
||||||
|
self.databricks.context.__dict__["pipeline_service"] = MOCK_PIPELINE_SERVICE
|
||||||
|
self.databricks.context.__dict__["job_id_list"] = [
|
||||||
|
mock_history_data[0]["job_id"]
|
||||||
|
]
|
||||||
|
|
||||||
|
@patch(
|
||||||
|
"metadata.ingestion.source.database.databricks.client.DatabricksClient.list_jobs"
|
||||||
|
)
|
||||||
|
def test_get_pipelines_list(self, list_jobs):
|
||||||
|
list_jobs.return_value = mock_data
|
||||||
|
results = list(self.databricks.get_pipelines_list())
|
||||||
|
self.assertEqual(mock_data, results)
|
||||||
|
|
||||||
|
def test_yield_pipeline(self):
|
||||||
|
pipelines = list(self.databricks.yield_pipeline(mock_data[0]))[0]
|
||||||
|
self.assertEqual(pipelines, EXPECTED_CREATED_PIPELINES)
|
||||||
|
|
||||||
|
@patch(
|
||||||
|
"metadata.ingestion.source.database.databricks.client.DatabricksClient.get_job_runs"
|
||||||
|
)
|
||||||
|
def test_yield_pipeline_status(self, get_job_runs):
|
||||||
|
get_job_runs.return_value = mock_history_data
|
||||||
|
pipeline_status = list(
|
||||||
|
self.databricks.yield_pipeline_status(mock_history_data[0]["job_id"])
|
||||||
|
)
|
||||||
|
self.assertEqual(pipeline_status, EXPECTED_PIPELINE_STATUS)
|
@ -0,0 +1,55 @@
|
|||||||
|
{
|
||||||
|
"$id": "https://open-metadata.org/schema/entity/services/connections/pipeline/DatabricksPipelineConnection.json",
|
||||||
|
"$schema": "http://json-schema.org/draft-07/schema#",
|
||||||
|
"title": "DatabricksPipelineConnection",
|
||||||
|
"description": "Databricks Connection Config",
|
||||||
|
"type": "object",
|
||||||
|
"javaType": "org.openmetadata.schema.services.connections.pipeline.DatabricksPipelineConnection",
|
||||||
|
"definitions": {
|
||||||
|
"databricksType": {
|
||||||
|
"description": "Service type.",
|
||||||
|
"type": "string",
|
||||||
|
"enum": [
|
||||||
|
"DatabricksPipeline"
|
||||||
|
],
|
||||||
|
"default": "DatabricksPipeline"
|
||||||
|
}
|
||||||
|
},
|
||||||
|
"properties": {
|
||||||
|
"type": {
|
||||||
|
"title": "Service Type",
|
||||||
|
"description": "Service Type",
|
||||||
|
"$ref": "#/definitions/databricksType",
|
||||||
|
"default": "DatabricksPipeline"
|
||||||
|
},
|
||||||
|
"hostPort": {
|
||||||
|
"title": "Host and Port",
|
||||||
|
"description": "Host and port of the Databricks service.",
|
||||||
|
"type": "string"
|
||||||
|
},
|
||||||
|
"token": {
|
||||||
|
"title": "Token",
|
||||||
|
"description": "Generated Token to connect to Databricks.",
|
||||||
|
"type": "string",
|
||||||
|
"format": "password"
|
||||||
|
},
|
||||||
|
"httpPath": {
|
||||||
|
"title": "Http Path",
|
||||||
|
"description": "Databricks compute resources URL.",
|
||||||
|
"type": "string"
|
||||||
|
},
|
||||||
|
"connectionArguments": {
|
||||||
|
"title": "Connection Arguments",
|
||||||
|
"$ref": "../connectionBasicType.json#/definitions/connectionArguments"
|
||||||
|
},
|
||||||
|
"supportsMetadataExtraction": {
|
||||||
|
"title": "Supports Metadata Extraction",
|
||||||
|
"$ref": "../connectionBasicType.json#/definitions/supportsMetadataExtraction"
|
||||||
|
}
|
||||||
|
},
|
||||||
|
"additionalProperties": false,
|
||||||
|
"required": [
|
||||||
|
"hostPort",
|
||||||
|
"token"
|
||||||
|
]
|
||||||
|
}
|
@ -13,7 +13,9 @@
|
|||||||
"pipelineServiceType": {
|
"pipelineServiceType": {
|
||||||
"description": "Type of pipeline service - Airflow or Prefect.",
|
"description": "Type of pipeline service - Airflow or Prefect.",
|
||||||
"type": "string",
|
"type": "string",
|
||||||
"javaInterfaces": ["org.openmetadata.schema.EnumInterface"],
|
"javaInterfaces": [
|
||||||
|
"org.openmetadata.schema.EnumInterface"
|
||||||
|
],
|
||||||
"enum": [
|
"enum": [
|
||||||
"Airflow",
|
"Airflow",
|
||||||
"GluePipeline",
|
"GluePipeline",
|
||||||
@ -22,7 +24,8 @@
|
|||||||
"Dagster",
|
"Dagster",
|
||||||
"Nifi",
|
"Nifi",
|
||||||
"DomoPipeline",
|
"DomoPipeline",
|
||||||
"CustomPipeline"
|
"CustomPipeline",
|
||||||
|
"DatabricksPipeline"
|
||||||
],
|
],
|
||||||
"javaEnums": [
|
"javaEnums": [
|
||||||
{
|
{
|
||||||
@ -48,6 +51,9 @@
|
|||||||
},
|
},
|
||||||
{
|
{
|
||||||
"name": "CustomPipeline"
|
"name": "CustomPipeline"
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name": "DatabricksPipeline"
|
||||||
}
|
}
|
||||||
]
|
]
|
||||||
},
|
},
|
||||||
@ -85,6 +91,9 @@
|
|||||||
},
|
},
|
||||||
{
|
{
|
||||||
"$ref": "./connections/pipeline/customPipelineConnection.json"
|
"$ref": "./connections/pipeline/customPipelineConnection.json"
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"$ref": "./connections/pipeline/databricksPipelineConnection.json"
|
||||||
}
|
}
|
||||||
]
|
]
|
||||||
}
|
}
|
||||||
@ -162,6 +171,10 @@
|
|||||||
"default": false
|
"default": false
|
||||||
}
|
}
|
||||||
},
|
},
|
||||||
"required": ["id", "name", "serviceType"],
|
"required": [
|
||||||
|
"id",
|
||||||
|
"name",
|
||||||
|
"serviceType"
|
||||||
|
],
|
||||||
"additionalProperties": false
|
"additionalProperties": false
|
||||||
}
|
}
|
Loading…
x
Reference in New Issue
Block a user