From 132cd497c7ccf051620e3d15566098d269beface Mon Sep 17 00:00:00 2001 From: Pere Miquel Brull Date: Fri, 3 Nov 2023 18:06:50 +0100 Subject: [PATCH 01/16] Prepare the skeleton for generic app registration --- ingestion/src/metadata/cmd.py | 11 + .../src/metadata/workflow/application.py | 61 ++++++ ingestion/src/metadata/workflow/base.py | 195 +++++------------- .../src/metadata/workflow/data_insight.py | 5 +- .../src/metadata/workflow/data_quality.py | 3 +- ingestion/src/metadata/workflow/ingestion.py | 167 +++++++++++++++ ingestion/src/metadata/workflow/metadata.py | 3 +- ingestion/src/metadata/workflow/profiler.py | 3 +- ingestion/src/metadata/workflow/usage.py | 5 +- .../openmetadata_managed_apis/utils/logger.py | 4 +- .../workflows/ingestion/application.py | 81 ++++++++ .../workflows/ingestion/common.py | 6 +- .../workflows/ingestion/registry.py | 2 + .../apps/AbstractNativeApplication.java | 8 +- .../ingestionPipelines/ingestionPipeline.json | 2 +- .../schema/metadataIngestion/application.json | 27 +++ .../applicationPipeline.json | 15 +- 17 files changed, 430 insertions(+), 168 deletions(-) create mode 100644 ingestion/src/metadata/workflow/application.py create mode 100644 ingestion/src/metadata/workflow/ingestion.py create mode 100644 openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py create mode 100644 openmetadata-spec/src/main/resources/json/schema/metadataIngestion/application.json diff --git a/ingestion/src/metadata/cmd.py b/ingestion/src/metadata/cmd.py index f839a9183710..23d4be4f5f03 100644 --- a/ingestion/src/metadata/cmd.py +++ b/ingestion/src/metadata/cmd.py @@ -18,6 +18,7 @@ from pathlib import Path from metadata.__version__ import get_metadata_version +from metadata.cli.app import run_app from metadata.cli.backup import UploadDestinationType, run_backup from metadata.cli.dataquality import run_test from metadata.cli.docker import BACKEND_DATABASES, DockerActions, run_docker @@ -50,6 +51,7 @@ class MetadataCommands(Enum): WEBHOOK = "webhook" INSIGHT = "insight" LINEAGE = "lineage" + APP = "app" OPENMETADATA_IMPORTS_MIGRATION = "openmetadata_imports_migration" OPENMETADATA_DAG_CONFIG_MIGRATION = "openmetadata_dag_config_migration" @@ -365,6 +367,11 @@ def get_parser(args=None): MetadataCommands.TEST.value, help="Workflow for running test suites" ) ) + create_common_config_parser_args( + sub_parser.add_parser( + MetadataCommands.APP.value, help="Workflow for running external applications" + ) + ) create_openmetadata_imports_migration_args( sub_parser.add_parser( MetadataCommands.OPENMETADATA_IMPORTS_MIGRATION.value, @@ -418,6 +425,8 @@ def metadata(args=None): # pylint: disable=too-many-branches config_file = contains_args.get("config") if config_file: path = Path(config_file).expanduser() + else: + raise ValueError("Could not load config file! Please specify the config path with `-c` or `--config`.") if contains_args.get("debug"): set_loggers_level(logging.DEBUG) elif contains_args.get("log_level"): @@ -437,6 +446,8 @@ def metadata(args=None): # pylint: disable=too-many-branches run_profiler(config_path=path) if metadata_workflow == MetadataCommands.TEST.value: run_test(config_path=path) + if metadata_workflow == MetadataCommands.APP.value: + run_app(config_path=path) if metadata_workflow == MetadataCommands.BACKUP.value: run_backup( common_backup_obj_instance=BackupRestoreArgs( diff --git a/ingestion/src/metadata/workflow/application.py b/ingestion/src/metadata/workflow/application.py new file mode 100644 index 000000000000..d629737407c0 --- /dev/null +++ b/ingestion/src/metadata/workflow/application.py @@ -0,0 +1,61 @@ +# 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. +""" +Generic Workflow entrypoint to execute Applications +""" +from abc import ABC, abstractmethod +from typing import TypeVar, Generic + +from metadata.generated.schema.entity.services.connections.metadata.openMetadataConnection import OpenMetadataConnection +from metadata.generated.schema.metadataIngestion.application import OpenMetadataApplicationConfig +from metadata.generated.schema.metadataIngestion.workflow import LogLevels + +from build.lib.metadata.generated.schema.entity.services.serviceType import ServiceType +from metadata.workflow.base import BaseWorkflow + +# Configuration class +C = TypeVar("C") + + +class ApplicationWorkflow(BaseWorkflow, ABC, Generic[C]): + """ + Base Application Workflow implementation + """ + config: OpenMetadataApplicationConfig + + def __init__(self, config_dict: dict): + + self.config = OpenMetadataApplicationConfig.parse_obj(config_dict) + + # Applications are associated to the OpenMetadata Service + self.service_type: ServiceType = ServiceType.Metadata + + metadata_config: OpenMetadataConnection = self.config.workflowConfig.openMetadataServerConfig + log_level: LogLevels = self.config.workflowConfig.loggerLevel + + super().__init__( + config=self.config, + log_level=log_level, + metadata_config=metadata_config, + service_type=self.service_type, + ) + + @classmethod + def create(cls, config_dict: dict): + return cls(config_dict) + + @abstractmethod + def post_init(self) -> None: + """Method to execute after we have initialized all the internals""" + + @abstractmethod + def execute_internal(self) -> None: + """Workflow-specific logic to execute safely""" diff --git a/ingestion/src/metadata/workflow/base.py b/ingestion/src/metadata/workflow/base.py index e5aab5f490af..11aa16a97a11 100644 --- a/ingestion/src/metadata/workflow/base.py +++ b/ingestion/src/metadata/workflow/base.py @@ -10,20 +10,14 @@ # limitations under the License. """ Base workflow definition. - -To be extended by any other workflow: -- ingestion -- lineage -- usage -- profiler -- test suite -- data insights """ -import traceback + import uuid from abc import ABC, abstractmethod from datetime import datetime -from typing import Optional, Tuple, TypeVar, cast +from typing import Dict, Optional, TypeVar, Union, Any + +from pydantic import BaseModel from metadata.generated.schema.api.services.ingestionPipelines.createIngestionPipeline import ( CreateIngestionPipelineRequest, @@ -31,23 +25,14 @@ from metadata.generated.schema.entity.services.connections.metadata.openMetadataConnection import ( OpenMetadataConnection, ) -from metadata.generated.schema.entity.services.connections.serviceConnection import ( - ServiceConnection, -) from metadata.generated.schema.entity.services.ingestionPipelines.ingestionPipeline import ( AirflowConfig, IngestionPipeline, PipelineState, ) -from metadata.generated.schema.metadataIngestion.workflow import ( - OpenMetadataWorkflowConfig, -) +from metadata.generated.schema.metadataIngestion.workflow import LogLevels from metadata.generated.schema.tests.testSuite import ServiceType from metadata.generated.schema.type.entityReference import EntityReference -from metadata.ingestion.api.parser import parse_workflow_config_gracefully -from metadata.ingestion.api.step import Step -from metadata.ingestion.api.steps import BulkSink, Processor, Sink, Source, Stage -from metadata.ingestion.models.custom_types import ServiceWithConnectionType from metadata.ingestion.ometa.client_utils import create_ometa_client from metadata.ingestion.ometa.ometa_api import OpenMetadata from metadata.timer.repeated_timer import RepeatedTimer @@ -56,7 +41,6 @@ get_pipeline_type_from_source_config, get_reference_type_from_service_type, get_service_class_from_service_type, - get_service_type_from_source_type, ) from metadata.utils.helpers import datetime_to_ts from metadata.utils.logger import ingestion_logger, set_loggers_level @@ -65,6 +49,7 @@ logger = ingestion_logger() +# Type of service linked to the Ingestion Pipeline T = TypeVar("T") REPORTS_INTERVAL_SECONDS = 60 @@ -81,46 +66,35 @@ class BaseWorkflow(ABC, WorkflowStatusMixin): Base workflow implementation """ - config: OpenMetadataWorkflowConfig + config: Union[Any, Dict] _run_id: Optional[str] = None - service_type: ServiceType - metadata_config: OpenMetadataConnection metadata: OpenMetadata + service_type: ServiceType - # All workflows require a source as a first step - source: Source - # All workflows execute a series of steps, aside from the source - steps: Tuple[Step] - - def __init__(self, config: OpenMetadataWorkflowConfig): + def __init__( + self, + config: Union[Any, Dict], + log_level: LogLevels, + metadata_config: OpenMetadataConnection, + service_type: ServiceType, + ): """ Disabling pylint to wait for workflow reimplementation as a topology """ self.config = config + self.service_type = service_type self._timer: Optional[RepeatedTimer] = None self._ingestion_pipeline: Optional[IngestionPipeline] = None self._start_ts = datetime_to_ts(datetime.now()) - set_loggers_level(config.workflowConfig.loggerLevel.value) - - self.service_type: ServiceType = get_service_type_from_source_type( - self.config.source.type - ) - - self.metadata_config: OpenMetadataConnection = ( - self.config.workflowConfig.openMetadataServerConfig - ) + set_loggers_level(log_level.value) # We create the ometa client at the workflow level and pass it to the steps - self.metadata = create_ometa_client(self.metadata_config) + self.metadata = create_ometa_client(metadata_config) self.set_ingestion_pipeline_status(state=PipelineState.running) - # Pick up the service connection from the API if needed - self._retrieve_service_connection_if_needed(self.service_type) - - # Informs the `source` and the rest of `steps` to execute - self.set_steps() + self.post_init() @property def ingestion_pipeline(self): @@ -130,61 +104,6 @@ def ingestion_pipeline(self): return self._ingestion_pipeline - @abstractmethod - def set_steps(self): - """ - initialize the tuple of steps to run for each workflow - and the source - """ - - def _execute_internal(self): - """ - Internal execution that needs to be filled - by each ingestion workflow. - - Pass each record from the source down the pipeline: - Source -> (Processor) -> Sink - or Source -> (Processor) -> Stage -> BulkSink - - Note how the Source class needs to be an Iterator. Specifically, - we are defining Sources as Generators. - """ - for record in self.source.run(): - processed_record = record - for step in self.steps: - # We only process the records for these Step types - if processed_record is not None and isinstance( - step, (Processor, Stage, Sink) - ): - processed_record = step.run(processed_record) - - # Try to pick up the BulkSink and execute it, if needed - bulk_sink = next( - (step for step in self.steps if isinstance(step, BulkSink)), None - ) - if bulk_sink: - bulk_sink.run() - - def execute(self) -> None: - """ - Main entrypoint - """ - self.timer.trigger() - try: - self._execute_internal() - - # If we reach this point, compute the success % and update the associated Ingestion Pipeline status - self.update_ingestion_status_at_end() - - # Any unhandled exception breaking the workflow should update the status - except Exception as err: - self.set_ingestion_pipeline_status(PipelineState.failed) - raise err - - # Force resource closing. Required for killing the threading - finally: - self.stop() - def stop(self) -> None: """ Main stopping logic @@ -213,9 +132,37 @@ def timer(self) -> RepeatedTimer: return self._timer @classmethod - def create(cls, config_dict: dict) -> "BaseWorkflow": - config = parse_workflow_config_gracefully(config_dict) - return cls(config) + @abstractmethod + def create(cls, config_dict: dict): + """Single function to execute to create a Workflow instance""" + + @abstractmethod + def post_init(self) -> None: + """Method to execute after we have initialized all the internals""" + + @abstractmethod + def execute_internal(self) -> None: + """Workflow-specific logic to execute safely""" + + def execute(self) -> None: + """ + Main entrypoint + """ + self.timer.trigger() + try: + self.execute_internal() + + # If we reach this point, compute the success % and update the associated Ingestion Pipeline status + self.update_ingestion_status_at_end() + + # Any unhandled exception breaking the workflow should update the status + except Exception as err: + self.set_ingestion_pipeline_status(PipelineState.failed) + raise err + + # Force resource closing. Required for killing the threading + finally: + self.stop() @property def run_id(self) -> str: @@ -231,48 +178,6 @@ def run_id(self) -> str: return self._run_id - def _retrieve_service_connection_if_needed(self, service_type: ServiceType) -> None: - """ - We override the current `serviceConnection` source config object if source workflow service already exists - in OM. When secrets' manager is configured, we retrieve the service connection from the secrets' manager. - Otherwise, we get the service connection from the service object itself through the default `SecretsManager`. - - :param service_type: source workflow service type - :return: - """ - if ( - not self.config.source.serviceConnection - and not self.metadata.config.forceEntityOverwriting - ): - service_name = self.config.source.serviceName - try: - service: ServiceWithConnectionType = cast( - ServiceWithConnectionType, - self.metadata.get_by_name( - get_service_class_from_service_type(service_type), - service_name, - ), - ) - if service: - self.config.source.serviceConnection = ServiceConnection( - __root__=service.connection - ) - else: - raise InvalidWorkflowJSONException( - f"Error getting the service [{service_name}] from the API. If it exists in OpenMetadata," - " make sure the ingestion-bot JWT token is valid and that the Workflow is deployed" - " with the latest one. If this error persists, recreate the JWT token and" - " redeploy the Workflow." - ) - except InvalidWorkflowJSONException as exc: - raise exc - except Exception as exc: - logger.debug(traceback.format_exc()) - logger.error( - f"Unknown error getting service connection for service name [{service_name}]" - f" using the secrets manager provider [{self.metadata.config.secretsManagerProvider}]: {exc}" - ) - def get_or_create_ingestion_pipeline(self) -> Optional[IngestionPipeline]: """ If we get the `ingestionPipelineFqn` from the `workflowConfig`, it means we want to diff --git a/ingestion/src/metadata/workflow/data_insight.py b/ingestion/src/metadata/workflow/data_insight.py index 80d90a379bf7..ea0b459caf00 100644 --- a/ingestion/src/metadata/workflow/data_insight.py +++ b/ingestion/src/metadata/workflow/data_insight.py @@ -20,11 +20,12 @@ from metadata.utils.logger import data_insight_logger from metadata.utils.time_utils import get_beginning_of_day_timestamp_mill from metadata.workflow.base import BaseWorkflow +from metadata.workflow.ingestion import IngestionWorkflow logger = data_insight_logger() -class DataInsightWorkflow(BaseWorkflow): +class DataInsightWorkflow(IngestionWorkflow): """Data insight ingestion workflow implementation""" retention_days = 7 @@ -66,7 +67,7 @@ def _get_sink(self) -> Sink: def _execute_internal(self): """Use parent logic and add step to process KPIs""" - super()._execute_internal() + super().execute_internal() self._run_kpi_processor() self._clean_up_web_analytics_events() diff --git a/ingestion/src/metadata/workflow/data_quality.py b/ingestion/src/metadata/workflow/data_quality.py index ca9ddd258671..4ec1036178f7 100644 --- a/ingestion/src/metadata/workflow/data_quality.py +++ b/ingestion/src/metadata/workflow/data_quality.py @@ -26,11 +26,12 @@ from metadata.utils.importer import import_sink_class from metadata.utils.logger import test_suite_logger from metadata.workflow.base import BaseWorkflow, T +from metadata.workflow.ingestion import IngestionWorkflow logger = test_suite_logger() -class TestSuiteWorkflow(BaseWorkflow): +class TestSuiteWorkflow(IngestionWorkflow): """ DAta Quality ingestion workflow implementation diff --git a/ingestion/src/metadata/workflow/ingestion.py b/ingestion/src/metadata/workflow/ingestion.py new file mode 100644 index 000000000000..f087b12c7638 --- /dev/null +++ b/ingestion/src/metadata/workflow/ingestion.py @@ -0,0 +1,167 @@ +# 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. +""" +Generic Workflow entrypoint to execute Ingestions + +To be extended by any other workflow: +- metadata +- lineage +- usage +- profiler +- test suite +- data insights +""" +import traceback +from abc import ABC, abstractmethod +from typing import Tuple, cast + +from metadata.generated.schema.entity.services.connections.metadata.openMetadataConnection import ( + OpenMetadataConnection, +) +from metadata.generated.schema.entity.services.connections.serviceConnection import ( + ServiceConnection, +) +from metadata.generated.schema.entity.services.serviceType import ServiceType +from metadata.generated.schema.metadataIngestion.workflow import ( + OpenMetadataWorkflowConfig, +) +from metadata.ingestion.api.parser import parse_workflow_config_gracefully +from metadata.ingestion.api.step import Step +from metadata.ingestion.api.steps import BulkSink, Processor, Sink, Source, Stage +from metadata.ingestion.models.custom_types import ServiceWithConnectionType +from metadata.utils.class_helper import ( + get_service_class_from_service_type, + get_service_type_from_source_type, +) +from metadata.utils.logger import ingestion_logger +from metadata.workflow.base import BaseWorkflow, InvalidWorkflowJSONException + +logger = ingestion_logger() + + +class IngestionWorkflow(BaseWorkflow, ABC): + """ + Base Ingestion Workflow implementation + """ + + config: OpenMetadataWorkflowConfig + + # All workflows require a source as a first step + source: Source + # All workflows execute a series of steps, aside from the source + steps: Tuple[Step] + + def __init__(self, config: OpenMetadataWorkflowConfig): + self.config = config + + self.service_type: ServiceType = get_service_type_from_source_type( + self.config.source.type + ) + + metadata_config: OpenMetadataConnection = ( + self.config.workflowConfig.openMetadataServerConfig + ) + + super().__init__( + config=config, + log_level=config.workflowConfig.loggerLevel, + metadata_config=metadata_config, + service_type=self.service_type, + ) + + @abstractmethod + def set_steps(self): + """ + initialize the tuple of steps to run for each workflow + and the source + """ + + def post_init(self) -> None: + # Pick up the service connection from the API if needed + self._retrieve_service_connection_if_needed(self.service_type) + + # Informs the `source` and the rest of `steps` to execute + self.set_steps() + + @classmethod + def create(cls, config_dict: dict): + config = parse_workflow_config_gracefully(config_dict) + return cls(config) + + def execute_internal(self): + """ + Internal execution that needs to be filled + by each ingestion workflow. + + Pass each record from the source down the pipeline: + Source -> (Processor) -> Sink + or Source -> (Processor) -> Stage -> BulkSink + + Note how the Source class needs to be an Iterator. Specifically, + we are defining Sources as Generators. + """ + for record in self.source.run(): + processed_record = record + for step in self.steps: + # We only process the records for these Step types + if processed_record is not None and isinstance( + step, (Processor, Stage, Sink) + ): + processed_record = step.run(processed_record) + + # Try to pick up the BulkSink and execute it, if needed + bulk_sink = next( + (step for step in self.steps if isinstance(step, BulkSink)), None + ) + if bulk_sink: + bulk_sink.run() + + def _retrieve_service_connection_if_needed(self, service_type: ServiceType) -> None: + """ + We override the current `serviceConnection` source config object if source workflow service already exists + in OM. When secrets' manager is configured, we retrieve the service connection from the secrets' manager. + Otherwise, we get the service connection from the service object itself through the default `SecretsManager`. + + :param service_type: source workflow service type + :return: + """ + if ( + not self.config.source.serviceConnection + and not self.metadata.config.forceEntityOverwriting + ): + service_name = self.config.source.serviceName + try: + service: ServiceWithConnectionType = cast( + ServiceWithConnectionType, + self.metadata.get_by_name( + get_service_class_from_service_type(service_type), + service_name, + ), + ) + if service: + self.config.source.serviceConnection = ServiceConnection( + __root__=service.connection + ) + else: + raise InvalidWorkflowJSONException( + f"Error getting the service [{service_name}] from the API. If it exists in OpenMetadata," + " make sure the ingestion-bot JWT token is valid and that the Workflow is deployed" + " with the latest one. If this error persists, recreate the JWT token and" + " redeploy the Workflow." + ) + except InvalidWorkflowJSONException as exc: + raise exc + except Exception as exc: + logger.debug(traceback.format_exc()) + logger.error( + f"Unknown error getting service connection for service name [{service_name}]" + f" using the secrets manager provider [{self.metadata.config.secretsManagerProvider}]: {exc}" + ) diff --git a/ingestion/src/metadata/workflow/metadata.py b/ingestion/src/metadata/workflow/metadata.py index 97a7faa71a97..f92823f601b5 100644 --- a/ingestion/src/metadata/workflow/metadata.py +++ b/ingestion/src/metadata/workflow/metadata.py @@ -20,11 +20,12 @@ ) from metadata.utils.logger import ingestion_logger from metadata.workflow.base import BaseWorkflow +from metadata.workflow.ingestion import IngestionWorkflow logger = ingestion_logger() -class MetadataWorkflow(BaseWorkflow): +class MetadataWorkflow(IngestionWorkflow): """ Metadata ingestion workflow implementation. """ diff --git a/ingestion/src/metadata/workflow/profiler.py b/ingestion/src/metadata/workflow/profiler.py index 54dea4b86e2a..b5b1b9d80390 100644 --- a/ingestion/src/metadata/workflow/profiler.py +++ b/ingestion/src/metadata/workflow/profiler.py @@ -22,11 +22,12 @@ from metadata.utils.importer import import_sink_class from metadata.utils.logger import profiler_logger from metadata.workflow.base import BaseWorkflow +from metadata.workflow.ingestion import IngestionWorkflow logger = profiler_logger() -class ProfilerWorkflow(BaseWorkflow): +class ProfilerWorkflow(IngestionWorkflow): """ Profiler ingestion workflow implementation diff --git a/ingestion/src/metadata/workflow/usage.py b/ingestion/src/metadata/workflow/usage.py index 7504d2b40a04..5db1adcdc2c1 100644 --- a/ingestion/src/metadata/workflow/usage.py +++ b/ingestion/src/metadata/workflow/usage.py @@ -22,11 +22,12 @@ ) from metadata.utils.logger import ingestion_logger from metadata.workflow.base import BaseWorkflow +from metadata.workflow.ingestion import IngestionWorkflow logger = ingestion_logger() -class UsageWorkflow(BaseWorkflow): +class UsageWorkflow(IngestionWorkflow): """ Usage ingestion workflow implementation. """ @@ -69,7 +70,7 @@ def _get_processor(self) -> Processor: processor_config = self.config.processor.dict().get("config", {}) processor: Processor = processor_class.create( processor_config, - self.metadata_config, + self.metadata, connection_type=str( self.config.source.serviceConnection.__root__.config.type.value ), diff --git a/openmetadata-airflow-apis/openmetadata_managed_apis/utils/logger.py b/openmetadata-airflow-apis/openmetadata_managed_apis/utils/logger.py index 9409640c8d35..aba68cfae0bc 100644 --- a/openmetadata-airflow-apis/openmetadata_managed_apis/utils/logger.py +++ b/openmetadata-airflow-apis/openmetadata_managed_apis/utils/logger.py @@ -1,8 +1,10 @@ import logging from enum import Enum from logging.handlers import RotatingFileHandler +from typing import Union from airflow.configuration import conf +from metadata.generated.schema.metadataIngestion.application import OpenMetadataApplicationConfig from metadata.generated.schema.metadataIngestion.workflow import ( OpenMetadataWorkflowConfig, @@ -58,7 +60,7 @@ def utils_logger(): return build_logger(Loggers.UTILS.value) -def set_operator_logger(workflow_config: OpenMetadataWorkflowConfig) -> None: +def set_operator_logger(workflow_config: Union[OpenMetadataWorkflowConfig, OpenMetadataApplicationConfig]) -> None: """ Handle logging for the Python Operator that will execute the ingestion diff --git a/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py b/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py new file mode 100644 index 000000000000..739c8ea44872 --- /dev/null +++ b/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py @@ -0,0 +1,81 @@ +# 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. +""" +Generic Workflow entrypoint to execute Applications +""" +import json +from typing import cast + +from airflow import DAG +from metadata.generated.schema.entity.services.ingestionPipelines.ingestionPipeline import IngestionPipeline +from metadata.generated.schema.metadataIngestion.application import OpenMetadataApplicationConfig +from metadata.generated.schema.metadataIngestion.applicationPipeline import ApplicationPipeline +from metadata.ingestion.models.encoders import show_secrets_encoder +from metadata.utils.workflow_output_handler import print_status +from metadata.workflow.application import ApplicationWorkflow + +from openmetadata_managed_apis.utils.logger import set_operator_logger +from openmetadata_managed_apis.workflows.ingestion.common import build_workflow_config_property, build_dag + + +def application_workflow(application_workflow_config: OpenMetadataApplicationConfig): + """ + Task that creates and runs the ingestion workflow. + + The workflow_config gets cooked form the incoming + ingestionPipeline. + + This is the callable used to create the PythonOperator + """ + + set_operator_logger(application_workflow_config) + + config = json.loads(application_workflow_config.json(encoder=show_secrets_encoder)) + workflow = ApplicationWorkflow.create(config) + + workflow.execute() + workflow.raise_from_status() + print_status(workflow) + workflow.stop() + + +def build_application_workflow_config( + ingestion_pipeline: IngestionPipeline, +) -> OpenMetadataApplicationConfig: + """ + Given an airflow_pipeline, prepare the workflow config JSON + """ + + # Here we have an application pipeline, so the Source Config is of type ApplicationPipeline + application_pipeline_conf: ApplicationPipeline = cast(ingestion_pipeline.sourceConfig.config, ApplicationPipeline) + + application_workflow_config = OpenMetadataApplicationConfig( + config=application_pipeline_conf.appConfig, + workflowConfig=build_workflow_config_property(ingestion_pipeline), + ingestionPipelineFQN=ingestion_pipeline.fullyQualifiedName.__root__, + ) + + return application_workflow_config + + +def build_application_dag(ingestion_pipeline: IngestionPipeline) -> DAG: + """ + Build a simple metadata workflow DAG + """ + application_workflow_config = build_application_workflow_config(ingestion_pipeline) + dag = build_dag( + task_name="application_task", + ingestion_pipeline=ingestion_pipeline, + workflow_config=application_workflow_config, + workflow_fn=application_workflow, + ) + + return dag diff --git a/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/common.py b/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/common.py index 9998dca6d86e..87ab4bc5f467 100644 --- a/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/common.py +++ b/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/common.py @@ -15,10 +15,12 @@ import uuid from datetime import datetime, timedelta from functools import partial -from typing import Callable +from typing import Callable, Union import airflow from airflow import DAG +from metadata.generated.schema.metadataIngestion.application import OpenMetadataApplicationConfig + from openmetadata_managed_apis.api.utils import clean_dag_id from pydantic import ValidationError from requests.utils import quote @@ -303,7 +305,7 @@ def send_failed_status_callback(workflow_config: OpenMetadataWorkflowConfig, _): def build_dag( task_name: str, ingestion_pipeline: IngestionPipeline, - workflow_config: OpenMetadataWorkflowConfig, + workflow_config: Union[OpenMetadataWorkflowConfig, OpenMetadataApplicationConfig], workflow_fn: Callable, ) -> DAG: """ diff --git a/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/registry.py b/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/registry.py index 23033a66a6b2..3ead53763969 100644 --- a/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/registry.py +++ b/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/registry.py @@ -13,6 +13,7 @@ Add a function for each type from PipelineType """ +from openmetadata_managed_apis.workflows.ingestion.application import build_application_dag from openmetadata_managed_apis.workflows.ingestion.data_insight import ( build_data_insight_dag, ) @@ -43,3 +44,4 @@ build_registry.add(PipelineType.TestSuite.value)(build_test_suite_dag) build_registry.add(PipelineType.dataInsight.value)(build_data_insight_dag) build_registry.add(PipelineType.elasticSearchReindex.value)(build_es_reindex_dag) +build_registry.add(PipelineType.application.value)(build_application_dag) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/AbstractNativeApplication.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/AbstractNativeApplication.java index 4dd8e4a8adb5..7f00c7ed4ea7 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/AbstractNativeApplication.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/AbstractNativeApplication.java @@ -42,10 +42,10 @@ @Slf4j public class AbstractNativeApplication implements NativeApplication { protected CollectionDAO collectionDAO; - private App app; + public App app; protected SearchRepository searchRepository; - private final CronMapper cronMapper = CronMapper.fromQuartzToUnix(); - private final CronParser cronParser = new CronParser(CronDefinitionBuilder.instanceDefinitionFor(QUARTZ)); + public final CronMapper cronMapper = CronMapper.fromQuartzToUnix(); + public final CronParser cronParser = new CronParser(CronDefinitionBuilder.instanceDefinitionFor(QUARTZ)); @Override public void init(App app, CollectionDAO dao, SearchRepository searchRepository) { @@ -187,7 +187,7 @@ public static AppRuntime getAppRuntime(App app) { return JsonUtils.convertValue(app.getRuntime(), ScheduledExecutionContext.class); } - private IngestionPipeline getIngestionPipeline(CreateIngestionPipeline create, String botname, String user) { + public IngestionPipeline getIngestionPipeline(CreateIngestionPipeline create, String botname, String user) { IngestionPipelineRepository ingestionPipelineRepository = (IngestionPipelineRepository) Entity.getEntityRepository(Entity.INGESTION_PIPELINE); OpenMetadataConnection openMetadataServerConnection = diff --git a/openmetadata-spec/src/main/resources/json/schema/entity/services/ingestionPipelines/ingestionPipeline.json b/openmetadata-spec/src/main/resources/json/schema/entity/services/ingestionPipelines/ingestionPipeline.json index d7606d2b50dd..b6121ecb2469 100644 --- a/openmetadata-spec/src/main/resources/json/schema/entity/services/ingestionPipelines/ingestionPipeline.json +++ b/openmetadata-spec/src/main/resources/json/schema/entity/services/ingestionPipelines/ingestionPipeline.json @@ -11,7 +11,7 @@ "description": "Type of Pipeline - metadata, usage", "type": "string", "javaType": "org.openmetadata.schema.entity.services.ingestionPipelines.PipelineType", - "enum": ["metadata", "usage", "lineage", "profiler", "TestSuite", "dataInsight", "elasticSearchReindex", "dbt", "Application"] + "enum": ["metadata", "usage", "lineage", "profiler", "TestSuite", "dataInsight", "elasticSearchReindex", "dbt", "application"] }, "pipelineStatus": { "type": "object", diff --git a/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/application.json b/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/application.json new file mode 100644 index 000000000000..28ce20980bbc --- /dev/null +++ b/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/application.json @@ -0,0 +1,27 @@ +{ + "$id": "https://open-metadata.org/schema/metadataIngestion/application.json", + "$schema": "http://json-schema.org/draft-07/schema#", + "title": "OpenMetadataApplicationConfig", + "description": "OpenMetadata Application Workflow definition.", + "type": "object", + "properties": { + "workflowConfig": { + "$ref": "workflow.json#/definitions/workflowConfig", + "description": "General Workflow configuration, such as the OpenMetadata server connection and logging level" + }, + "config": { + "$ref": "../type/basic.json#/definitions/componentConfig", + "description": "Application-specific configuration" + }, + "ingestionPipelineFQN": { + "description": "Fully qualified name of ingestion pipeline, used to identify the current ingestion pipeline", + "type": "string" + }, + "pipelineRunId": { + "description": "Unique identifier of pipeline run, used to identify the current pipeline run", + "$ref": "../type/basic.json#/definitions/uuid" + } + }, + "required": ["workflowConfig"], + "additionalProperties": false +} \ No newline at end of file diff --git a/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/applicationPipeline.json b/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/applicationPipeline.json index 2cab45bcfd80..eb01454431f4 100644 --- a/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/applicationPipeline.json +++ b/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/applicationPipeline.json @@ -5,8 +5,8 @@ "description": "Application Pipeline Configuration.", "type": "object", "definitions": { - "testSuiteConfigType": { - "description": "Pipeline Source Config For Application Pipeline type", + "applicationConfigType": { + "description": "Pipeline Source Config For Application Pipeline type. Nothing is required.", "type": "string", "enum": ["Application"], "default": "Application" @@ -15,14 +15,13 @@ "properties": { "type": { "description": "Pipeline type", - "$ref": "#/definitions/testSuiteConfigType", - "default": "TestSuite" + "$ref": "#/definitions/applicationConfigType", + "default": "Application" }, - "entityFullyQualifiedName": { - "description": "Fully qualified name of the application to be executed", - "$ref": "../type/basic.json#/definitions/fullyQualifiedEntityName" + "appConfig": { + "$ref": "../type/basic.json#/definitions/componentConfig", + "description": "Application-specific configuration" } }, - "required": ["type", "entityFullyQualifiedName"], "additionalProperties": false } From e809619b56b68cdf5591333abd9617921a68dc6a Mon Sep 17 00:00:00 2001 From: Pere Miquel Brull Date: Fri, 3 Nov 2023 18:08:17 +0100 Subject: [PATCH 02/16] Prepare the skeleton for generic app registration --- ingestion/src/metadata/cli/app.py | 51 +++++++++++++++++++++++++++++++ 1 file changed, 51 insertions(+) create mode 100644 ingestion/src/metadata/cli/app.py diff --git a/ingestion/src/metadata/cli/app.py b/ingestion/src/metadata/cli/app.py new file mode 100644 index 000000000000..0baafa3f9d36 --- /dev/null +++ b/ingestion/src/metadata/cli/app.py @@ -0,0 +1,51 @@ +# 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. + +""" +Profiler utility for the metadata CLI +""" +import sys +import traceback +from pathlib import Path + +from metadata.config.common import load_config_file +from metadata.utils.logger import cli_logger +from metadata.workflow.application import ApplicationWorkflow +from metadata.workflow.workflow_output_handler import ( + WorkflowType, + print_init_error, + print_status, +) + +logger = cli_logger() + + +def run_app(config_path: Path) -> None: + """ + Run the application workflow from a config path + to a JSON or YAML file + :param config_path: Path to load JSON config + """ + + config_dict = None + try: + config_dict = load_config_file(config_path) + workflow = ApplicationWorkflow.create(config_dict) + logger.debug(f"Using config: {workflow.config}") + except Exception as exc: + logger.debug(traceback.format_exc()) + print_init_error(exc, config_dict, WorkflowType.INGEST) + sys.exit(1) + + workflow.execute() + workflow.stop() + print_status(workflow) + workflow.raise_from_status() From ed939a83315f419483fd3303a1aecc50015063b1 Mon Sep 17 00:00:00 2001 From: Pere Miquel Brull Date: Sun, 5 Nov 2023 11:25:21 +0100 Subject: [PATCH 03/16] Handle app runner --- .../src/metadata/applications/__init__.py | 0 .../src/metadata/applications/toy_app.py | 44 +++ ingestion/src/metadata/cli/app.py | 7 +- ingestion/src/metadata/cmd.py | 7 +- ingestion/src/metadata/utils/importer.py | 5 +- ingestion/src/metadata/utils/logger.py | 9 + .../src/metadata/workflow/application.py | 102 ++++++- .../workflow/application_output_handler.py | 34 +++ ingestion/src/metadata/workflow/base.py | 32 ++- .../src/metadata/workflow/data_insight.py | 1 - .../src/metadata/workflow/data_quality.py | 2 +- ingestion/src/metadata/workflow/ingestion.py | 36 ++- ingestion/src/metadata/workflow/metadata.py | 1 - .../src/metadata/workflow/output_handler.py | 249 +++++++++++++++++ ingestion/src/metadata/workflow/profiler.py | 1 - ingestion/src/metadata/workflow/usage.py | 1 - .../workflow/workflow_output_handler.py | 256 +----------------- .../workflow/workflow_status_mixin.py | 33 +-- .../openmetadata_managed_apis/utils/logger.py | 8 +- .../workflows/ingestion/application.py | 25 +- .../workflows/ingestion/common.py | 5 +- .../workflows/ingestion/registry.py | 4 +- .../schema/metadataIngestion/application.json | 4 + 23 files changed, 542 insertions(+), 324 deletions(-) create mode 100644 ingestion/src/metadata/applications/__init__.py create mode 100644 ingestion/src/metadata/applications/toy_app.py create mode 100644 ingestion/src/metadata/workflow/application_output_handler.py create mode 100644 ingestion/src/metadata/workflow/output_handler.py diff --git a/ingestion/src/metadata/applications/__init__.py b/ingestion/src/metadata/applications/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/ingestion/src/metadata/applications/toy_app.py b/ingestion/src/metadata/applications/toy_app.py new file mode 100644 index 000000000000..82a5bb6f49f2 --- /dev/null +++ b/ingestion/src/metadata/applications/toy_app.py @@ -0,0 +1,44 @@ +# 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. +""" +Toy to test applications +""" +from metadata.utils.logger import app_logger +from metadata.workflow.application import AppRunner + +logger = app_logger() + + +class ToyApp(AppRunner): + """ + Toy Application + You can execute it with `metadata app -c ` + with a YAML file like: + + sourcePythonClass: metadata.applications.toy_app.ToyApp + config: + key: value + workflowConfig: + loggerLevel: INFO + openMetadataServerConfig: + hostPort: http://localhost:8585/api + authProvider: openmetadata + securityConfig: + jwtToken: "..." + """ + + def run(self) -> None: + """Test run""" + + logger.info(f"I am {__name__} running with {self.config}.") + + def close(self) -> None: + """Nothing to close""" diff --git a/ingestion/src/metadata/cli/app.py b/ingestion/src/metadata/cli/app.py index 0baafa3f9d36..ac7f88b4aee8 100644 --- a/ingestion/src/metadata/cli/app.py +++ b/ingestion/src/metadata/cli/app.py @@ -19,11 +19,7 @@ from metadata.config.common import load_config_file from metadata.utils.logger import cli_logger from metadata.workflow.application import ApplicationWorkflow -from metadata.workflow.workflow_output_handler import ( - WorkflowType, - print_init_error, - print_status, -) +from metadata.workflow.application_output_handler import print_status logger = cli_logger() @@ -42,7 +38,6 @@ def run_app(config_path: Path) -> None: logger.debug(f"Using config: {workflow.config}") except Exception as exc: logger.debug(traceback.format_exc()) - print_init_error(exc, config_dict, WorkflowType.INGEST) sys.exit(1) workflow.execute() diff --git a/ingestion/src/metadata/cmd.py b/ingestion/src/metadata/cmd.py index 23d4be4f5f03..95c52dce690a 100644 --- a/ingestion/src/metadata/cmd.py +++ b/ingestion/src/metadata/cmd.py @@ -369,7 +369,8 @@ def get_parser(args=None): ) create_common_config_parser_args( sub_parser.add_parser( - MetadataCommands.APP.value, help="Workflow for running external applications" + MetadataCommands.APP.value, + help="Workflow for running external applications", ) ) create_openmetadata_imports_migration_args( @@ -426,7 +427,9 @@ def metadata(args=None): # pylint: disable=too-many-branches if config_file: path = Path(config_file).expanduser() else: - raise ValueError("Could not load config file! Please specify the config path with `-c` or `--config`.") + raise ValueError( + "Could not load config file! Please specify the config path with `-c` or `--config`." + ) if contains_args.get("debug"): set_loggers_level(logging.DEBUG) elif contains_args.get("log_level"): diff --git a/ingestion/src/metadata/utils/importer.py b/ingestion/src/metadata/utils/importer.py index da3c5ecbf16e..09dac6f5caa3 100644 --- a/ingestion/src/metadata/utils/importer.py +++ b/ingestion/src/metadata/utils/importer.py @@ -14,7 +14,7 @@ import importlib import traceback from enum import Enum -from typing import Callable, Optional, Type, TypeVar +from typing import Any, Callable, Optional, Type, TypeVar from pydantic import BaseModel @@ -23,7 +23,6 @@ ) from metadata.generated.schema.entity.services.serviceType import ServiceType from metadata.generated.schema.metadataIngestion.workflow import Sink as WorkflowSink -from metadata.ingestion.api.step import Step from metadata.ingestion.api.steps import BulkSink, Processor, Sink, Source, Stage from metadata.utils.class_helper import get_service_type_from_source_type from metadata.utils.logger import utils_logger @@ -86,7 +85,7 @@ def get_class_name_root(type_: str) -> str: ) -def import_from_module(key: str) -> Type[Step]: +def import_from_module(key: str) -> Type[Any]: """ Dynamically import an object from a module path """ diff --git a/ingestion/src/metadata/utils/logger.py b/ingestion/src/metadata/utils/logger.py index 6d5d38d471ff..05e72487b231 100644 --- a/ingestion/src/metadata/utils/logger.py +++ b/ingestion/src/metadata/utils/logger.py @@ -52,6 +52,7 @@ class Loggers(Enum): TEST_SUITE = "TestSuite" DATA_INSIGHT = "DataInsight" QUERY_RUNNER = "QueryRunner" + APP = "App" @DynamicClassAttribute def value(self): @@ -151,6 +152,14 @@ def great_expectations_logger(): return logging.getLogger(Loggers.GREAT_EXPECTATIONS.value) +def app_logger(): + """ + Method to get the UTILS logger + """ + + return logging.getLogger(Loggers.APP.value) + + def query_runner_logger(): """ Method to get the QUERY_RUNNER logger diff --git a/ingestion/src/metadata/workflow/application.py b/ingestion/src/metadata/workflow/application.py index d629737407c0..9efc33bbb187 100644 --- a/ingestion/src/metadata/workflow/application.py +++ b/ingestion/src/metadata/workflow/application.py @@ -12,33 +12,63 @@ Generic Workflow entrypoint to execute Applications """ from abc import ABC, abstractmethod -from typing import TypeVar, Generic - -from metadata.generated.schema.entity.services.connections.metadata.openMetadataConnection import OpenMetadataConnection -from metadata.generated.schema.metadataIngestion.application import OpenMetadataApplicationConfig -from metadata.generated.schema.metadataIngestion.workflow import LogLevels +from typing import Any, Dict, List, Optional from build.lib.metadata.generated.schema.entity.services.serviceType import ServiceType +from metadata.config.common import WorkflowExecutionError +from metadata.generated.schema.entity.services.connections.metadata.openMetadataConnection import ( + OpenMetadataConnection, +) +from metadata.generated.schema.metadataIngestion.application import ( + OpenMetadataApplicationConfig, +) +from metadata.generated.schema.metadataIngestion.workflow import LogLevels +from metadata.ingestion.api.models import StackTraceError +from metadata.ingestion.api.step import Step +from metadata.ingestion.ometa.ometa_api import OpenMetadata +from metadata.utils.importer import import_from_module +from metadata.utils.logger import ingestion_logger from metadata.workflow.base import BaseWorkflow +from metadata.workflow.workflow_status_mixin import SUCCESS_THRESHOLD_VALUE + +logger = ingestion_logger() + + +class AppRunner(Step, ABC): + """Class that knows how to execute the Application logic.""" -# Configuration class -C = TypeVar("C") + def __init__(self, config: Optional[Dict[str, Any]], metadata: OpenMetadata): + self.config = config + self.metadata = metadata + + super().__init__() + + @abstractmethod + def run(self) -> None: + """App logic to execute""" + + @classmethod + def create(cls, config_dict: dict, metadata: OpenMetadata) -> "Step": + return cls(config=config_dict, metadata=metadata) -class ApplicationWorkflow(BaseWorkflow, ABC, Generic[C]): - """ - Base Application Workflow implementation - """ +class ApplicationWorkflow(BaseWorkflow, ABC): + """Base Application Workflow implementation""" + config: OpenMetadataApplicationConfig + runner: Optional[AppRunner] def __init__(self, config_dict: dict): + self.runner = None # Will be passed in post-init self.config = OpenMetadataApplicationConfig.parse_obj(config_dict) # Applications are associated to the OpenMetadata Service self.service_type: ServiceType = ServiceType.Metadata - metadata_config: OpenMetadataConnection = self.config.workflowConfig.openMetadataServerConfig + metadata_config: OpenMetadataConnection = ( + self.config.workflowConfig.openMetadataServerConfig + ) log_level: LogLevels = self.config.workflowConfig.loggerLevel super().__init__( @@ -52,10 +82,52 @@ def __init__(self, config_dict: dict): def create(cls, config_dict: dict): return cls(config_dict) - @abstractmethod def post_init(self) -> None: - """Method to execute after we have initialized all the internals""" + """ + Method to execute after we have initialized all the internals. + Here we will load the runner since it needs the `metadata` object + """ + runner_class = import_from_module(self.config.sourcePythonClass) + if not issubclass(runner_class, AppRunner): + raise ValueError( + "We need a valid AppRunner to initialize the ApplicationWorkflow!" + ) + + try: + self.runner = runner_class( + config=self.config.config.__root__ if self.config.config else None, + metadata=self.metadata, + ) + except Exception as exc: + logger.error( + f"Error trying to init the AppRunner [{self.config.sourcePythonClass}] due to [{exc}]" + ) + raise exc - @abstractmethod def execute_internal(self) -> None: """Workflow-specific logic to execute safely""" + self.runner.run() + + def calculate_success(self) -> float: + return self.runner.get_status().calculate_success() + + def get_failures(self) -> List[StackTraceError]: + return self.source.get_status().failures + + def workflow_steps(self) -> List[Step]: + return [self.runner] + + def raise_from_status_internal(self, raise_warnings=False): + """Check failed status in the runner""" + if ( + self.runner.get_status().failures + and self.calculate_success() < SUCCESS_THRESHOLD_VALUE + ): + raise WorkflowExecutionError( + "Source reported errors", self.source.get_status() + ) + + if raise_warnings and self.runner.get_status().warnings: + raise WorkflowExecutionError( + "Runner reported warnings", self.runner.get_status() + ) diff --git a/ingestion/src/metadata/workflow/application_output_handler.py b/ingestion/src/metadata/workflow/application_output_handler.py new file mode 100644 index 000000000000..e29c60ec5bb5 --- /dev/null +++ b/ingestion/src/metadata/workflow/application_output_handler.py @@ -0,0 +1,34 @@ +# 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. + +""" +Module handles the output messages for applications +""" +import time + +from metadata.utils.helpers import pretty_print_time_duration +from metadata.utils.logger import ANSI, log_ansi_encoded_string +from metadata.workflow.output_handler import print_workflow_summary + + +def print_status(workflow: "ApplicationWorkflow") -> None: + """ + Print the workflow results + """ + print_workflow_summary(workflow) + + if workflow.runner.get_status().source_start_time: + log_ansi_encoded_string( + color=ANSI.BRIGHT_CYAN, + bold=True, + message="Workflow finished in time: " + f"{pretty_print_time_duration(time.time()-workflow.runner.get_status().source_start_time)}", + ) diff --git a/ingestion/src/metadata/workflow/base.py b/ingestion/src/metadata/workflow/base.py index 11aa16a97a11..dd9b32e5c389 100644 --- a/ingestion/src/metadata/workflow/base.py +++ b/ingestion/src/metadata/workflow/base.py @@ -15,9 +15,7 @@ import uuid from abc import ABC, abstractmethod from datetime import datetime -from typing import Dict, Optional, TypeVar, Union, Any - -from pydantic import BaseModel +from typing import Any, Dict, List, Optional, TypeVar, Union from metadata.generated.schema.api.services.ingestionPipelines.createIngestionPipeline import ( CreateIngestionPipelineRequest, @@ -33,6 +31,8 @@ from metadata.generated.schema.metadataIngestion.workflow import LogLevels from metadata.generated.schema.tests.testSuite import ServiceType from metadata.generated.schema.type.entityReference import EntityReference +from metadata.ingestion.api.models import StackTraceError +from metadata.ingestion.api.step import Step from metadata.ingestion.ometa.client_utils import create_ometa_client from metadata.ingestion.ometa.ometa_api import OpenMetadata from metadata.timer.repeated_timer import RepeatedTimer @@ -44,7 +44,7 @@ ) from metadata.utils.helpers import datetime_to_ts from metadata.utils.logger import ingestion_logger, set_loggers_level -from metadata.workflow.workflow_output_handler import get_ingestion_status_timer +from metadata.workflow.output_handler import report_ingestion_status from metadata.workflow.workflow_status_mixin import WorkflowStatusMixin logger = ingestion_logger() @@ -113,7 +113,7 @@ def stop(self) -> None: self.timer.stop() self.metadata.close() - for step in self.steps: + for step in self.workflow_steps(): try: step.close() except Exception as exc: @@ -125,8 +125,8 @@ def timer(self) -> RepeatedTimer: Status timer: It will print the source & sink status every `interval` seconds. """ if not self._timer: - self._timer = get_ingestion_status_timer( - interval=REPORTS_INTERVAL_SECONDS, logger=logger, workflow=self + self._timer = RepeatedTimer( + REPORTS_INTERVAL_SECONDS, report_ingestion_status, logger, self ) return self._timer @@ -144,6 +144,22 @@ def post_init(self) -> None: def execute_internal(self) -> None: """Workflow-specific logic to execute safely""" + @abstractmethod + def calculate_success(self) -> float: + """Get the success % of the internal execution""" + + @abstractmethod + def get_failures(self) -> List[StackTraceError]: + """Get the failures to flag whether if the workflow succeeded or not""" + + @abstractmethod + def workflow_steps(self) -> List[Step]: + """Steps to report status from""" + + @abstractmethod + def raise_from_status_internal(self, raise_warnings=False) -> None: + """Based on the internal workflow status, raise a WorkflowExecutionError""" + def execute(self) -> None: """ Main entrypoint @@ -153,7 +169,7 @@ def execute(self) -> None: self.execute_internal() # If we reach this point, compute the success % and update the associated Ingestion Pipeline status - self.update_ingestion_status_at_end() + self.update_pipeline_status_at_end() # Any unhandled exception breaking the workflow should update the status except Exception as err: diff --git a/ingestion/src/metadata/workflow/data_insight.py b/ingestion/src/metadata/workflow/data_insight.py index ea0b459caf00..1d598fffa5d9 100644 --- a/ingestion/src/metadata/workflow/data_insight.py +++ b/ingestion/src/metadata/workflow/data_insight.py @@ -19,7 +19,6 @@ from metadata.utils.importer import import_sink_class from metadata.utils.logger import data_insight_logger from metadata.utils.time_utils import get_beginning_of_day_timestamp_mill -from metadata.workflow.base import BaseWorkflow from metadata.workflow.ingestion import IngestionWorkflow logger = data_insight_logger() diff --git a/ingestion/src/metadata/workflow/data_quality.py b/ingestion/src/metadata/workflow/data_quality.py index 4ec1036178f7..4de7f7920bfa 100644 --- a/ingestion/src/metadata/workflow/data_quality.py +++ b/ingestion/src/metadata/workflow/data_quality.py @@ -25,7 +25,7 @@ from metadata.utils import fqn from metadata.utils.importer import import_sink_class from metadata.utils.logger import test_suite_logger -from metadata.workflow.base import BaseWorkflow, T +from metadata.workflow.base import T from metadata.workflow.ingestion import IngestionWorkflow logger = test_suite_logger() diff --git a/ingestion/src/metadata/workflow/ingestion.py b/ingestion/src/metadata/workflow/ingestion.py index f087b12c7638..8a18e82c7280 100644 --- a/ingestion/src/metadata/workflow/ingestion.py +++ b/ingestion/src/metadata/workflow/ingestion.py @@ -21,8 +21,9 @@ """ import traceback from abc import ABC, abstractmethod -from typing import Tuple, cast +from typing import List, Tuple, cast +from metadata.config.common import WorkflowExecutionError from metadata.generated.schema.entity.services.connections.metadata.openMetadataConnection import ( OpenMetadataConnection, ) @@ -33,6 +34,7 @@ from metadata.generated.schema.metadataIngestion.workflow import ( OpenMetadataWorkflowConfig, ) +from metadata.ingestion.api.models import StackTraceError from metadata.ingestion.api.parser import parse_workflow_config_gracefully from metadata.ingestion.api.step import Step from metadata.ingestion.api.steps import BulkSink, Processor, Sink, Source, Stage @@ -43,6 +45,7 @@ ) from metadata.utils.logger import ingestion_logger from metadata.workflow.base import BaseWorkflow, InvalidWorkflowJSONException +from metadata.workflow.workflow_status_mixin import SUCCESS_THRESHOLD_VALUE logger = ingestion_logger() @@ -124,6 +127,37 @@ def execute_internal(self): if bulk_sink: bulk_sink.run() + def calculate_success(self) -> float: + return self.source.get_status().calculate_success() + + def get_failures(self) -> List[StackTraceError]: + return self.source.get_status().failures + + def workflow_steps(self) -> List[Step]: + return [self.source] + list(self.steps) + + def raise_from_status_internal(self, raise_warnings=False): + """ + Check the status of all steps + """ + if ( + self.source.get_status().failures + and self.calculate_success() < SUCCESS_THRESHOLD_VALUE + ): + raise WorkflowExecutionError( + "Source reported errors", self.source.get_status() + ) + + for step in self.steps: + if step.status.failures: + raise WorkflowExecutionError( + f"{step.__class__.__name__} reported errors", step.get_status() + ) + if raise_warnings and step.status.warnings: + raise WorkflowExecutionError( + f"{step.__class__.__name__} reported warnings", step.get_status() + ) + def _retrieve_service_connection_if_needed(self, service_type: ServiceType) -> None: """ We override the current `serviceConnection` source config object if source workflow service already exists diff --git a/ingestion/src/metadata/workflow/metadata.py b/ingestion/src/metadata/workflow/metadata.py index f92823f601b5..3d221c3519c9 100644 --- a/ingestion/src/metadata/workflow/metadata.py +++ b/ingestion/src/metadata/workflow/metadata.py @@ -19,7 +19,6 @@ import_source_class, ) from metadata.utils.logger import ingestion_logger -from metadata.workflow.base import BaseWorkflow from metadata.workflow.ingestion import IngestionWorkflow logger = ingestion_logger() diff --git a/ingestion/src/metadata/workflow/output_handler.py b/ingestion/src/metadata/workflow/output_handler.py new file mode 100644 index 000000000000..2b242b717a51 --- /dev/null +++ b/ingestion/src/metadata/workflow/output_handler.py @@ -0,0 +1,249 @@ +# 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. + +""" +Common Output Handling methods +""" +import traceback +from enum import Enum +from logging import Logger +from pathlib import Path +from typing import Dict, List + +from pydantic import BaseModel +from tabulate import tabulate + +from metadata.generated.schema.metadataIngestion.workflow import LogLevels +from metadata.ingestion.api.models import StackTraceError +from metadata.ingestion.api.status import Status +from metadata.ingestion.api.step import Step +from metadata.ingestion.api.steps import BulkSink, Processor, Sink, Source, Stage +from metadata.utils.logger import ANSI, log_ansi_encoded_string + +WORKFLOW_FAILURE_MESSAGE = "Workflow finished with failures" +WORKFLOW_WARNING_MESSAGE = "Workflow finished with warnings" +WORKFLOW_SUCCESS_MESSAGE = "Workflow finished successfully" + + +class Failure(BaseModel): + """ + Auxiliary class to print the error per status + """ + + name: str + failures: List[StackTraceError] + + +class Summary(BaseModel): + """ + Auxiliary class to calculate the summary of all statuses + """ + + records = 0 + warnings = 0 + errors = 0 + filtered = 0 + + def __add__(self, other): + self.records += other.records + self.warnings += other.warnings + self.errors += other.errors + self.filtered += other.filtered + return self + + +class WorkflowType(Enum): + """ + Workflow type enums + """ + + INGEST = "ingest" + PROFILE = "profile" + TEST = "test" + LINEAGE = "lineage" + USAGE = "usage" + INSIGHT = "insight" + APP = "application" + + +EXAMPLES_WORKFLOW_PATH: Path = Path(__file__).parent / "../examples" / "workflows" + +URLS = { + WorkflowType.INGEST: "https://docs.open-metadata.org/connectors/ingestion/workflows/metadata", + WorkflowType.PROFILE: "https://docs.open-metadata.org/connectors/ingestion/workflows/profiler", + WorkflowType.TEST: "https://docs.open-metadata.org/connectors/ingestion/workflows/data-quality", + WorkflowType.LINEAGE: "https://docs.open-metadata.org/connectors/ingestion/workflows/lineage", + WorkflowType.USAGE: "https://docs.open-metadata.org/connectors/ingestion/workflows/usage", +} + +DEFAULT_EXAMPLE_FILE = { + WorkflowType.INGEST: "bigquery", + WorkflowType.PROFILE: "bigquery_profiler", + WorkflowType.TEST: "test_suite", + WorkflowType.LINEAGE: "bigquery_lineage", + WorkflowType.USAGE: "bigquery_usage", +} + + +def print_more_info(workflow_type: WorkflowType) -> None: + """ + Print more information message + """ + log_ansi_encoded_string( + message=f"\nFor more information, please visit: {URLS[workflow_type]}" + "\nOr join us in Slack: https://slack.open-metadata.org/" + ) + + +def print_error_msg(msg: str) -> None: + """ + Print message with error style + """ + log_ansi_encoded_string(color=ANSI.BRIGHT_RED, bold=False, message=f"{msg}") + + +def get_summary(status: Status) -> Summary: + records = len(status.records) + warnings = len(status.warnings) + errors = len(status.failures) + filtered = len(status.filtered) + return Summary(records=records, warnings=warnings, errors=errors, filtered=filtered) + + +def get_failures(failure: Failure) -> List[Dict[str, str]]: + return [ + { + "From": failure.name, + "Entity Name": f.name, + "Message": f.error, + "Stack Trace": f.stack_trace, + } + for f in failure.failures + ] + + +def print_failures_if_apply(failures: List[Failure]) -> None: + # take only the ones that contain failures + failures = [f for f in failures if f.failures] + if failures: + # create a list of dictionaries' list + all_data = [get_failures(failure) for failure in failures] + # create a single of dictionaries + data = [f for fs in all_data for f in fs] + # create a dictionary with a key and a list of values from the list + error_table = {k: [dic[k] for dic in data] for k in data[0]} + if len(list(error_table.items())[0][1]) > 100: + log_ansi_encoded_string( + bold=True, message="Showing only the first 100 failures:" + ) + # truncate list if number of values are over 100 + error_table = {k: v[:100] for k, v in error_table.items()} + else: + log_ansi_encoded_string(bold=True, message="List of failures:") + + log_ansi_encoded_string( + message=f"\n{tabulate(error_table, headers='keys', tablefmt='grid')}" + ) + + +def get_generic_step_name(step: Step) -> str: + """ + Since we cannot directly log the step name + as step.__class__.__name__ since it brings too + much internal info (e.g., MetadataRestSink), we'll + just check here for the simplification. + """ + for step_type in (Source, Processor, Stage, Sink, BulkSink): + if isinstance(step, step_type): + return step_type.__name__ + + return type(step).__name__ + + +def is_debug_enabled(workflow) -> bool: + return ( + hasattr(workflow, "config") + and hasattr(workflow.config, "workflowConfig") + and hasattr(workflow.config.workflowConfig, "loggerLevel") + and workflow.config.workflowConfig.loggerLevel is LogLevels.DEBUG + ) + + +def print_workflow_summary(workflow: "BaseWorkflow") -> None: + """ + Args: + workflow: the workflow status to be printed + + Returns: + Print Workflow status when the workflow logger level is DEBUG + """ + + if is_debug_enabled(workflow): + print_workflow_status_debug(workflow) + + failures = [] + total_records = 0 + total_errors = 0 + for step in workflow.workflow_steps(): + step_summary = get_summary(step.get_status()) + total_records += step_summary.records + total_errors += step_summary.errors + failures.append( + Failure( + name=get_generic_step_name(step), failures=step.get_status().failures + ) + ) + + log_ansi_encoded_string( + bold=True, message=f"Workflow {get_generic_step_name(step)} Summary:" + ) + log_ansi_encoded_string(message=f"Processed records: {step_summary.records}") + log_ansi_encoded_string(message=f"Warnings: {step_summary.warnings}") + if isinstance(step, Source): + log_ansi_encoded_string(message=f"Filtered: {step_summary.filtered}") + log_ansi_encoded_string(message=f"Errors: {step_summary.errors}") + + print_failures_if_apply(failures) + + total_success = max(total_records, 1) + log_ansi_encoded_string( + color=ANSI.BRIGHT_CYAN, + bold=True, + message=f"Success %: " + f"{round(total_success * 100 / (total_success + total_errors), 2)}", + ) + + +def print_workflow_status_debug(workflow: "BaseWorkflow") -> None: + """Print the statuses from each workflow step""" + log_ansi_encoded_string(bold=True, message="Statuses detailed info:") + for step in workflow.workflow_steps(): + log_ansi_encoded_string( + bold=True, message=f"{get_generic_step_name(step)} Status:" + ) + log_ansi_encoded_string(message=step.get_status().as_string()) + + +def report_ingestion_status(logger: Logger, workflow: "BaseWorkflow") -> None: + """ + Given a logger, use it to INFO the workflow status + """ + try: + for step in workflow.workflow_steps(): + logger.info( + f"{get_generic_step_name(step)}: Processed {len(step.status.records)} records," + f" filtered {len(step.status.filtered)} records," + f" found {len(step.status.failures)} errors" + ) + + except Exception as exc: + logger.debug(traceback.format_exc()) + logger.error(f"Wild exception reporting status - {exc}") diff --git a/ingestion/src/metadata/workflow/profiler.py b/ingestion/src/metadata/workflow/profiler.py index b5b1b9d80390..7e886c3b6971 100644 --- a/ingestion/src/metadata/workflow/profiler.py +++ b/ingestion/src/metadata/workflow/profiler.py @@ -21,7 +21,6 @@ from metadata.profiler.source.metadata import OpenMetadataSource from metadata.utils.importer import import_sink_class from metadata.utils.logger import profiler_logger -from metadata.workflow.base import BaseWorkflow from metadata.workflow.ingestion import IngestionWorkflow logger = profiler_logger() diff --git a/ingestion/src/metadata/workflow/usage.py b/ingestion/src/metadata/workflow/usage.py index 5db1adcdc2c1..152fc67d1616 100644 --- a/ingestion/src/metadata/workflow/usage.py +++ b/ingestion/src/metadata/workflow/usage.py @@ -21,7 +21,6 @@ import_stage_class, ) from metadata.utils.logger import ingestion_logger -from metadata.workflow.base import BaseWorkflow from metadata.workflow.ingestion import IngestionWorkflow logger = ingestion_logger() diff --git a/ingestion/src/metadata/workflow/workflow_output_handler.py b/ingestion/src/metadata/workflow/workflow_output_handler.py index 302316d8b024..7461054b0e9c 100644 --- a/ingestion/src/metadata/workflow/workflow_output_handler.py +++ b/ingestion/src/metadata/workflow/workflow_output_handler.py @@ -14,109 +14,27 @@ """ import time -import traceback -from enum import Enum -from logging import Logger -from pathlib import Path -from typing import Dict, List, Type, Union - -from pydantic import BaseModel -from tabulate import tabulate +from typing import Type, Union from metadata.config.common import ConfigurationError -from metadata.generated.schema.metadataIngestion.workflow import LogLevels -from metadata.ingestion.api.models import StackTraceError from metadata.ingestion.api.parser import ( InvalidWorkflowException, ParsingConfigurationError, ) -from metadata.ingestion.api.status import Status -from metadata.ingestion.api.step import Step -from metadata.ingestion.api.steps import BulkSink, Processor, Sink, Source, Stage -from metadata.timer.repeated_timer import RepeatedTimer from metadata.utils.constants import UTF_8 from metadata.utils.helpers import pretty_print_time_duration from metadata.utils.logger import ANSI, log_ansi_encoded_string - -WORKFLOW_FAILURE_MESSAGE = "Workflow finished with failures" -WORKFLOW_WARNING_MESSAGE = "Workflow finished with warnings" -WORKFLOW_SUCCESS_MESSAGE = "Workflow finished successfully" - - -class Failure(BaseModel): - """ - Auxiliary class to print the error per status - """ - - name: str - failures: List[StackTraceError] - - -class Summary(BaseModel): - """ - Auxiliary class to calculate the summary of all statuses - """ - - records = 0 - warnings = 0 - errors = 0 - filtered = 0 - - def __add__(self, other): - self.records += other.records - self.warnings += other.warnings - self.errors += other.errors - self.filtered += other.filtered - return self - - -class WorkflowType(Enum): - """ - Workflow type enums - """ - - INGEST = "ingest" - PROFILE = "profile" - TEST = "test" - LINEAGE = "lineage" - USAGE = "usage" - INSIGHT = "insight" - - -EXAMPLES_WORKFLOW_PATH: Path = Path(__file__).parent / "../examples" / "workflows" - -URLS = { - WorkflowType.INGEST: "https://docs.open-metadata.org/connectors/ingestion/workflows/metadata", - WorkflowType.PROFILE: "https://docs.open-metadata.org/connectors/ingestion/workflows/profiler", - WorkflowType.TEST: "https://docs.open-metadata.org/connectors/ingestion/workflows/data-quality", - WorkflowType.LINEAGE: "https://docs.open-metadata.org/connectors/ingestion/workflows/lineage", - WorkflowType.USAGE: "https://docs.open-metadata.org/connectors/ingestion/workflows/usage", -} - -DEFAULT_EXAMPLE_FILE = { - WorkflowType.INGEST: "bigquery", - WorkflowType.PROFILE: "bigquery_profiler", - WorkflowType.TEST: "test_suite", - WorkflowType.LINEAGE: "bigquery_lineage", - WorkflowType.USAGE: "bigquery_usage", -} - - -def print_more_info(workflow_type: WorkflowType) -> None: - """ - Print more information message - """ - log_ansi_encoded_string( - message=f"\nFor more information, please visit: {URLS[workflow_type]}" - "\nOr join us in Slack: https://slack.open-metadata.org/" - ) - - -def print_error_msg(msg: str) -> None: - """ - Print message with error style - """ - log_ansi_encoded_string(color=ANSI.BRIGHT_RED, bold=False, message=f"{msg}") +from metadata.workflow.output_handler import ( + DEFAULT_EXAMPLE_FILE, + EXAMPLES_WORKFLOW_PATH, + WORKFLOW_FAILURE_MESSAGE, + WORKFLOW_SUCCESS_MESSAGE, + WORKFLOW_WARNING_MESSAGE, + WorkflowType, + print_error_msg, + print_more_info, + print_workflow_summary, +) def calculate_ingestion_type(source_type_name: str) -> WorkflowType: @@ -197,7 +115,7 @@ def print_init_error( print_more_info(workflow_type) -def print_status(workflow: "BaseWorkflow") -> None: +def print_status(workflow: "IngestionWorkflow") -> None: """ Print the workflow results """ @@ -228,151 +146,3 @@ def print_status(workflow: "BaseWorkflow") -> None: log_ansi_encoded_string( color=ANSI.GREEN, bold=True, message=WORKFLOW_SUCCESS_MESSAGE ) - - -def is_debug_enabled(workflow) -> bool: - return ( - hasattr(workflow, "config") - and hasattr(workflow.config, "workflowConfig") - and hasattr(workflow.config.workflowConfig, "loggerLevel") - and workflow.config.workflowConfig.loggerLevel is LogLevels.DEBUG - ) - - -def get_generic_step_name(step: Step) -> str: - """ - Since we cannot directly log the step name - as step.__class__.__name__ since it brings too - much internal info (e.g., MetadataRestSink), we'll - just check here for the simplification. - """ - for step_type in (Source, Processor, Stage, Sink, BulkSink): - if isinstance(step, step_type): - return step_type.__name__ - - return type(step).__name__ - - -def print_workflow_summary(workflow: "BaseWorkflow") -> None: - """ - Args: - workflow: the workflow status to be printed - - Returns: - Print Workflow status when the workflow logger level is DEBUG - """ - - if is_debug_enabled(workflow): - print_workflow_status_debug(workflow) - - failures = [] - total_records = 0 - total_errors = 0 - for step in [workflow.source] + list(workflow.steps): - step_summary = get_summary(step.get_status()) - total_records += step_summary.records - total_errors += step_summary.errors - failures.append( - Failure( - name=get_generic_step_name(step), failures=step.get_status().failures - ) - ) - - log_ansi_encoded_string( - bold=True, message=f"Workflow {get_generic_step_name(step)} Summary:" - ) - log_ansi_encoded_string(message=f"Processed records: {step_summary.records}") - log_ansi_encoded_string(message=f"Warnings: {step_summary.warnings}") - if isinstance(step, Source): - log_ansi_encoded_string(message=f"Filtered: {step_summary.filtered}") - log_ansi_encoded_string(message=f"Errors: {step_summary.errors}") - - print_failures_if_apply(failures) - - total_success = max(total_records, 1) - log_ansi_encoded_string( - color=ANSI.BRIGHT_CYAN, - bold=True, - message=f"Success %: " - f"{round(total_success * 100 / (total_success + total_errors), 2)}", - ) - - -def print_workflow_status_debug(workflow: "BaseWorkflow") -> None: - """Print the statuses from each workflow step""" - log_ansi_encoded_string(bold=True, message="Statuses detailed info:") - for step in [workflow.source] + list(workflow.steps): - log_ansi_encoded_string( - bold=True, message=f"{get_generic_step_name(step)} Status:" - ) - log_ansi_encoded_string(message=step.get_status().as_string()) - - -def get_summary(status: Status) -> Summary: - records = len(status.records) - warnings = len(status.warnings) - errors = len(status.failures) - filtered = len(status.filtered) - return Summary(records=records, warnings=warnings, errors=errors, filtered=filtered) - - -def get_failures(failure: Failure) -> List[Dict[str, str]]: - return [ - { - "From": failure.name, - "Entity Name": f.name, - "Message": f.error, - "Stack Trace": f.stack_trace, - } - for f in failure.failures - ] - - -def print_failures_if_apply(failures: List[Failure]) -> None: - # take only the ones that contain failures - failures = [f for f in failures if f.failures] - if failures: - # create a list of dictionaries' list - all_data = [get_failures(failure) for failure in failures] - # create a single of dictionaries - data = [f for fs in all_data for f in fs] - # create a dictionary with a key and a list of values from the list - error_table = {k: [dic[k] for dic in data] for k in data[0]} - if len(list(error_table.items())[0][1]) > 100: - log_ansi_encoded_string( - bold=True, message="Showing only the first 100 failures:" - ) - # truncate list if number of values are over 100 - error_table = {k: v[:100] for k, v in error_table.items()} - else: - log_ansi_encoded_string(bold=True, message="List of failures:") - - log_ansi_encoded_string( - message=f"\n{tabulate(error_table, headers='keys', tablefmt='grid')}" - ) - - -def report_ingestion_status(logger: Logger, workflow: "BaseWorkflow") -> None: - """ - Given a logger, use it to INFO the workflow status - """ - try: - for step in [workflow.source] + list(workflow.steps): - logger.info( - f"{get_generic_step_name(step)}: Processed {len(step.status.records)} records," - f" filtered {len(step.status.filtered)} records," - f" found {len(step.status.failures)} errors" - ) - - except Exception as exc: - logger.debug(traceback.format_exc()) - logger.error(f"Wild exception reporting status - {exc}") - - -def get_ingestion_status_timer( - interval: int, logger: Logger, workflow: "BaseWorkflow" -) -> RepeatedTimer: - """ - Prepare the threading Timer to execute the report_ingestion_status - """ - return RepeatedTimer(interval, report_ingestion_status, logger, workflow) diff --git a/ingestion/src/metadata/workflow/workflow_status_mixin.py b/ingestion/src/metadata/workflow/workflow_status_mixin.py index 5928957a5cca..0a43a14ac3cf 100644 --- a/ingestion/src/metadata/workflow/workflow_status_mixin.py +++ b/ingestion/src/metadata/workflow/workflow_status_mixin.py @@ -64,28 +64,6 @@ def run_id(self) -> str: return self._run_id - def _raise_from_status_internal(self, raise_warnings=False): - """ - Check the status of all steps - """ - if ( - self.source.get_status().failures - and self._get_source_success() < SUCCESS_THRESHOLD_VALUE - ): - raise WorkflowExecutionError( - "Source reported errors", self.source.get_status() - ) - - for step in self.steps: - if step.status.failures: - raise WorkflowExecutionError( - f"{step.__class__.__name__} reported errors", step.get_status() - ) - if raise_warnings and step.status.warnings: - raise WorkflowExecutionError( - f"{step.__class__.__name__} reported warnings", step.get_status() - ) - def _new_pipeline_status(self, state: PipelineState) -> PipelineStatus: """Create new Pipeline Status""" return PipelineStatus( @@ -121,26 +99,23 @@ def set_ingestion_pipeline_status( self.config.ingestionPipelineFQN, pipeline_status ) - def update_ingestion_status_at_end(self): + def update_pipeline_status_at_end(self): """ Once the execute method is done, update the status as OK or KO depending on the success rate. """ pipeline_state = PipelineState.success - if SUCCESS_THRESHOLD_VALUE <= self._get_source_success() < 100: + if SUCCESS_THRESHOLD_VALUE <= self.calculate_success() < 100: pipeline_state = PipelineState.partialSuccess self.set_ingestion_pipeline_status(pipeline_state) - def _get_source_success(self): - return self.source.get_status().calculate_success() - def raise_from_status(self, raise_warnings=False): """ Method to raise error if failed execution and updating Ingestion Pipeline Status """ try: - self._raise_from_status_internal(raise_warnings) + self.raise_from_status_internal(raise_warnings) except WorkflowExecutionError as err: self.set_ingestion_pipeline_status(PipelineState.failed) raise err @@ -149,6 +124,6 @@ def result_status(self) -> int: """ Returns 1 if source status is failed, 0 otherwise. """ - if self.source.get_status().failures: + if self.get_failures(): return 1 return 0 diff --git a/openmetadata-airflow-apis/openmetadata_managed_apis/utils/logger.py b/openmetadata-airflow-apis/openmetadata_managed_apis/utils/logger.py index aba68cfae0bc..fdbd48ff6ff4 100644 --- a/openmetadata-airflow-apis/openmetadata_managed_apis/utils/logger.py +++ b/openmetadata-airflow-apis/openmetadata_managed_apis/utils/logger.py @@ -4,8 +4,10 @@ from typing import Union from airflow.configuration import conf -from metadata.generated.schema.metadataIngestion.application import OpenMetadataApplicationConfig +from metadata.generated.schema.metadataIngestion.application import ( + OpenMetadataApplicationConfig, +) from metadata.generated.schema.metadataIngestion.workflow import ( OpenMetadataWorkflowConfig, ) @@ -60,7 +62,9 @@ def utils_logger(): return build_logger(Loggers.UTILS.value) -def set_operator_logger(workflow_config: Union[OpenMetadataWorkflowConfig, OpenMetadataApplicationConfig]) -> None: +def set_operator_logger( + workflow_config: Union[OpenMetadataWorkflowConfig, OpenMetadataApplicationConfig] +) -> None: """ Handle logging for the Python Operator that will execute the ingestion diff --git a/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py b/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py index 739c8ea44872..5e2355664ccb 100644 --- a/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py +++ b/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py @@ -15,16 +15,25 @@ from typing import cast from airflow import DAG -from metadata.generated.schema.entity.services.ingestionPipelines.ingestionPipeline import IngestionPipeline -from metadata.generated.schema.metadataIngestion.application import OpenMetadataApplicationConfig -from metadata.generated.schema.metadataIngestion.applicationPipeline import ApplicationPipeline +from openmetadata_managed_apis.utils.logger import set_operator_logger +from openmetadata_managed_apis.workflows.ingestion.common import ( + build_dag, + build_workflow_config_property, +) + +from metadata.generated.schema.entity.services.ingestionPipelines.ingestionPipeline import ( + IngestionPipeline, +) +from metadata.generated.schema.metadataIngestion.application import ( + OpenMetadataApplicationConfig, +) +from metadata.generated.schema.metadataIngestion.applicationPipeline import ( + ApplicationPipeline, +) from metadata.ingestion.models.encoders import show_secrets_encoder from metadata.utils.workflow_output_handler import print_status from metadata.workflow.application import ApplicationWorkflow -from openmetadata_managed_apis.utils.logger import set_operator_logger -from openmetadata_managed_apis.workflows.ingestion.common import build_workflow_config_property, build_dag - def application_workflow(application_workflow_config: OpenMetadataApplicationConfig): """ @@ -55,7 +64,9 @@ def build_application_workflow_config( """ # Here we have an application pipeline, so the Source Config is of type ApplicationPipeline - application_pipeline_conf: ApplicationPipeline = cast(ingestion_pipeline.sourceConfig.config, ApplicationPipeline) + application_pipeline_conf: ApplicationPipeline = cast( + ingestion_pipeline.sourceConfig.config, ApplicationPipeline + ) application_workflow_config = OpenMetadataApplicationConfig( config=application_pipeline_conf.appConfig, diff --git a/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/common.py b/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/common.py index 87ab4bc5f467..764284c7ebc5 100644 --- a/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/common.py +++ b/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/common.py @@ -19,8 +19,6 @@ import airflow from airflow import DAG -from metadata.generated.schema.metadataIngestion.application import OpenMetadataApplicationConfig - from openmetadata_managed_apis.api.utils import clean_dag_id from pydantic import ValidationError from requests.utils import quote @@ -33,6 +31,9 @@ from metadata.generated.schema.entity.services.pipelineService import PipelineService from metadata.generated.schema.entity.services.searchService import SearchService from metadata.generated.schema.entity.services.storageService import StorageService +from metadata.generated.schema.metadataIngestion.application import ( + OpenMetadataApplicationConfig, +) from metadata.ingestion.models.encoders import show_secrets_encoder from metadata.ingestion.ometa.ometa_api import OpenMetadata from metadata.workflow.workflow_output_handler import print_status diff --git a/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/registry.py b/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/registry.py index 3ead53763969..3eb253b5f3bc 100644 --- a/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/registry.py +++ b/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/registry.py @@ -13,7 +13,9 @@ Add a function for each type from PipelineType """ -from openmetadata_managed_apis.workflows.ingestion.application import build_application_dag +from openmetadata_managed_apis.workflows.ingestion.application import ( + build_application_dag, +) from openmetadata_managed_apis.workflows.ingestion.data_insight import ( build_data_insight_dag, ) diff --git a/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/application.json b/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/application.json index 28ce20980bbc..1f940c4170d6 100644 --- a/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/application.json +++ b/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/application.json @@ -13,6 +13,10 @@ "$ref": "../type/basic.json#/definitions/componentConfig", "description": "Application-specific configuration" }, + "sourcePythonClass": { + "description": "Source Python Class Name to run the application", + "type": "string" + }, "ingestionPipelineFQN": { "description": "Fully qualified name of ingestion pipeline, used to identify the current ingestion pipeline", "type": "string" From 9ab5e8d57e1ff85efa4606cc7e81d8e7f49f6298 Mon Sep 17 00:00:00 2001 From: Pere Miquel Brull Date: Sun, 5 Nov 2023 17:19:34 +0100 Subject: [PATCH 04/16] Prepare the skeleton for generic app registration --- ingestion/src/metadata/applications/pii.py | 195 ++++++++++++++++++ .../src/metadata/applications/toy_app.py | 44 ---- .../src/metadata/workflow/application.py | 8 +- .../src/metadata/workflow/output_handler.py | 2 +- .../workflows/ingestion/application.py | 10 +- .../apps/bundles/pii/AutoPIIApplication.java | 122 +++++++++++ .../AutoPIIApplication.json | 28 +++ .../externalAppIngestionConfig.json | 2 +- .../externalApplicationConfig.json | 19 ++ .../schema/metadataIngestion/application.json | 12 +- .../applicationPipeline.json | 4 + 11 files changed, 387 insertions(+), 59 deletions(-) create mode 100644 ingestion/src/metadata/applications/pii.py delete mode 100644 ingestion/src/metadata/applications/toy_app.py create mode 100644 openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/pii/AutoPIIApplication.java create mode 100644 openmetadata-service/src/main/resources/json/data/appMarketPlaceDefinition/AutoPIIApplication.json create mode 100644 openmetadata-spec/src/main/resources/json/schema/entity/applications/configuration/externalApplicationConfig.json diff --git a/ingestion/src/metadata/applications/pii.py b/ingestion/src/metadata/applications/pii.py new file mode 100644 index 000000000000..89cb5c202868 --- /dev/null +++ b/ingestion/src/metadata/applications/pii.py @@ -0,0 +1,195 @@ +# 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. +""" +PII application +""" +import traceback +from typing import Any, Dict, Iterable, List, Optional + +from metadata.generated.schema.entity.data.table import Column, Table, TableData +from metadata.generated.schema.type.tagLabel import ( + LabelType, + State, + TagLabel, + TagSource, +) +from metadata.ingestion.api.models import StackTraceError +from metadata.ingestion.models.table_metadata import ColumnTag +from metadata.ingestion.ometa.ometa_api import OpenMetadata +from metadata.pii.constants import PII +from metadata.pii.scanners.column_name_scanner import ColumnNameScanner +from metadata.pii.scanners.ner_scanner import NERScanner +from metadata.utils.logger import app_logger +from metadata.workflow.application import AppRunner + +logger = app_logger() + +DEFAULT_CONFIDENCE = 80 + + +class PIIApp(AppRunner): + """ + PII Application + You can execute it with `metadata app -c ` + with a YAML file like: + + sourcePythonClass: metadata.applications.pii.PIIApp + config: + confidenceLevel: 80 + workflowConfig: + loggerLevel: INFO + openMetadataServerConfig: + hostPort: http://localhost:8585/api + authProvider: openmetadata + securityConfig: + jwtToken: "..." + """ + + def __init__(self, config: Optional[Dict[str, Any]], metadata: OpenMetadata): + super().__init__(config, metadata) + + self._ner_scanner = None + self.confidence_threshold = ( + config.get("confidenceLevel", DEFAULT_CONFIDENCE) + if config + else DEFAULT_CONFIDENCE + ) + + @staticmethod + def build_column_tag(tag_fqn: str, column_fqn: str) -> ColumnTag: + """ + Build the tag and run the PATCH + """ + tag_label = TagLabel( + tagFQN=tag_fqn, + source=TagSource.Classification, + state=State.Suggested, + labelType=LabelType.Automated, + ) + + return ColumnTag(column_fqn=column_fqn, tag_label=tag_label) + + @property + def ner_scanner(self) -> NERScanner: + """Load the NER Scanner only if called""" + if self._ner_scanner is None: + self._ner_scanner = NERScanner() + + return self._ner_scanner + + def process_column( + self, + idx: int, + column: Column, + table_data: Optional[TableData], + confidence_threshold: float, + ) -> Optional[List[ColumnTag]]: + """ + Tag a column with PII if we find it using our scanners + """ + + # First, check if the column we are about to process + # already has PII tags or not + column_has_pii_tag = any( + (PII in tag.tagFQN.__root__ for tag in column.tags or []) + ) + + # If it has PII tags, we skip the processing + # for the column + if column_has_pii_tag is True: + return None + + # Scan by column name. If no results there, check the sample data, if any + tag_and_confidence = ColumnNameScanner.scan(column.name.__root__) or ( + self.ner_scanner.scan([row[idx] for row in table_data.rows]) + if table_data + else None + ) + + if ( + tag_and_confidence + and tag_and_confidence.tag_fqn + and tag_and_confidence.confidence >= confidence_threshold / 100 + ): + # We support returning +1 tags for a single column in _run + return [ + self.build_column_tag( + tag_fqn=tag_and_confidence.tag_fqn, + column_fqn=column.fullyQualifiedName.__root__, + ) + ] + + return None + + def process_table(self, table: Table) -> Optional[List[ColumnTag]]: + """Run the patching of the table""" + column_tags = [] + for idx, column in enumerate(table.columns): + try: + col_tags = self.process_column( + idx=idx, + column=column, + table_data=table.sampleData, + confidence_threshold=self.confidence_threshold, + ) + if col_tags: + column_tags.extend(col_tags) + except Exception as err: + self.status.failed( + StackTraceError( + name=table.fullyQualifiedName.__root__, + error=f"Error computing PII tags for [{column}] - [{err}]", + stack_trace=traceback.format_exc(), + ) + ) + + if column_tags: + return column_tags + + return None + + def patch_columns(self, table: Table, column_tags: List[ColumnTag]) -> None: + """Patch columns with PII""" + patched = self.metadata.patch_column_tags(table=table, column_tags=column_tags) + if not patched: + self.status.failed( + StackTraceError( + name=table.fullyQualifiedName.__root__, + error="Error patching tags for table", + ) + ) + else: + logger.debug( + f"Successfully patched tag {column_tags} for {table.fullyQualifiedName.__root__}" + ) + + def run(self) -> None: + """ + The PII Application will: + 1. List tables + 2. Check their column names and sample data (if any) + 3. PATCH PII tags when needed + """ + tables: Iterable[Table] = self.metadata.list_all_entities( + entity=Table, fields=["sampleData"] + ) + for table in tables: + column_tags = self.process_table(table) + if column_tags: + self.status.scanned(table) + + else: + self.status.filter( + key=table.fullyQualifiedName.__root__, reason="No PII found" + ) + + def close(self) -> None: + """Nothing to close""" diff --git a/ingestion/src/metadata/applications/toy_app.py b/ingestion/src/metadata/applications/toy_app.py deleted file mode 100644 index 82a5bb6f49f2..000000000000 --- a/ingestion/src/metadata/applications/toy_app.py +++ /dev/null @@ -1,44 +0,0 @@ -# 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. -""" -Toy to test applications -""" -from metadata.utils.logger import app_logger -from metadata.workflow.application import AppRunner - -logger = app_logger() - - -class ToyApp(AppRunner): - """ - Toy Application - You can execute it with `metadata app -c ` - with a YAML file like: - - sourcePythonClass: metadata.applications.toy_app.ToyApp - config: - key: value - workflowConfig: - loggerLevel: INFO - openMetadataServerConfig: - hostPort: http://localhost:8585/api - authProvider: openmetadata - securityConfig: - jwtToken: "..." - """ - - def run(self) -> None: - """Test run""" - - logger.info(f"I am {__name__} running with {self.config}.") - - def close(self) -> None: - """Nothing to close""" diff --git a/ingestion/src/metadata/workflow/application.py b/ingestion/src/metadata/workflow/application.py index 9efc33bbb187..477370c7c7a9 100644 --- a/ingestion/src/metadata/workflow/application.py +++ b/ingestion/src/metadata/workflow/application.py @@ -87,7 +87,7 @@ def post_init(self) -> None: Method to execute after we have initialized all the internals. Here we will load the runner since it needs the `metadata` object """ - runner_class = import_from_module(self.config.sourcePythonClass) + runner_class = import_from_module(self.config.appConfig.sourcePythonClass) if not issubclass(runner_class, AppRunner): raise ValueError( "We need a valid AppRunner to initialize the ApplicationWorkflow!" @@ -95,12 +95,14 @@ def post_init(self) -> None: try: self.runner = runner_class( - config=self.config.config.__root__ if self.config.config else None, + config=self.config.appConfig.config.__root__ + if self.config.appConfig.config + else None, metadata=self.metadata, ) except Exception as exc: logger.error( - f"Error trying to init the AppRunner [{self.config.sourcePythonClass}] due to [{exc}]" + f"Error trying to init the AppRunner [{self.config.appConfig.sourcePythonClass}] due to [{exc}]" ) raise exc diff --git a/ingestion/src/metadata/workflow/output_handler.py b/ingestion/src/metadata/workflow/output_handler.py index 2b242b717a51..990bb32f2a84 100644 --- a/ingestion/src/metadata/workflow/output_handler.py +++ b/ingestion/src/metadata/workflow/output_handler.py @@ -207,7 +207,7 @@ def print_workflow_summary(workflow: "BaseWorkflow") -> None: ) log_ansi_encoded_string(message=f"Processed records: {step_summary.records}") log_ansi_encoded_string(message=f"Warnings: {step_summary.warnings}") - if isinstance(step, Source): + if step_summary.filtered: log_ansi_encoded_string(message=f"Filtered: {step_summary.filtered}") log_ansi_encoded_string(message=f"Errors: {step_summary.errors}") diff --git a/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py b/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py index 5e2355664ccb..a7d39886a6b3 100644 --- a/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py +++ b/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py @@ -21,6 +21,9 @@ build_workflow_config_property, ) +from metadata.generated.schema.entity.applications.configuration.externalApplicationConfig import ( + ExternalApplicationConfig, +) from metadata.generated.schema.entity.services.ingestionPipelines.ingestionPipeline import ( IngestionPipeline, ) @@ -31,8 +34,8 @@ ApplicationPipeline, ) from metadata.ingestion.models.encoders import show_secrets_encoder -from metadata.utils.workflow_output_handler import print_status from metadata.workflow.application import ApplicationWorkflow +from metadata.workflow.application_output_handler import print_status def application_workflow(application_workflow_config: OpenMetadataApplicationConfig): @@ -69,7 +72,10 @@ def build_application_workflow_config( ) application_workflow_config = OpenMetadataApplicationConfig( - config=application_pipeline_conf.appConfig, + appConfig=ExternalApplicationConfig( + sourcePythonClass=application_pipeline_conf.sourcePythonClass, + config=application_pipeline_conf.appConfig, + ), workflowConfig=build_workflow_config_property(ingestion_pipeline), ingestionPipelineFQN=ingestion_pipeline.fullyQualifiedName.__root__, ) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/pii/AutoPIIApplication.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/pii/AutoPIIApplication.java new file mode 100644 index 000000000000..efb8a836155f --- /dev/null +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/pii/AutoPIIApplication.java @@ -0,0 +1,122 @@ +package org.openmetadata.service.apps.bundles.pii; + +import com.cronutils.model.Cron; +import java.util.List; +import lombok.extern.slf4j.Slf4j; +import org.openmetadata.schema.api.services.ingestionPipelines.CreateIngestionPipeline; +import org.openmetadata.schema.entity.app.App; +import org.openmetadata.schema.entity.applications.configuration.ExternalApplicationConfig; +import org.openmetadata.schema.entity.services.ingestionPipelines.AirflowConfig; +import org.openmetadata.schema.entity.services.ingestionPipelines.IngestionPipeline; +import org.openmetadata.schema.entity.services.ingestionPipelines.PipelineType; +import org.openmetadata.schema.metadataIngestion.ApplicationPipeline; +import org.openmetadata.schema.metadataIngestion.SourceConfig; +import org.openmetadata.schema.type.EntityReference; +import org.openmetadata.schema.type.ProviderType; +import org.openmetadata.schema.type.Relationship; +import org.openmetadata.service.Entity; +import org.openmetadata.service.apps.AbstractNativeApplication; +import org.openmetadata.service.exception.EntityNotFoundException; +import org.openmetadata.service.jdbi3.CollectionDAO; +import org.openmetadata.service.jdbi3.IngestionPipelineRepository; +import org.openmetadata.service.jdbi3.MetadataServiceRepository; +import org.openmetadata.service.search.SearchRepository; +import org.openmetadata.service.util.FullyQualifiedName; +import org.openmetadata.service.util.JsonUtils; + +@Slf4j +public class AutoPIIApplication extends AbstractNativeApplication { + + private static final String INGESTION_PIPELINE_NAME = "metaPilotPipeline"; + private static final String SERVICE_NAME = "OpenMetadata"; + + @Override + public void init(App app, CollectionDAO dao, SearchRepository searchRepository) { + super.init(app, dao, searchRepository); + this.app = app; + LOG.info("Data Insights App is initialized"); + } + + /** + * MetaPilot is an external App that accepts one ApiKey parameter and runs a workflow based on it. + * + *

The App will register an IngestionPipeline against the OpenMetadata service with default daily scheduled + * configurations. + */ + @Override + public void initializeExternalApp() { + + ExternalApplicationConfig config = + JsonUtils.convertValue(app.getAppConfiguration(), ExternalApplicationConfig.class); + IngestionPipelineRepository ingestionPipelineRepository = + (IngestionPipelineRepository) Entity.getEntityRepository(Entity.INGESTION_PIPELINE); + + // Check if the Ingestion Pipeline has already been created + try { + String fqn = FullyQualifiedName.add(SERVICE_NAME, INGESTION_PIPELINE_NAME); + IngestionPipeline storedPipeline = + ingestionPipelineRepository.getByName(null, fqn, ingestionPipelineRepository.getFields("id")); + + // Init Application Code for Some Initialization + List records = + collectionDAO + .relationshipDAO() + .findTo(app.getId(), Entity.APPLICATION, Relationship.HAS.ordinal(), Entity.INGESTION_PIPELINE); + + if (records.isEmpty()) { + // Add Ingestion Pipeline to Application + collectionDAO + .relationshipDAO() + .insert( + app.getId(), + storedPipeline.getId(), + Entity.APPLICATION, + Entity.INGESTION_PIPELINE, + Relationship.HAS.ordinal()); + } + + // Otherwise, create it + } catch (EntityNotFoundException ex) { + MetadataServiceRepository serviceEntityRepository = + (MetadataServiceRepository) Entity.getEntityRepository(Entity.METADATA_SERVICE); + EntityReference service = + serviceEntityRepository + .getByName(null, SERVICE_NAME, serviceEntityRepository.getFields("id")) + .getEntityReference(); + + Cron quartzCron = cronParser.parse(app.getAppSchedule().getCronExpression()); + + CreateIngestionPipeline createPipelineRequest = + new CreateIngestionPipeline() + .withName(INGESTION_PIPELINE_NAME) + .withDisplayName(app.getDisplayName()) + .withDescription(app.getDescription()) + .withPipelineType(PipelineType.APPLICATION) + .withSourceConfig( + new SourceConfig() + .withConfig( + new ApplicationPipeline() + .withSourcePythonClass(config.getSourcePythonClass()) + .withAppConfig(config.getConfig()))) + .withAirflowConfig(new AirflowConfig().withScheduleInterval(cronMapper.map(quartzCron).asString())) + .withService(service); + + // Get Pipeline + IngestionPipeline dataInsightPipeline = + getIngestionPipeline(createPipelineRequest, String.format("%sBot", app.getName()), "admin") + .withProvider(ProviderType.USER); + ingestionPipelineRepository.setFullyQualifiedName(dataInsightPipeline); + ingestionPipelineRepository.initializeEntity(dataInsightPipeline); + + // Add Ingestion Pipeline to Application + collectionDAO + .relationshipDAO() + .insert( + app.getId(), + dataInsightPipeline.getId(), + Entity.APPLICATION, + Entity.INGESTION_PIPELINE, + Relationship.HAS.ordinal()); + } + } +} diff --git a/openmetadata-service/src/main/resources/json/data/appMarketPlaceDefinition/AutoPIIApplication.json b/openmetadata-service/src/main/resources/json/data/appMarketPlaceDefinition/AutoPIIApplication.json new file mode 100644 index 000000000000..a18750733ffe --- /dev/null +++ b/openmetadata-service/src/main/resources/json/data/appMarketPlaceDefinition/AutoPIIApplication.json @@ -0,0 +1,28 @@ +{ + "name": "AutoPIIApplication", + "displayName": "Auto PII", + "description": "Use NLP models to automatically tag PII information", + "features": "Auto tag PII information in your tables' columns", + "appType": "external", + "appScreenshots": ["AutoPIIApplication.png"], + "developer": "Collate Inc.", + "developerUrl": "https://www.getcollate.io", + "privacyPolicyUrl": "https://www.getcollate.io", + "supportEmail": "support@getcollate.io", + "scheduleType": "Scheduled", + "permission": "All", + "className": "org.openmetadata.service.apps.bundles.pii.AutoPIIApplication", + "runtime": { + "enabled": "true" + }, + "appConfiguration": { + "sourcePythonClass": "metadata.applications.pii.PIIApp", + "config": { + "confidenceLevel": 80 + } + }, + "appSchedule": { + "scheduleType": "Custom", + "cronExpression": "0 0 0 * * ?" + } +} \ No newline at end of file diff --git a/openmetadata-spec/src/main/resources/json/schema/entity/applications/configuration/externalAppIngestionConfig.json b/openmetadata-spec/src/main/resources/json/schema/entity/applications/configuration/externalAppIngestionConfig.json index b30c12aa402f..8f4282e713a2 100644 --- a/openmetadata-spec/src/main/resources/json/schema/entity/applications/configuration/externalAppIngestionConfig.json +++ b/openmetadata-spec/src/main/resources/json/schema/entity/applications/configuration/externalAppIngestionConfig.json @@ -2,7 +2,7 @@ "$id": "https://open-metadata.org/schema/entity/applications/configuration/externalAppIngestionConfig.json", "$schema": "http://json-schema.org/draft-07/schema#", "title": "ExternalAppIngestionConfig", - "description": "This schema defines External App Ingestion Config used for app working with Ingestion.", + "description": "This schema defines External App Ingestion Config used for app working with Ingestion. Only used for Data Insights.", "type": "object", "javaType": "org.openmetadata.schema.entity.app.ExternalAppIngestionConfig", "definitions": { diff --git a/openmetadata-spec/src/main/resources/json/schema/entity/applications/configuration/externalApplicationConfig.json b/openmetadata-spec/src/main/resources/json/schema/entity/applications/configuration/externalApplicationConfig.json new file mode 100644 index 000000000000..47bbe6a38f8a --- /dev/null +++ b/openmetadata-spec/src/main/resources/json/schema/entity/applications/configuration/externalApplicationConfig.json @@ -0,0 +1,19 @@ +{ + "$id": "https://open-metadata.org/schema/entity/applications/configuration/externalApplicationConfig.json", + "$schema": "http://json-schema.org/draft-07/schema#", + "title": "ExternalApplicationConfig", + "description": "External Application Pipeline Configuration.", + "type": "object", + "properties": { + "config": { + "$ref": "../../../type/basic.json#/definitions/componentConfig", + "description": "Application-specific configuration" + }, + "sourcePythonClass": { + "description": "Source Python Class Name to run the application", + "type": "string" + } + }, + "additionalProperties": false, + "required": ["sourcePythonClass"] +} diff --git a/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/application.json b/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/application.json index 1f940c4170d6..e5a67f97a2d9 100644 --- a/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/application.json +++ b/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/application.json @@ -2,20 +2,16 @@ "$id": "https://open-metadata.org/schema/metadataIngestion/application.json", "$schema": "http://json-schema.org/draft-07/schema#", "title": "OpenMetadataApplicationConfig", - "description": "OpenMetadata Application Workflow definition.", + "description": "OpenMetadata Ingestion Framework definition for Applications, i.e., the YAML shape we require.", "type": "object", "properties": { "workflowConfig": { "$ref": "workflow.json#/definitions/workflowConfig", "description": "General Workflow configuration, such as the OpenMetadata server connection and logging level" }, - "config": { - "$ref": "../type/basic.json#/definitions/componentConfig", - "description": "Application-specific configuration" - }, - "sourcePythonClass": { - "description": "Source Python Class Name to run the application", - "type": "string" + "appConfig": { + "$ref": "../entity/applications/configuration/externalApplicationConfig.json", + "description": "External Application configuration" }, "ingestionPipelineFQN": { "description": "Fully qualified name of ingestion pipeline, used to identify the current ingestion pipeline", diff --git a/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/applicationPipeline.json b/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/applicationPipeline.json index eb01454431f4..4ff088aff2f0 100644 --- a/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/applicationPipeline.json +++ b/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/applicationPipeline.json @@ -18,6 +18,10 @@ "$ref": "#/definitions/applicationConfigType", "default": "Application" }, + "sourcePythonClass": { + "description": "Source Python Class Name to run the application", + "type": "string" + }, "appConfig": { "$ref": "../type/basic.json#/definitions/componentConfig", "description": "Application-specific configuration" From dd9f8a61dbe3e7bdc58d1a770a605a9fdb58caf1 Mon Sep 17 00:00:00 2001 From: Pere Miquel Brull Date: Sun, 5 Nov 2023 17:42:02 +0100 Subject: [PATCH 05/16] Prepare the skeleton for generic app registration --- .../service/apps/bundles/pii/AutoPIIApplication.java | 2 +- .../data/appMarketPlaceDefinition/AutoPIIApplication.json | 8 +++----- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/pii/AutoPIIApplication.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/pii/AutoPIIApplication.java index efb8a836155f..efc8d75f1657 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/pii/AutoPIIApplication.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/pii/AutoPIIApplication.java @@ -27,7 +27,7 @@ @Slf4j public class AutoPIIApplication extends AbstractNativeApplication { - private static final String INGESTION_PIPELINE_NAME = "metaPilotPipeline"; + private static final String INGESTION_PIPELINE_NAME = "autoPIIPipeline"; private static final String SERVICE_NAME = "OpenMetadata"; @Override diff --git a/openmetadata-service/src/main/resources/json/data/appMarketPlaceDefinition/AutoPIIApplication.json b/openmetadata-service/src/main/resources/json/data/appMarketPlaceDefinition/AutoPIIApplication.json index a18750733ffe..708954916f6f 100644 --- a/openmetadata-service/src/main/resources/json/data/appMarketPlaceDefinition/AutoPIIApplication.json +++ b/openmetadata-service/src/main/resources/json/data/appMarketPlaceDefinition/AutoPIIApplication.json @@ -4,7 +4,9 @@ "description": "Use NLP models to automatically tag PII information", "features": "Auto tag PII information in your tables' columns", "appType": "external", - "appScreenshots": ["AutoPIIApplication.png"], + "appScreenshots": [ + "AutoPIIApplication.png" + ], "developer": "Collate Inc.", "developerUrl": "https://www.getcollate.io", "privacyPolicyUrl": "https://www.getcollate.io", @@ -20,9 +22,5 @@ "config": { "confidenceLevel": 80 } - }, - "appSchedule": { - "scheduleType": "Custom", - "cronExpression": "0 0 0 * * ?" } } \ No newline at end of file From 3d7aa7101f6edb9130292e24add9675e14f59576 Mon Sep 17 00:00:00 2001 From: Pere Miquel Brull Date: Mon, 6 Nov 2023 15:56:13 +0100 Subject: [PATCH 06/16] Allow deployment --- .../src/metadata/workflow/application.py | 2 +- .../workflows/ingestion/application.py | 10 ++++----- .../ui/src/assets/svg/AutoPIIApplication.svg | 22 +++++++++++++++++++ .../AutoPIIApplication.json | 19 ++++++++++++++++ 4 files changed, 46 insertions(+), 7 deletions(-) create mode 100644 openmetadata-ui/src/main/resources/ui/src/assets/svg/AutoPIIApplication.svg create mode 100644 openmetadata-ui/src/main/resources/ui/src/utils/ApplicationSchemas/AutoPIIApplication.json diff --git a/ingestion/src/metadata/workflow/application.py b/ingestion/src/metadata/workflow/application.py index 477370c7c7a9..48ad01a096e6 100644 --- a/ingestion/src/metadata/workflow/application.py +++ b/ingestion/src/metadata/workflow/application.py @@ -14,7 +14,7 @@ from abc import ABC, abstractmethod from typing import Any, Dict, List, Optional -from build.lib.metadata.generated.schema.entity.services.serviceType import ServiceType +from metadata.generated.schema.entity.services.serviceType import ServiceType from metadata.config.common import WorkflowExecutionError from metadata.generated.schema.entity.services.connections.metadata.openMetadataConnection import ( OpenMetadataConnection, diff --git a/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py b/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py index a7d39886a6b3..da45832b31ef 100644 --- a/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py +++ b/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py @@ -38,7 +38,7 @@ from metadata.workflow.application_output_handler import print_status -def application_workflow(application_workflow_config: OpenMetadataApplicationConfig): +def application_workflow(workflow_config: OpenMetadataApplicationConfig): """ Task that creates and runs the ingestion workflow. @@ -48,9 +48,9 @@ def application_workflow(application_workflow_config: OpenMetadataApplicationCon This is the callable used to create the PythonOperator """ - set_operator_logger(application_workflow_config) + set_operator_logger(workflow_config) - config = json.loads(application_workflow_config.json(encoder=show_secrets_encoder)) + config = json.loads(workflow_config.json(encoder=show_secrets_encoder)) workflow = ApplicationWorkflow.create(config) workflow.execute() @@ -67,9 +67,7 @@ def build_application_workflow_config( """ # Here we have an application pipeline, so the Source Config is of type ApplicationPipeline - application_pipeline_conf: ApplicationPipeline = cast( - ingestion_pipeline.sourceConfig.config, ApplicationPipeline - ) + application_pipeline_conf: ApplicationPipeline = ingestion_pipeline.sourceConfig.config application_workflow_config = OpenMetadataApplicationConfig( appConfig=ExternalApplicationConfig( diff --git a/openmetadata-ui/src/main/resources/ui/src/assets/svg/AutoPIIApplication.svg b/openmetadata-ui/src/main/resources/ui/src/assets/svg/AutoPIIApplication.svg new file mode 100644 index 000000000000..0ed0c62850cc --- /dev/null +++ b/openmetadata-ui/src/main/resources/ui/src/assets/svg/AutoPIIApplication.svg @@ -0,0 +1,22 @@ + + + + + + + + + + + + + + + + + + + + + + diff --git a/openmetadata-ui/src/main/resources/ui/src/utils/ApplicationSchemas/AutoPIIApplication.json b/openmetadata-ui/src/main/resources/ui/src/utils/ApplicationSchemas/AutoPIIApplication.json new file mode 100644 index 000000000000..47bbe6a38f8a --- /dev/null +++ b/openmetadata-ui/src/main/resources/ui/src/utils/ApplicationSchemas/AutoPIIApplication.json @@ -0,0 +1,19 @@ +{ + "$id": "https://open-metadata.org/schema/entity/applications/configuration/externalApplicationConfig.json", + "$schema": "http://json-schema.org/draft-07/schema#", + "title": "ExternalApplicationConfig", + "description": "External Application Pipeline Configuration.", + "type": "object", + "properties": { + "config": { + "$ref": "../../../type/basic.json#/definitions/componentConfig", + "description": "Application-specific configuration" + }, + "sourcePythonClass": { + "description": "Source Python Class Name to run the application", + "type": "string" + } + }, + "additionalProperties": false, + "required": ["sourcePythonClass"] +} From ffd0b9f19e14528795af36e7b1da83cc33d7b7ea Mon Sep 17 00:00:00 2001 From: Pere Miquel Brull Date: Mon, 6 Nov 2023 18:00:40 +0100 Subject: [PATCH 07/16] Fix PII APP --- ingestion/src/metadata/applications/pii.py | 6 +++--- ingestion/src/metadata/workflow/application.py | 2 +- .../workflows/ingestion/application.py | 5 +++-- 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/ingestion/src/metadata/applications/pii.py b/ingestion/src/metadata/applications/pii.py index 89cb5c202868..221d1da633e2 100644 --- a/ingestion/src/metadata/applications/pii.py +++ b/ingestion/src/metadata/applications/pii.py @@ -167,6 +167,7 @@ def patch_columns(self, table: Table, column_tags: List[ColumnTag]) -> None: ) ) else: + self.status.scanned(table) logger.debug( f"Successfully patched tag {column_tags} for {table.fullyQualifiedName.__root__}" ) @@ -179,13 +180,12 @@ def run(self) -> None: 3. PATCH PII tags when needed """ tables: Iterable[Table] = self.metadata.list_all_entities( - entity=Table, fields=["sampleData"] + entity=Table, fields=["sampleData", "tags"] ) for table in tables: column_tags = self.process_table(table) if column_tags: - self.status.scanned(table) - + self.patch_columns(table=table, column_tags=column_tags) else: self.status.filter( key=table.fullyQualifiedName.__root__, reason="No PII found" diff --git a/ingestion/src/metadata/workflow/application.py b/ingestion/src/metadata/workflow/application.py index 48ad01a096e6..5e1656c06b61 100644 --- a/ingestion/src/metadata/workflow/application.py +++ b/ingestion/src/metadata/workflow/application.py @@ -14,11 +14,11 @@ from abc import ABC, abstractmethod from typing import Any, Dict, List, Optional -from metadata.generated.schema.entity.services.serviceType import ServiceType from metadata.config.common import WorkflowExecutionError from metadata.generated.schema.entity.services.connections.metadata.openMetadataConnection import ( OpenMetadataConnection, ) +from metadata.generated.schema.entity.services.serviceType import ServiceType from metadata.generated.schema.metadataIngestion.application import ( OpenMetadataApplicationConfig, ) diff --git a/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py b/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py index da45832b31ef..d4f7e6d4aba5 100644 --- a/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py +++ b/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py @@ -12,7 +12,6 @@ Generic Workflow entrypoint to execute Applications """ import json -from typing import cast from airflow import DAG from openmetadata_managed_apis.utils.logger import set_operator_logger @@ -67,7 +66,9 @@ def build_application_workflow_config( """ # Here we have an application pipeline, so the Source Config is of type ApplicationPipeline - application_pipeline_conf: ApplicationPipeline = ingestion_pipeline.sourceConfig.config + application_pipeline_conf: ApplicationPipeline = ( + ingestion_pipeline.sourceConfig.config + ) application_workflow_config = OpenMetadataApplicationConfig( appConfig=ExternalApplicationConfig( From 19e60c5a3501a840eef67bea62868a0cafcb4cb4 Mon Sep 17 00:00:00 2001 From: Pere Miquel Brull Date: Tue, 7 Nov 2023 08:00:19 +0100 Subject: [PATCH 08/16] Fix lint --- ingestion/Makefile | 2 +- ingestion/src/metadata/cli/app.py | 1 + ingestion/src/metadata/cmd.py | 28 +++++++++---------- .../src/metadata/workflow/data_insight.py | 10 ++++++- .../tests/unit/workflow/test_base_workflow.py | 6 ++-- 5 files changed, 28 insertions(+), 19 deletions(-) diff --git a/ingestion/Makefile b/ingestion/Makefile index 80d88dac1ce6..54141ed3e747 100644 --- a/ingestion/Makefile +++ b/ingestion/Makefile @@ -27,7 +27,7 @@ install_all: ## Install the ingestion module with all dependencies .PHONY: install_apis install_apis: ## Install the REST APIs module to the current environment - python -m pip install $(ROOT_DIR)openmetadata-airflow-apis/ + python -m pip install $(ROOT_DIR)/openmetadata-airflow-apis/ .PHONY: lint lint: ## Run pylint on the Python sources to analyze the codebase diff --git a/ingestion/src/metadata/cli/app.py b/ingestion/src/metadata/cli/app.py index ac7f88b4aee8..7cf3f7c0d52c 100644 --- a/ingestion/src/metadata/cli/app.py +++ b/ingestion/src/metadata/cli/app.py @@ -37,6 +37,7 @@ def run_app(config_path: Path) -> None: workflow = ApplicationWorkflow.create(config_dict) logger.debug(f"Using config: {workflow.config}") except Exception as exc: + logger.error(f"Error running the application {exc}") logger.debug(traceback.format_exc()) sys.exit(1) diff --git a/ingestion/src/metadata/cmd.py b/ingestion/src/metadata/cmd.py index 95c52dce690a..e50819fb6d41 100644 --- a/ingestion/src/metadata/cmd.py +++ b/ingestion/src/metadata/cmd.py @@ -56,6 +56,17 @@ class MetadataCommands(Enum): OPENMETADATA_DAG_CONFIG_MIGRATION = "openmetadata_dag_config_migration" +RUN_PATH_METHODS = { + MetadataCommands.INGEST.value: run_ingest, + MetadataCommands.USAGE.value: run_usage, + MetadataCommands.LINEAGE.value: run_lineage, + MetadataCommands.INSIGHT.value: run_insight, + MetadataCommands.PROFILE.value: run_profiler, + MetadataCommands.TEST.value: run_test, + MetadataCommands.APP.value: run_app, +} + + OM_IMPORTS_MIGRATION = """ Update DAG files generated after creating workflow in 0.11 and before. In 0.12 the airflow managed API package name changed from `openmetadata` to @@ -437,20 +448,9 @@ def metadata(args=None): # pylint: disable=too-many-branches else: set_loggers_level(logging.INFO) - if metadata_workflow == MetadataCommands.INGEST.value: - run_ingest(config_path=path) - if metadata_workflow == MetadataCommands.USAGE.value: - run_usage(config_path=path) - if metadata_workflow == MetadataCommands.LINEAGE.value: - run_lineage(config_path=path) - if metadata_workflow == MetadataCommands.INSIGHT.value: - run_insight(config_path=path) - if metadata_workflow == MetadataCommands.PROFILE.value: - run_profiler(config_path=path) - if metadata_workflow == MetadataCommands.TEST.value: - run_test(config_path=path) - if metadata_workflow == MetadataCommands.APP.value: - run_app(config_path=path) + if metadata_workflow in RUN_PATH_METHODS: + RUN_PATH_METHODS[metadata_workflow](path) + if metadata_workflow == MetadataCommands.BACKUP.value: run_backup( common_backup_obj_instance=BackupRestoreArgs( diff --git a/ingestion/src/metadata/workflow/data_insight.py b/ingestion/src/metadata/workflow/data_insight.py index 1d598fffa5d9..b9f88ae54efb 100644 --- a/ingestion/src/metadata/workflow/data_insight.py +++ b/ingestion/src/metadata/workflow/data_insight.py @@ -14,6 +14,9 @@ from metadata.data_insight.processor.kpi.kpi_runner import KpiRunner from metadata.data_insight.source.metadata import DataInsightSource from metadata.generated.schema.analytics.basic import WebAnalyticEventType +from metadata.generated.schema.metadataIngestion.workflow import ( + OpenMetadataWorkflowConfig, +) from metadata.generated.schema.tests.testSuite import ServiceType from metadata.ingestion.api.steps import Sink from metadata.utils.importer import import_sink_class @@ -29,6 +32,11 @@ class DataInsightWorkflow(IngestionWorkflow): retention_days = 7 + def __init__(self, config: OpenMetadataWorkflowConfig): + super().__init__(config) + + self.sink = None + def _retrieve_service_connection_if_needed(self, service_type: ServiceType) -> None: """No service connection needed for data insight""" return None @@ -58,7 +66,7 @@ def _get_sink(self) -> Sink: """Retrieve sink for data insight workflow""" sink_type = "metadata-rest" sink_class = import_sink_class(sink_type=sink_type) - sink_config = {"api_endpoint": self.metadata_config.hostPort} + sink_config = {"api_endpoint": self.metadata.config.hostPort} sink: Sink = sink_class.create(sink_config, self.metadata) logger.debug(f"Sink type:{self.config.sink.type}, {sink_class} configured") diff --git a/ingestion/tests/unit/workflow/test_base_workflow.py b/ingestion/tests/unit/workflow/test_base_workflow.py index 06c844fdaa94..0273c67311cc 100644 --- a/ingestion/tests/unit/workflow/test_base_workflow.py +++ b/ingestion/tests/unit/workflow/test_base_workflow.py @@ -36,7 +36,7 @@ from metadata.ingestion.api.step import Step from metadata.ingestion.api.steps import Sink from metadata.ingestion.api.steps import Source as WorkflowSource -from metadata.workflow.base import BaseWorkflow +from metadata.workflow.ingestion import IngestionWorkflow class SimpleSource(WorkflowSource): @@ -104,7 +104,7 @@ def close(self) -> None: """Nothing to do""" -class SimpleWorkflow(BaseWorkflow): +class SimpleWorkflow(IngestionWorkflow): """ Simple Workflow for testing """ @@ -115,7 +115,7 @@ def set_steps(self): self.steps: Tuple[Step] = (SimpleSink(),) -class BrokenWorkflow(BaseWorkflow): +class BrokenWorkflow(IngestionWorkflow): """ Simple Workflow for testing """ From 94df5afedfee9c69c068fece019e7b1ded24bb46 Mon Sep 17 00:00:00 2001 From: Pere Miquel Brull Date: Tue, 7 Nov 2023 08:46:52 +0100 Subject: [PATCH 09/16] Fix PII APP --- ingestion/src/metadata/workflow/base.py | 2 ++ ingestion/src/metadata/workflow/data_insight.py | 7 +++---- ingestion/tests/unit/profiler/test_workflow.py | 1 + 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/ingestion/src/metadata/workflow/base.py b/ingestion/src/metadata/workflow/base.py index dd9b32e5c389..eb337e77c6f8 100644 --- a/ingestion/src/metadata/workflow/base.py +++ b/ingestion/src/metadata/workflow/base.py @@ -69,6 +69,7 @@ class BaseWorkflow(ABC, WorkflowStatusMixin): config: Union[Any, Dict] _run_id: Optional[str] = None metadata: OpenMetadata + metadata_config: OpenMetadataConnection service_type: ServiceType def __init__( @@ -90,6 +91,7 @@ def __init__( set_loggers_level(log_level.value) # We create the ometa client at the workflow level and pass it to the steps + self.metadata_config = metadata_config self.metadata = create_ometa_client(metadata_config) self.set_ingestion_pipeline_status(state=PipelineState.running) diff --git a/ingestion/src/metadata/workflow/data_insight.py b/ingestion/src/metadata/workflow/data_insight.py index b9f88ae54efb..b8068f0b284f 100644 --- a/ingestion/src/metadata/workflow/data_insight.py +++ b/ingestion/src/metadata/workflow/data_insight.py @@ -11,12 +11,11 @@ """ Workflow definition for the data insight """ +from metadata.generated.schema.metadataIngestion.workflow import OpenMetadataWorkflowConfig + from metadata.data_insight.processor.kpi.kpi_runner import KpiRunner from metadata.data_insight.source.metadata import DataInsightSource from metadata.generated.schema.analytics.basic import WebAnalyticEventType -from metadata.generated.schema.metadataIngestion.workflow import ( - OpenMetadataWorkflowConfig, -) from metadata.generated.schema.tests.testSuite import ServiceType from metadata.ingestion.api.steps import Sink from metadata.utils.importer import import_sink_class @@ -66,7 +65,7 @@ def _get_sink(self) -> Sink: """Retrieve sink for data insight workflow""" sink_type = "metadata-rest" sink_class = import_sink_class(sink_type=sink_type) - sink_config = {"api_endpoint": self.metadata.config.hostPort} + sink_config = {"api_endpoint": self.metadata_config.hostPort} sink: Sink = sink_class.create(sink_config, self.metadata) logger.debug(f"Sink type:{self.config.sink.type}, {sink_class} configured") diff --git a/ingestion/tests/unit/profiler/test_workflow.py b/ingestion/tests/unit/profiler/test_workflow.py index f03cd07d93a8..4d4dac766224 100644 --- a/ingestion/tests/unit/profiler/test_workflow.py +++ b/ingestion/tests/unit/profiler/test_workflow.py @@ -17,6 +17,7 @@ from unittest.mock import patch import sqlalchemy as sqa +from metadata.ingestion.ometa.ometa_api import OpenMetadata from pytest import raises from sqlalchemy.orm import declarative_base From 9605a3534919042436ff23b70a9543bce54c8f77 Mon Sep 17 00:00:00 2001 From: Pere Miquel Brull Date: Tue, 7 Nov 2023 09:04:55 +0100 Subject: [PATCH 10/16] Fix PII APP --- ingestion/src/metadata/workflow/data_insight.py | 5 +++-- ingestion/tests/unit/profiler/test_workflow.py | 1 - 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/ingestion/src/metadata/workflow/data_insight.py b/ingestion/src/metadata/workflow/data_insight.py index b8068f0b284f..3ddcdd4e0d44 100644 --- a/ingestion/src/metadata/workflow/data_insight.py +++ b/ingestion/src/metadata/workflow/data_insight.py @@ -11,11 +11,12 @@ """ Workflow definition for the data insight """ -from metadata.generated.schema.metadataIngestion.workflow import OpenMetadataWorkflowConfig - from metadata.data_insight.processor.kpi.kpi_runner import KpiRunner from metadata.data_insight.source.metadata import DataInsightSource from metadata.generated.schema.analytics.basic import WebAnalyticEventType +from metadata.generated.schema.metadataIngestion.workflow import ( + OpenMetadataWorkflowConfig, +) from metadata.generated.schema.tests.testSuite import ServiceType from metadata.ingestion.api.steps import Sink from metadata.utils.importer import import_sink_class diff --git a/ingestion/tests/unit/profiler/test_workflow.py b/ingestion/tests/unit/profiler/test_workflow.py index 4d4dac766224..f03cd07d93a8 100644 --- a/ingestion/tests/unit/profiler/test_workflow.py +++ b/ingestion/tests/unit/profiler/test_workflow.py @@ -17,7 +17,6 @@ from unittest.mock import patch import sqlalchemy as sqa -from metadata.ingestion.ometa.ometa_api import OpenMetadata from pytest import raises from sqlalchemy.orm import declarative_base From 76f30f4df3a344f1321fc68f08a4125df90d70af Mon Sep 17 00:00:00 2001 From: Pere Miquel Brull Date: Wed, 8 Nov 2023 09:58:08 +0100 Subject: [PATCH 11/16] Prepare config-based external apps --- .../applications/{pii.py => auto_tagger.py} | 20 ++++---- .../src/metadata/workflow/application.py | 17 ++++--- .../workflows/ingestion/application.py | 9 ++-- .../test_workflow_creation.py | 47 ++++++++++++++++++- ...n.java => ExternalApplicationHandler.java} | 15 +++--- .../apps/AppMarketPlaceResource.java | 3 +- .../service/resources/apps/AppResource.java | 3 +- ...cation.json => AutoTaggerApplication.json} | 12 ++--- .../json/schema/entity/applications/app.json | 6 ++- .../external/autoTaggerAppConfig.json | 30 ++++++++++++ .../externalApplicationConfig.json | 21 ++++----- .../marketplace/appMarketPlaceDefinition.json | 4 ++ .../createAppMarketPlaceDefinitionReq.json | 4 ++ .../schema/metadataIngestion/application.json | 6 ++- .../applicationPipeline.json | 4 +- ...lication.svg => AutoTaggerApplication.svg} | 0 16 files changed, 147 insertions(+), 54 deletions(-) rename ingestion/src/metadata/applications/{pii.py => auto_tagger.py} (92%) rename openmetadata-service/src/main/java/org/openmetadata/service/apps/{bundles/pii/AutoPIIApplication.java => ExternalApplicationHandler.java} (90%) rename openmetadata-service/src/main/resources/json/data/appMarketPlaceDefinition/{AutoPIIApplication.json => AutoTaggerApplication.json} (58%) create mode 100644 openmetadata-spec/src/main/resources/json/schema/entity/applications/configuration/external/autoTaggerAppConfig.json rename openmetadata-ui/src/main/resources/ui/src/assets/svg/{AutoPIIApplication.svg => AutoTaggerApplication.svg} (100%) diff --git a/ingestion/src/metadata/applications/pii.py b/ingestion/src/metadata/applications/auto_tagger.py similarity index 92% rename from ingestion/src/metadata/applications/pii.py rename to ingestion/src/metadata/applications/auto_tagger.py index 221d1da633e2..1080fabce234 100644 --- a/ingestion/src/metadata/applications/pii.py +++ b/ingestion/src/metadata/applications/auto_tagger.py @@ -12,12 +12,16 @@ PII application """ import traceback -from typing import Any, Dict, Iterable, List, Optional +from typing import Iterable, List, Optional +from metadata.generated.schema.entity.applications.configuration.external.autoTaggerAppConfig import ( + AutoTaggerAppConfig, +) from metadata.generated.schema.entity.data.table import Column, Table, TableData from metadata.generated.schema.type.tagLabel import ( LabelType, State, + TagFQN, TagLabel, TagSource, ) @@ -35,13 +39,13 @@ DEFAULT_CONFIDENCE = 80 -class PIIApp(AppRunner): +class AutoTaggerApp(AppRunner): """ PII Application You can execute it with `metadata app -c ` with a YAML file like: - sourcePythonClass: metadata.applications.pii.PIIApp + sourcePythonClass: metadata.applications.auto_tagger.AutoTaggerApp config: confidenceLevel: 80 workflowConfig: @@ -53,15 +57,11 @@ class PIIApp(AppRunner): jwtToken: "..." """ - def __init__(self, config: Optional[Dict[str, Any]], metadata: OpenMetadata): + def __init__(self, config: AutoTaggerAppConfig, metadata: OpenMetadata): super().__init__(config, metadata) self._ner_scanner = None - self.confidence_threshold = ( - config.get("confidenceLevel", DEFAULT_CONFIDENCE) - if config - else DEFAULT_CONFIDENCE - ) + self.confidence_threshold = config.confidenceLevel or DEFAULT_CONFIDENCE @staticmethod def build_column_tag(tag_fqn: str, column_fqn: str) -> ColumnTag: @@ -69,7 +69,7 @@ def build_column_tag(tag_fqn: str, column_fqn: str) -> ColumnTag: Build the tag and run the PATCH """ tag_label = TagLabel( - tagFQN=tag_fqn, + tagFQN=TagFQN(__root__=tag_fqn), source=TagSource.Classification, state=State.Suggested, labelType=LabelType.Automated, diff --git a/ingestion/src/metadata/workflow/application.py b/ingestion/src/metadata/workflow/application.py index 5e1656c06b61..96126b60eb1b 100644 --- a/ingestion/src/metadata/workflow/application.py +++ b/ingestion/src/metadata/workflow/application.py @@ -12,9 +12,12 @@ Generic Workflow entrypoint to execute Applications """ from abc import ABC, abstractmethod -from typing import Any, Dict, List, Optional +from typing import List, Optional from metadata.config.common import WorkflowExecutionError +from metadata.generated.schema.entity.applications.configuration.externalApplicationConfig import ( + AppConfig, +) from metadata.generated.schema.entity.services.connections.metadata.openMetadataConnection import ( OpenMetadataConnection, ) @@ -37,7 +40,9 @@ class AppRunner(Step, ABC): """Class that knows how to execute the Application logic.""" - def __init__(self, config: Optional[Dict[str, Any]], metadata: OpenMetadata): + def __init__( + self, config: AppConfig.__fields__["__root__"].type_, metadata: OpenMetadata + ): self.config = config self.metadata = metadata @@ -87,7 +92,7 @@ def post_init(self) -> None: Method to execute after we have initialized all the internals. Here we will load the runner since it needs the `metadata` object """ - runner_class = import_from_module(self.config.appConfig.sourcePythonClass) + runner_class = import_from_module(self.config.sourcePythonClass) if not issubclass(runner_class, AppRunner): raise ValueError( "We need a valid AppRunner to initialize the ApplicationWorkflow!" @@ -95,14 +100,14 @@ def post_init(self) -> None: try: self.runner = runner_class( - config=self.config.appConfig.config.__root__ - if self.config.appConfig.config + config=self.config.appConfig.__root__ + if self.config.appConfig else None, metadata=self.metadata, ) except Exception as exc: logger.error( - f"Error trying to init the AppRunner [{self.config.appConfig.sourcePythonClass}] due to [{exc}]" + f"Error trying to init the AppRunner [{self.config.sourcePythonClass}] due to [{exc}]" ) raise exc diff --git a/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py b/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py index d4f7e6d4aba5..dc19438447d0 100644 --- a/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py +++ b/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py @@ -21,7 +21,7 @@ ) from metadata.generated.schema.entity.applications.configuration.externalApplicationConfig import ( - ExternalApplicationConfig, + ExternalApplicationConfig, AppConfig, ) from metadata.generated.schema.entity.services.ingestionPipelines.ingestionPipeline import ( IngestionPipeline, @@ -71,9 +71,10 @@ def build_application_workflow_config( ) application_workflow_config = OpenMetadataApplicationConfig( - appConfig=ExternalApplicationConfig( - sourcePythonClass=application_pipeline_conf.sourcePythonClass, - config=application_pipeline_conf.appConfig, + sourcePythonClass=application_pipeline_conf.sourcePythonClass, + # We pass the generic class and let each app cast the actual object + appConfig=AppConfig( + __root__=application_pipeline_conf.appConfig.__root__, ), workflowConfig=build_workflow_config_property(ingestion_pipeline), ingestionPipelineFQN=ingestion_pipeline.fullyQualifiedName.__root__, diff --git a/openmetadata-airflow-apis/tests/unit/ingestion_pipeline/test_workflow_creation.py b/openmetadata-airflow-apis/tests/unit/ingestion_pipeline/test_workflow_creation.py index cf9be2d7863e..37f37df2b77e 100644 --- a/openmetadata-airflow-apis/tests/unit/ingestion_pipeline/test_workflow_creation.py +++ b/openmetadata-airflow-apis/tests/unit/ingestion_pipeline/test_workflow_creation.py @@ -17,6 +17,12 @@ from unittest import TestCase from unittest.mock import patch +from metadata.generated.schema.entity.applications.configuration.external.autoTaggerAppConfig import AutoTaggerAppConfig +from metadata.generated.schema.entity.applications.configuration.externalApplicationConfig import AppConfig +from metadata.generated.schema.metadataIngestion.applicationPipeline import ApplicationPipeline +from metadata.workflow.application import ApplicationWorkflow + +from openmetadata_managed_apis.workflows.ingestion.application import build_application_workflow_config from openmetadata_managed_apis.workflows.ingestion.lineage import ( build_lineage_workflow_config, ) @@ -315,7 +321,7 @@ def test_profiler_workflow(self): def test_test_suite_workflow(self): """ Validate that the ingestionPipeline can be parsed - and properly load a Profiler Workflow + and properly load a Test Suite Workflow """ ingestion_pipeline = IngestionPipeline( @@ -344,3 +350,42 @@ def test_test_suite_workflow(self): config = json.loads(workflow_config.json(encoder=show_secrets_encoder)) parse_workflow_config_gracefully(config) + + @patch.object( + ApplicationWorkflow, + "set_ingestion_pipeline_status", + mock_set_ingestion_pipeline_status, + ) + def test_application_workflow(self): + """ + Validate that the ingestionPipeline can be parsed + and properly load an Application Workflow + """ + + ingestion_pipeline = IngestionPipeline( + id=uuid.uuid4(), + name="test_auto_tagger_application", + pipelineType=PipelineType.application, + fullyQualifiedName="OpenMetadata.test_auto_tagger_application", + sourceConfig=SourceConfig( + config=ApplicationPipeline( + type="Application", + appConfig=AppConfig(__root__=AutoTaggerAppConfig(confidenceLevel=80)), + sourcePythonClass="metadata.applications.auto_tagger.AutoTaggerApp", + ) + ), + openMetadataServerConnection=self.server_config, + airflowConfig=AirflowConfig( + startDate="2022-06-10T15:06:47+00:00", + ), + service=EntityReference( + id=uuid.uuid4(), + type="metadata", + name="OpenMetadata", + ), + ) + + workflow_config = build_application_workflow_config(ingestion_pipeline) + config = json.loads(workflow_config.json(encoder=show_secrets_encoder)) + + ApplicationWorkflow.create(config) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/pii/AutoPIIApplication.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/ExternalApplicationHandler.java similarity index 90% rename from openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/pii/AutoPIIApplication.java rename to openmetadata-service/src/main/java/org/openmetadata/service/apps/ExternalApplicationHandler.java index efc8d75f1657..f855b5eef89b 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/bundles/pii/AutoPIIApplication.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/ExternalApplicationHandler.java @@ -1,4 +1,4 @@ -package org.openmetadata.service.apps.bundles.pii; +package org.openmetadata.service.apps; import com.cronutils.model.Cron; import java.util.List; @@ -15,7 +15,6 @@ import org.openmetadata.schema.type.ProviderType; import org.openmetadata.schema.type.Relationship; import org.openmetadata.service.Entity; -import org.openmetadata.service.apps.AbstractNativeApplication; import org.openmetadata.service.exception.EntityNotFoundException; import org.openmetadata.service.jdbi3.CollectionDAO; import org.openmetadata.service.jdbi3.IngestionPipelineRepository; @@ -25,16 +24,14 @@ import org.openmetadata.service.util.JsonUtils; @Slf4j -public class AutoPIIApplication extends AbstractNativeApplication { - - private static final String INGESTION_PIPELINE_NAME = "autoPIIPipeline"; +public class ExternalApplicationHandler extends AbstractNativeApplication { private static final String SERVICE_NAME = "OpenMetadata"; @Override public void init(App app, CollectionDAO dao, SearchRepository searchRepository) { super.init(app, dao, searchRepository); this.app = app; - LOG.info("Data Insights App is initialized"); + LOG.info(String.format("%s App is initialized", app.getName())); } /** @@ -53,7 +50,7 @@ public void initializeExternalApp() { // Check if the Ingestion Pipeline has already been created try { - String fqn = FullyQualifiedName.add(SERVICE_NAME, INGESTION_PIPELINE_NAME); + String fqn = FullyQualifiedName.add(SERVICE_NAME, app.getName()); IngestionPipeline storedPipeline = ingestionPipelineRepository.getByName(null, fqn, ingestionPipelineRepository.getFields("id")); @@ -88,7 +85,7 @@ public void initializeExternalApp() { CreateIngestionPipeline createPipelineRequest = new CreateIngestionPipeline() - .withName(INGESTION_PIPELINE_NAME) + .withName(app.getName()) .withDisplayName(app.getDisplayName()) .withDescription(app.getDescription()) .withPipelineType(PipelineType.APPLICATION) @@ -96,7 +93,7 @@ public void initializeExternalApp() { new SourceConfig() .withConfig( new ApplicationPipeline() - .withSourcePythonClass(config.getSourcePythonClass()) + .withSourcePythonClass(app.getSourcePythonClass()) .withAppConfig(config.getConfig()))) .withAirflowConfig(new AirflowConfig().withScheduleInterval(cronMapper.map(quartzCron).asString())) .withService(service); diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/resources/apps/AppMarketPlaceResource.java b/openmetadata-service/src/main/java/org/openmetadata/service/resources/apps/AppMarketPlaceResource.java index e2b6191c6bac..964eaf57baf9 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/resources/apps/AppMarketPlaceResource.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/resources/apps/AppMarketPlaceResource.java @@ -382,7 +382,8 @@ private AppMarketPlaceDefinition getApplicationDefinition( .withPermission(create.getPermission()) .withAppLogoUrl(create.getAppLogoUrl()) .withAppScreenshots(create.getAppScreenshots()) - .withFeatures(create.getFeatures()); + .withFeatures(create.getFeatures()) + .withSourcePythonClass(create.getSourcePythonClass()); // Validate App validateApplication(app); diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/resources/apps/AppResource.java b/openmetadata-service/src/main/java/org/openmetadata/service/resources/apps/AppResource.java index ac0de81bedba..481268ad58ef 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/resources/apps/AppResource.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/resources/apps/AppResource.java @@ -752,7 +752,8 @@ private App getApplication( .withAppSchedule(createAppRequest.getAppSchedule()) .withAppLogoUrl(marketPlaceDefinition.getAppLogoUrl()) .withAppScreenshots(marketPlaceDefinition.getAppScreenshots()) - .withFeatures(marketPlaceDefinition.getFeatures()); + .withFeatures(marketPlaceDefinition.getFeatures()) + .withSourcePythonClass(marketPlaceDefinition.getSourcePythonClass()); // validate Bot if provided validateAndAddBot(app, createAppRequest.getBot()); diff --git a/openmetadata-service/src/main/resources/json/data/appMarketPlaceDefinition/AutoPIIApplication.json b/openmetadata-service/src/main/resources/json/data/appMarketPlaceDefinition/AutoTaggerApplication.json similarity index 58% rename from openmetadata-service/src/main/resources/json/data/appMarketPlaceDefinition/AutoPIIApplication.json rename to openmetadata-service/src/main/resources/json/data/appMarketPlaceDefinition/AutoTaggerApplication.json index 708954916f6f..e658e5e4eaa7 100644 --- a/openmetadata-service/src/main/resources/json/data/appMarketPlaceDefinition/AutoPIIApplication.json +++ b/openmetadata-service/src/main/resources/json/data/appMarketPlaceDefinition/AutoTaggerApplication.json @@ -1,8 +1,8 @@ { - "name": "AutoPIIApplication", - "displayName": "Auto PII", - "description": "Use NLP models to automatically tag PII information", - "features": "Auto tag PII information in your tables' columns", + "name": "AutoTaggerApplication", + "displayName": "AutoTagger", + "description": "Use NLP models to automatically tag PII information and add your own business rules for automatic data tagging.", + "features": "Auto Tag information in your tables & columns", "appType": "external", "appScreenshots": [ "AutoPIIApplication.png" @@ -13,12 +13,12 @@ "supportEmail": "support@getcollate.io", "scheduleType": "Scheduled", "permission": "All", - "className": "org.openmetadata.service.apps.bundles.pii.AutoPIIApplication", + "className": "org.openmetadata.service.apps.ExternalApplicationHandler", + "sourcePythonClass": "metadata.applications.auto_tagger.AutoTaggerApp", "runtime": { "enabled": "true" }, "appConfiguration": { - "sourcePythonClass": "metadata.applications.pii.PIIApp", "config": { "confidenceLevel": 80 } diff --git a/openmetadata-spec/src/main/resources/json/schema/entity/applications/app.json b/openmetadata-spec/src/main/resources/json/schema/entity/applications/app.json index 20d24970f5be..f7e6feca872a 100644 --- a/openmetadata-spec/src/main/resources/json/schema/entity/applications/app.json +++ b/openmetadata-spec/src/main/resources/json/schema/entity/applications/app.json @@ -162,7 +162,11 @@ "type": "string" }, "className": { - "description": "Full Qualified ClassName for the Schedule", + "description": "Fully Qualified ClassName for the Schedule", + "type": "string" + }, + "sourcePythonClass": { + "description": "Fully Qualified class name for the Python source that will execute the external application.", "type": "string" }, "appType": { diff --git a/openmetadata-spec/src/main/resources/json/schema/entity/applications/configuration/external/autoTaggerAppConfig.json b/openmetadata-spec/src/main/resources/json/schema/entity/applications/configuration/external/autoTaggerAppConfig.json new file mode 100644 index 000000000000..ad056ba6fcee --- /dev/null +++ b/openmetadata-spec/src/main/resources/json/schema/entity/applications/configuration/external/autoTaggerAppConfig.json @@ -0,0 +1,30 @@ +{ + "$id": "https://open-metadata.org/schema/entity/applications/configuration/external/autoTaggerAppConfig.json", + "$schema": "http://json-schema.org/draft-07/schema#", + "title": "AutoTaggerAppConfig", + "description": "Configuration for the Auto Tagger External Application.", + "type": "object", + "definitions": { + "autoTaggerApp": { + "description": "Application type.", + "type": "string", + "enum": ["AutoTagger"], + "default": "AutoTagger" + } + }, + "properties": { + "type": { + "title": "Service Type", + "description": "Service Type", + "$ref": "#/definitions/autoTaggerApp", + "default": "AutoTagger" + }, + "confidenceLevel": { + "title": "Confidence Level", + "type": "integer", + "description": "Confidence level for the ML models to apply the PII tagging.", + "default": 80 + } + }, + "additionalProperties": false +} diff --git a/openmetadata-spec/src/main/resources/json/schema/entity/applications/configuration/externalApplicationConfig.json b/openmetadata-spec/src/main/resources/json/schema/entity/applications/configuration/externalApplicationConfig.json index 47bbe6a38f8a..14c56720160c 100644 --- a/openmetadata-spec/src/main/resources/json/schema/entity/applications/configuration/externalApplicationConfig.json +++ b/openmetadata-spec/src/main/resources/json/schema/entity/applications/configuration/externalApplicationConfig.json @@ -4,16 +4,13 @@ "title": "ExternalApplicationConfig", "description": "External Application Pipeline Configuration.", "type": "object", - "properties": { - "config": { - "$ref": "../../../type/basic.json#/definitions/componentConfig", - "description": "Application-specific configuration" - }, - "sourcePythonClass": { - "description": "Source Python Class Name to run the application", - "type": "string" + "definitions": { + "appConfig": { + "oneOf": [ + { + "$ref": "./external/autoTaggerAppConfig.json" + } + ] } - }, - "additionalProperties": false, - "required": ["sourcePythonClass"] -} + } +} \ No newline at end of file diff --git a/openmetadata-spec/src/main/resources/json/schema/entity/applications/marketplace/appMarketPlaceDefinition.json b/openmetadata-spec/src/main/resources/json/schema/entity/applications/marketplace/appMarketPlaceDefinition.json index 3f4c46c83ec0..6d7f0b692fe0 100644 --- a/openmetadata-spec/src/main/resources/json/schema/entity/applications/marketplace/appMarketPlaceDefinition.json +++ b/openmetadata-spec/src/main/resources/json/schema/entity/applications/marketplace/appMarketPlaceDefinition.json @@ -89,6 +89,10 @@ "description": "Full Qualified ClassName for the the application", "type": "string" }, + "sourcePythonClass": { + "description": "Fully Qualified class name for the Python source that will execute the external application.", + "type": "string" + }, "appType": { "description": "This schema defines the type of application.", "$ref": "../app.json#/definitions/appType" diff --git a/openmetadata-spec/src/main/resources/json/schema/entity/applications/marketplace/createAppMarketPlaceDefinitionReq.json b/openmetadata-spec/src/main/resources/json/schema/entity/applications/marketplace/createAppMarketPlaceDefinitionReq.json index fe4a874a51c1..3529769c25ec 100644 --- a/openmetadata-spec/src/main/resources/json/schema/entity/applications/marketplace/createAppMarketPlaceDefinitionReq.json +++ b/openmetadata-spec/src/main/resources/json/schema/entity/applications/marketplace/createAppMarketPlaceDefinitionReq.json @@ -48,6 +48,10 @@ "description": "Full Qualified ClassName for the the application", "type": "string" }, + "sourcePythonClass": { + "description": "Fully Qualified class name for the Python source that will execute the external application.", + "type": "string" + }, "appType": { "description": "This schema defines the type of application.", "$ref": "../app.json#/definitions/appType" diff --git a/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/application.json b/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/application.json index e5a67f97a2d9..b27f974b25f9 100644 --- a/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/application.json +++ b/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/application.json @@ -9,8 +9,12 @@ "$ref": "workflow.json#/definitions/workflowConfig", "description": "General Workflow configuration, such as the OpenMetadata server connection and logging level" }, + "sourcePythonClass": { + "description": "Source Python Class Name to run the application", + "type": "string" + }, "appConfig": { - "$ref": "../entity/applications/configuration/externalApplicationConfig.json", + "$ref": "../entity/applications/configuration/externalApplicationConfig.json#/definitions/appConfig", "description": "External Application configuration" }, "ingestionPipelineFQN": { diff --git a/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/applicationPipeline.json b/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/applicationPipeline.json index 4ff088aff2f0..783ae5e22583 100644 --- a/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/applicationPipeline.json +++ b/openmetadata-spec/src/main/resources/json/schema/metadataIngestion/applicationPipeline.json @@ -23,8 +23,8 @@ "type": "string" }, "appConfig": { - "$ref": "../type/basic.json#/definitions/componentConfig", - "description": "Application-specific configuration" + "$ref": "../entity/applications/configuration/externalApplicationConfig.json#/definitions/appConfig", + "description": "External Application configuration" } }, "additionalProperties": false diff --git a/openmetadata-ui/src/main/resources/ui/src/assets/svg/AutoPIIApplication.svg b/openmetadata-ui/src/main/resources/ui/src/assets/svg/AutoTaggerApplication.svg similarity index 100% rename from openmetadata-ui/src/main/resources/ui/src/assets/svg/AutoPIIApplication.svg rename to openmetadata-ui/src/main/resources/ui/src/assets/svg/AutoTaggerApplication.svg From 63b6fe236aa06f48db071b11e532d7e844c6c9b9 Mon Sep 17 00:00:00 2001 From: Pere Miquel Brull Date: Wed, 8 Nov 2023 12:14:59 +0100 Subject: [PATCH 12/16] Prepare config-based external apps --- .../src/metadata/applications/auto_tagger.py | 7 +++++- .../src/metadata/workflow/application.py | 7 ++++++ .../workflows/ingestion/application.py | 2 +- .../test_workflow_creation.py | 23 +++++++++++++------ .../externalApplicationConfig.json | 11 +++++++-- 5 files changed, 39 insertions(+), 11 deletions(-) diff --git a/ingestion/src/metadata/applications/auto_tagger.py b/ingestion/src/metadata/applications/auto_tagger.py index 1080fabce234..b9636f432d0c 100644 --- a/ingestion/src/metadata/applications/auto_tagger.py +++ b/ingestion/src/metadata/applications/auto_tagger.py @@ -32,7 +32,7 @@ from metadata.pii.scanners.column_name_scanner import ColumnNameScanner from metadata.pii.scanners.ner_scanner import NERScanner from metadata.utils.logger import app_logger -from metadata.workflow.application import AppRunner +from metadata.workflow.application import AppRunner, InvalidAppConfiguration logger = app_logger() @@ -60,6 +60,11 @@ class AutoTaggerApp(AppRunner): def __init__(self, config: AutoTaggerAppConfig, metadata: OpenMetadata): super().__init__(config, metadata) + if not isinstance(config, AutoTaggerAppConfig): + raise InvalidAppConfiguration( + f"AutoTagger Runner expects an AutoTaggerAppConfig, we got [{config}]" + ) + self._ner_scanner = None self.confidence_threshold = config.confidenceLevel or DEFAULT_CONFIDENCE diff --git a/ingestion/src/metadata/workflow/application.py b/ingestion/src/metadata/workflow/application.py index 96126b60eb1b..1ed65c03252f 100644 --- a/ingestion/src/metadata/workflow/application.py +++ b/ingestion/src/metadata/workflow/application.py @@ -37,6 +37,13 @@ logger = ingestion_logger() +class InvalidAppConfiguration(Exception): + """ + To be raised if the config received by the App + is not the one expected + """ + + class AppRunner(Step, ABC): """Class that knows how to execute the Application logic.""" diff --git a/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py b/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py index dc19438447d0..e05fd3388227 100644 --- a/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py +++ b/openmetadata-airflow-apis/openmetadata_managed_apis/workflows/ingestion/application.py @@ -21,7 +21,7 @@ ) from metadata.generated.schema.entity.applications.configuration.externalApplicationConfig import ( - ExternalApplicationConfig, AppConfig, + AppConfig, ) from metadata.generated.schema.entity.services.ingestionPipelines.ingestionPipeline import ( IngestionPipeline, diff --git a/openmetadata-airflow-apis/tests/unit/ingestion_pipeline/test_workflow_creation.py b/openmetadata-airflow-apis/tests/unit/ingestion_pipeline/test_workflow_creation.py index 37f37df2b77e..2fb423ccd515 100644 --- a/openmetadata-airflow-apis/tests/unit/ingestion_pipeline/test_workflow_creation.py +++ b/openmetadata-airflow-apis/tests/unit/ingestion_pipeline/test_workflow_creation.py @@ -17,12 +17,9 @@ from unittest import TestCase from unittest.mock import patch -from metadata.generated.schema.entity.applications.configuration.external.autoTaggerAppConfig import AutoTaggerAppConfig -from metadata.generated.schema.entity.applications.configuration.externalApplicationConfig import AppConfig -from metadata.generated.schema.metadataIngestion.applicationPipeline import ApplicationPipeline -from metadata.workflow.application import ApplicationWorkflow - -from openmetadata_managed_apis.workflows.ingestion.application import build_application_workflow_config +from openmetadata_managed_apis.workflows.ingestion.application import ( + build_application_workflow_config, +) from openmetadata_managed_apis.workflows.ingestion.lineage import ( build_lineage_workflow_config, ) @@ -39,6 +36,12 @@ build_usage_workflow_config, ) +from metadata.generated.schema.entity.applications.configuration.external.autoTaggerAppConfig import ( + AutoTaggerAppConfig, +) +from metadata.generated.schema.entity.applications.configuration.externalApplicationConfig import ( + AppConfig, +) from metadata.generated.schema.entity.services.connections.metadata.openMetadataConnection import ( OpenMetadataConnection, ) @@ -48,6 +51,9 @@ IngestionPipeline, PipelineType, ) +from metadata.generated.schema.metadataIngestion.applicationPipeline import ( + ApplicationPipeline, +) from metadata.generated.schema.metadataIngestion.databaseServiceMetadataPipeline import ( DatabaseServiceMetadataPipeline, ) @@ -74,6 +80,7 @@ from metadata.ingestion.api.parser import parse_workflow_config_gracefully from metadata.ingestion.models.encoders import show_secrets_encoder from metadata.ingestion.ometa.ometa_api import OpenMetadata +from metadata.workflow.application import ApplicationWorkflow from metadata.workflow.data_quality import TestSuiteWorkflow from metadata.workflow.metadata import MetadataWorkflow from metadata.workflow.profiler import ProfilerWorkflow @@ -370,7 +377,9 @@ def test_application_workflow(self): sourceConfig=SourceConfig( config=ApplicationPipeline( type="Application", - appConfig=AppConfig(__root__=AutoTaggerAppConfig(confidenceLevel=80)), + appConfig=AppConfig( + __root__=AutoTaggerAppConfig(confidenceLevel=80) + ), sourcePythonClass="metadata.applications.auto_tagger.AutoTaggerApp", ) ), diff --git a/openmetadata-spec/src/main/resources/json/schema/entity/applications/configuration/externalApplicationConfig.json b/openmetadata-spec/src/main/resources/json/schema/entity/applications/configuration/externalApplicationConfig.json index 14c56720160c..f6398af87316 100644 --- a/openmetadata-spec/src/main/resources/json/schema/entity/applications/configuration/externalApplicationConfig.json +++ b/openmetadata-spec/src/main/resources/json/schema/entity/applications/configuration/externalApplicationConfig.json @@ -12,5 +12,12 @@ } ] } - } -} \ No newline at end of file + }, + "properties": { + "config": { + "$ref": "#/definitions/appConfig" + } + }, + "additionalProperties": false, + "required": ["config"] +} From 1ae880eb1c152a90a5ac62a895f17ecfbfce94d7 Mon Sep 17 00:00:00 2001 From: Pere Miquel Brull Date: Wed, 8 Nov 2023 12:44:42 +0100 Subject: [PATCH 13/16] Fix lint --- ingestion/Makefile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ingestion/Makefile b/ingestion/Makefile index 54141ed3e747..6d57e471f3b8 100644 --- a/ingestion/Makefile +++ b/ingestion/Makefile @@ -90,7 +90,7 @@ run_apis_tests: ## Run the openmetadata airflow apis tests coverage_apis: ## Run the python tests on openmetadata-airflow-apis $(MAKE) run_apis_tests coverage xml --rcfile $(ROOT_DIR)/openmetadata-airflow-apis/.coveragerc -o $(ROOT_DIR)/openmetadata-airflow-apis/coverage.xml - sed -e "s/$(shell python -c "import site; import os; from pathlib import Path; print(os.path.relpath(site.getsitepackages()[0], str(Path.cwd())).replace('/','\/'))")\///g" $(INGESTION_DIR)/openmetadata-airflow-apis/coverage.xml >> $(INGESTION_DIR)/openmetadata-airflow-apis/ci-coverage.xml + sed -e "s/$(shell python -c "import site; import os; from pathlib import Path; print(os.path.relpath(site.getsitepackages()[0], str(Path.cwd())).replace('/','\/'))")\///g" $(ROOT_DIR)/openmetadata-airflow-apis/coverage.xml >> $(ROOT_DIR)/openmetadata-airflow-apis/ci-coverage.xml From 112e0dd317e4b81d156b820732a9282bc7a5f113 Mon Sep 17 00:00:00 2001 From: Pere Miquel Brull Date: Wed, 8 Nov 2023 14:53:15 +0100 Subject: [PATCH 14/16] Prepare config-based external apps --- ingestion/src/metadata/workflow/data_insight.py | 15 ++++----------- 1 file changed, 4 insertions(+), 11 deletions(-) diff --git a/ingestion/src/metadata/workflow/data_insight.py b/ingestion/src/metadata/workflow/data_insight.py index 3ddcdd4e0d44..ce006299b4a2 100644 --- a/ingestion/src/metadata/workflow/data_insight.py +++ b/ingestion/src/metadata/workflow/data_insight.py @@ -14,9 +14,6 @@ from metadata.data_insight.processor.kpi.kpi_runner import KpiRunner from metadata.data_insight.source.metadata import DataInsightSource from metadata.generated.schema.analytics.basic import WebAnalyticEventType -from metadata.generated.schema.metadataIngestion.workflow import ( - OpenMetadataWorkflowConfig, -) from metadata.generated.schema.tests.testSuite import ServiceType from metadata.ingestion.api.steps import Sink from metadata.utils.importer import import_sink_class @@ -32,11 +29,6 @@ class DataInsightWorkflow(IngestionWorkflow): retention_days = 7 - def __init__(self, config: OpenMetadataWorkflowConfig): - super().__init__(config) - - self.sink = None - def _retrieve_service_connection_if_needed(self, service_type: ServiceType) -> None: """No service connection needed for data insight""" return None @@ -47,7 +39,8 @@ def _run_kpi_processor(self) -> None: kpi_runner = KpiRunner(self.metadata) for kpi_result in kpi_runner.run(): - self.sink.run(kpi_result) + # Pick up the sink + self.steps[0].run(kpi_result) def _clean_up_web_analytics_events(self) -> None: """ @@ -81,5 +74,5 @@ def _execute_internal(self): def set_steps(self): self.source = DataInsightSource.create(self.metadata) # type: ignore - self.sink = self._get_sink() - self.steps = (self.sink,) + sink = self._get_sink() + self.steps = (sink,) From 1db095812006f0e7f03fda8a23427cff66e327bd Mon Sep 17 00:00:00 2001 From: Pere Miquel Brull Date: Wed, 8 Nov 2023 15:25:21 +0100 Subject: [PATCH 15/16] Fix DI errors --- ingestion/src/metadata/utils/logger.py | 2 +- ingestion/src/metadata/workflow/application.py | 1 + ingestion/src/metadata/workflow/data_insight.py | 2 +- 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/ingestion/src/metadata/utils/logger.py b/ingestion/src/metadata/utils/logger.py index 05e72487b231..d14832d0892b 100644 --- a/ingestion/src/metadata/utils/logger.py +++ b/ingestion/src/metadata/utils/logger.py @@ -154,7 +154,7 @@ def great_expectations_logger(): def app_logger(): """ - Method to get the UTILS logger + Method to get the APP logger """ return logging.getLogger(Loggers.APP.value) diff --git a/ingestion/src/metadata/workflow/application.py b/ingestion/src/metadata/workflow/application.py index 1ed65c03252f..7a191792eebd 100644 --- a/ingestion/src/metadata/workflow/application.py +++ b/ingestion/src/metadata/workflow/application.py @@ -73,6 +73,7 @@ class ApplicationWorkflow(BaseWorkflow, ABC): def __init__(self, config_dict: dict): self.runner = None # Will be passed in post-init + # TODO: Create a parse_gracefully method self.config = OpenMetadataApplicationConfig.parse_obj(config_dict) # Applications are associated to the OpenMetadata Service diff --git a/ingestion/src/metadata/workflow/data_insight.py b/ingestion/src/metadata/workflow/data_insight.py index ce006299b4a2..d0a30be8bd85 100644 --- a/ingestion/src/metadata/workflow/data_insight.py +++ b/ingestion/src/metadata/workflow/data_insight.py @@ -65,7 +65,7 @@ def _get_sink(self) -> Sink: return sink - def _execute_internal(self): + def execute_internal(self): """Use parent logic and add step to process KPIs""" super().execute_internal() self._run_kpi_processor() From 25b1909c401d006814b69d648a069fa36713c2c4 Mon Sep 17 00:00:00 2001 From: Pere Miquel Brull Date: Wed, 8 Nov 2023 17:52:20 +0100 Subject: [PATCH 16/16] Amend comments --- .../apps/AbstractNativeApplication.java | 9 ++--- .../apps/ExternalApplicationHandler.java | 34 +++++++------------ 2 files changed, 18 insertions(+), 25 deletions(-) diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/AbstractNativeApplication.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/AbstractNativeApplication.java index 7f00c7ed4ea7..adde19256cc4 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/AbstractNativeApplication.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/AbstractNativeApplication.java @@ -12,6 +12,7 @@ import com.cronutils.model.definition.CronDefinitionBuilder; import com.cronutils.parser.CronParser; import java.util.List; +import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.openmetadata.schema.AppRuntime; import org.openmetadata.schema.api.services.ingestionPipelines.CreateIngestionPipeline; @@ -42,10 +43,10 @@ @Slf4j public class AbstractNativeApplication implements NativeApplication { protected CollectionDAO collectionDAO; - public App app; + private @Getter App app; protected SearchRepository searchRepository; - public final CronMapper cronMapper = CronMapper.fromQuartzToUnix(); - public final CronParser cronParser = new CronParser(CronDefinitionBuilder.instanceDefinitionFor(QUARTZ)); + private final @Getter CronMapper cronMapper = CronMapper.fromQuartzToUnix(); + private final @Getter CronParser cronParser = new CronParser(CronDefinitionBuilder.instanceDefinitionFor(QUARTZ)); @Override public void init(App app, CollectionDAO dao, SearchRepository searchRepository) { @@ -187,7 +188,7 @@ public static AppRuntime getAppRuntime(App app) { return JsonUtils.convertValue(app.getRuntime(), ScheduledExecutionContext.class); } - public IngestionPipeline getIngestionPipeline(CreateIngestionPipeline create, String botname, String user) { + protected IngestionPipeline getIngestionPipeline(CreateIngestionPipeline create, String botname, String user) { IngestionPipelineRepository ingestionPipelineRepository = (IngestionPipelineRepository) Entity.getEntityRepository(Entity.INGESTION_PIPELINE); OpenMetadataConnection openMetadataServerConnection = diff --git a/openmetadata-service/src/main/java/org/openmetadata/service/apps/ExternalApplicationHandler.java b/openmetadata-service/src/main/java/org/openmetadata/service/apps/ExternalApplicationHandler.java index f855b5eef89b..69f14ad689dc 100644 --- a/openmetadata-service/src/main/java/org/openmetadata/service/apps/ExternalApplicationHandler.java +++ b/openmetadata-service/src/main/java/org/openmetadata/service/apps/ExternalApplicationHandler.java @@ -4,7 +4,6 @@ import java.util.List; import lombok.extern.slf4j.Slf4j; import org.openmetadata.schema.api.services.ingestionPipelines.CreateIngestionPipeline; -import org.openmetadata.schema.entity.app.App; import org.openmetadata.schema.entity.applications.configuration.ExternalApplicationConfig; import org.openmetadata.schema.entity.services.ingestionPipelines.AirflowConfig; import org.openmetadata.schema.entity.services.ingestionPipelines.IngestionPipeline; @@ -19,7 +18,6 @@ import org.openmetadata.service.jdbi3.CollectionDAO; import org.openmetadata.service.jdbi3.IngestionPipelineRepository; import org.openmetadata.service.jdbi3.MetadataServiceRepository; -import org.openmetadata.service.search.SearchRepository; import org.openmetadata.service.util.FullyQualifiedName; import org.openmetadata.service.util.JsonUtils; @@ -27,13 +25,6 @@ public class ExternalApplicationHandler extends AbstractNativeApplication { private static final String SERVICE_NAME = "OpenMetadata"; - @Override - public void init(App app, CollectionDAO dao, SearchRepository searchRepository) { - super.init(app, dao, searchRepository); - this.app = app; - LOG.info(String.format("%s App is initialized", app.getName())); - } - /** * MetaPilot is an external App that accepts one ApiKey parameter and runs a workflow based on it. * @@ -44,13 +35,13 @@ public void init(App app, CollectionDAO dao, SearchRepository searchRepository) public void initializeExternalApp() { ExternalApplicationConfig config = - JsonUtils.convertValue(app.getAppConfiguration(), ExternalApplicationConfig.class); + JsonUtils.convertValue(this.getApp().getAppConfiguration(), ExternalApplicationConfig.class); IngestionPipelineRepository ingestionPipelineRepository = (IngestionPipelineRepository) Entity.getEntityRepository(Entity.INGESTION_PIPELINE); // Check if the Ingestion Pipeline has already been created try { - String fqn = FullyQualifiedName.add(SERVICE_NAME, app.getName()); + String fqn = FullyQualifiedName.add(SERVICE_NAME, this.getApp().getName()); IngestionPipeline storedPipeline = ingestionPipelineRepository.getByName(null, fqn, ingestionPipelineRepository.getFields("id")); @@ -58,14 +49,14 @@ public void initializeExternalApp() { List records = collectionDAO .relationshipDAO() - .findTo(app.getId(), Entity.APPLICATION, Relationship.HAS.ordinal(), Entity.INGESTION_PIPELINE); + .findTo(this.getApp().getId(), Entity.APPLICATION, Relationship.HAS.ordinal(), Entity.INGESTION_PIPELINE); if (records.isEmpty()) { // Add Ingestion Pipeline to Application collectionDAO .relationshipDAO() .insert( - app.getId(), + this.getApp().getId(), storedPipeline.getId(), Entity.APPLICATION, Entity.INGESTION_PIPELINE, @@ -81,26 +72,27 @@ public void initializeExternalApp() { .getByName(null, SERVICE_NAME, serviceEntityRepository.getFields("id")) .getEntityReference(); - Cron quartzCron = cronParser.parse(app.getAppSchedule().getCronExpression()); + Cron quartzCron = this.getCronParser().parse(this.getApp().getAppSchedule().getCronExpression()); CreateIngestionPipeline createPipelineRequest = new CreateIngestionPipeline() - .withName(app.getName()) - .withDisplayName(app.getDisplayName()) - .withDescription(app.getDescription()) + .withName(this.getApp().getName()) + .withDisplayName(this.getApp().getDisplayName()) + .withDescription(this.getApp().getDescription()) .withPipelineType(PipelineType.APPLICATION) .withSourceConfig( new SourceConfig() .withConfig( new ApplicationPipeline() - .withSourcePythonClass(app.getSourcePythonClass()) + .withSourcePythonClass(this.getApp().getSourcePythonClass()) .withAppConfig(config.getConfig()))) - .withAirflowConfig(new AirflowConfig().withScheduleInterval(cronMapper.map(quartzCron).asString())) + .withAirflowConfig( + new AirflowConfig().withScheduleInterval(this.getCronMapper().map(quartzCron).asString())) .withService(service); // Get Pipeline IngestionPipeline dataInsightPipeline = - getIngestionPipeline(createPipelineRequest, String.format("%sBot", app.getName()), "admin") + getIngestionPipeline(createPipelineRequest, String.format("%sBot", this.getApp().getName()), "admin") .withProvider(ProviderType.USER); ingestionPipelineRepository.setFullyQualifiedName(dataInsightPipeline); ingestionPipelineRepository.initializeEntity(dataInsightPipeline); @@ -109,7 +101,7 @@ public void initializeExternalApp() { collectionDAO .relationshipDAO() .insert( - app.getId(), + this.getApp().getId(), dataInsightPipeline.getId(), Entity.APPLICATION, Entity.INGESTION_PIPELINE,