MINOR - Implement SAP Hana Lineage (#17615)

* MINOR - SAP Hana Lineage

* skeleton

* parser

* lineage

* manage formulas

* add cvs

* add cvs

* better typing

* enum

* handle cvs

* saphana docs
This commit is contained in:
Pere Miquel Brull 2024-08-30 07:42:43 +02:00
parent c0a23c667b
commit 72afde0aca
15 changed files with 1801 additions and 3 deletions

View File

@ -0,0 +1,772 @@
# 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.
"""
Parse CDATA XMLs from SAP Hana
"""
import itertools
import re
import traceback
import xml.etree.ElementTree as ET
from collections import defaultdict
from enum import Enum
from functools import lru_cache
from typing import Dict, Iterable, List, NewType, Optional, Set
from pydantic import Field, computed_field
from typing_extensions import Annotated
from metadata.generated.schema.api.lineage.addLineage import AddLineageRequest
from metadata.generated.schema.entity.data.table import Table
from metadata.generated.schema.entity.services.ingestionPipelines.status import (
StackTraceError,
)
from metadata.generated.schema.type.basic import FullyQualifiedEntityName
from metadata.generated.schema.type.entityLineage import (
ColumnLineage,
EntitiesEdge,
LineageDetails,
Source,
)
from metadata.generated.schema.type.entityReference import EntityReference
from metadata.ingestion.api.models import Either
from metadata.ingestion.lineage.sql_lineage import get_column_fqn
from metadata.ingestion.models.custom_pydantic import BaseModel
from metadata.ingestion.ometa.ometa_api import OpenMetadata
from metadata.ingestion.source.database.saphana.models import (
SYS_BIC_SCHEMA_NAME,
ViewType,
)
from metadata.utils import fqn
from metadata.utils.constants import ENTITY_REFERENCE_TYPE_MAP
from metadata.utils.dispatch import enum_register
from metadata.utils.logger import ingestion_logger
logger = ingestion_logger()
class CDATAParsingError(Exception):
"""Error parsing CDATA XML"""
XSI_NS = {"xsi": "http://www.w3.org/2001/XMLSchema-instance"}
NAMESPACE_DICT = {
ViewType.ANALYTIC_VIEW.value: {
"Cube": "http://www.sap.com/ndb/BiModelCube.ecore",
**XSI_NS,
},
ViewType.CALCULATION_VIEW.value: {
"Calculation": "http://www.sap.com/ndb/BiModelCalculation.ecore",
**XSI_NS,
},
ViewType.ATTRIBUTE_VIEW.value: {
"Dimension": "http://www.sap.com/ndb/BiModelDimension.ecore",
**XSI_NS,
},
}
FORMULA_PATTERN = re.compile(r"\"(.*?)\"")
class CDATAKeys(Enum):
"""Keys to access data in CDATA XML files"""
COLUMN_OBJECT_NAME = "columnObjectName"
COLUMN_OBJECT = "columnObject"
COLUMN_NAME = "columnName"
SCHEMA_NAME = "schemaName"
ATTRIBUTES = "attributes"
ATTRIBUTE = "attribute"
KEY_MAPPING = "keyMapping"
MAPPING = "mapping"
SOURCE = "source"
TARGET = "target"
NODE = "node"
TYPE = "type"
INPUT = "input"
CALCULATION_VIEWS = "calculationViews"
CALCULATION_VIEW = "calculationView"
CALCULATION_VIEW_ATTRIBUTES = "calculatedViewAttributes"
CALCULATION_VIEW_ATTRIBUTE = "calculatedViewAttribute"
RESOURCE_URI = "resourceUri"
CALCULATED_ATTRS = "calculatedAttributes"
KEY_CALCULATION = "keyCalculation"
FORMULA = "formula"
BASE_MEASURES = "baseMeasures"
MEASURE = "measure"
MEASURE_MAPPING = "measureMapping"
PRIVATE_MEASURE_GROUP = "privateMeasureGroup"
LOGICAL_MODEL = "logicalModel"
DATA_SOURCES = "dataSources"
DATA_SOURCE = "DataSource" # yes, with capital D
ID = "id"
class CalculatedAttrKey(Enum):
CALCULATED_ATTRIBUTE = "calculatedAttribute"
CALCULATED_VIEW_ATTRIBUTE = "calculatedViewAttribute"
class ParentSource(BaseModel):
"""Parent Source of a given column"""
# TODO: Multiple sources from the same parent should be possible
source: Annotated[
str, Field(..., description="Column name in the parent Data Source")
]
parent: Annotated[str, Field(..., description="Parent ID")]
class DataSourceMapping(BaseModel):
"""Column Mapping of DataSources and Logical Calculated Views"""
target: Annotated[
str, Field(..., description="Column name in the provided Data Source")
]
parents: Annotated[
List[ParentSource], Field(..., description="Parent Sources for a target col")
]
class DataSource(BaseModel):
"""Data source from CDATA XML"""
name: Annotated[str, Field(..., description="Data Source name")]
location: Annotated[
Optional[str], Field(None, description="Schema or project for the Data Source")
]
source_type: Annotated[
Optional[ViewType],
Field(..., description="Data Source type"),
]
mapping: Annotated[
Optional[Dict[str, DataSourceMapping]],
Field(
None,
description="Logical source column mapping. Key: source column; value: mapping",
),
]
def get_entity(
self,
metadata: OpenMetadata,
service_name: str,
) -> Table:
"""Build the Entity Reference for this DataSource"""
if self.source_type == ViewType.LOGICAL:
raise CDATAParsingError(
f"We could not find the logical DataSource origin for {self.name}"
)
if self.source_type == ViewType.DATA_BASE_TABLE:
# The source is a table, so the location is the schema
fqn_ = fqn.build(
metadata=metadata,
entity_type=Table,
service_name=service_name,
database_name=None, # TODO: Can we assume HXE?
schema_name=self.location,
table_name=self.name,
)
else:
# The source is a CalculationView, AttributeView or AnalyticView
# package from <resourceUri>/SFLIGHT.MODELING/calculationviews/CV_SFLIGHT_SBOOK</resourceUri>
package = self.location.split("/")[1]
fqn_ = fqn.build(
metadata=metadata,
entity_type=Table,
service_name=service_name,
database_name=None,
schema_name=SYS_BIC_SCHEMA_NAME,
table_name=f"{package}/{self.name}",
)
return metadata.get_by_name(entity=Table, fqn=fqn_)
def __hash__(self):
return hash(self.location) + hash(self.name) + hash(self.source_type)
# Given the DataSource ID, get the DataSource from the CDATA XML
DataSourceMap = NewType("DataSourceMap", Dict[str, DataSource])
class ColumnMapping(BaseModel):
"""Column Mapping from CDATA XML"""
data_source: Annotated[DataSource, Field(..., description="Source table name")]
sources: Annotated[List[str], Field(..., description="Source column names")]
target: Annotated[str, Field(..., description="Destination column name")]
formula: Annotated[
Optional[str], Field(None, description="Formula used to derive the column")
]
class ParsedLineage(BaseModel):
"""Parsed Lineage from CDATA XML. For each view, we'll parse the sources"""
mappings: Annotated[
Optional[List[ColumnMapping]], Field([], description="Column mappings")
]
@computed_field
@property
def sources(self) -> Set[DataSource]:
"""Get all the different source tables we'll need to iterate over"""
return {mapping.data_source for mapping in self.mappings}
@lru_cache(maxsize=256)
def find_target(self, column: str) -> Optional[ColumnMapping]:
"""Find the column mapping based on the target column"""
return next(
(mapping for mapping in self.mappings if mapping.target == column), None
)
def __add__(self, other: "ParsedLineage") -> "ParsedLineage":
"""Merge two parsed lineages"""
return ParsedLineage(mappings=self.mappings + other.mappings)
def __hash__(self):
"""
Note that the LRU Cache require us to implement the __hash__ method, otherwise
the BaseModel is not hashable. Since we just want a per-instance cache, we'll use the id
"""
return id(self)
def to_request(
self, metadata: OpenMetadata, service_name: str, to_entity: Table
) -> Iterable[Either[AddLineageRequest]]:
"""Given the target entity, build the AddLineageRequest based on the sources in `self`"""
for source in self.sources:
try:
source_table = source.get_entity(
metadata=metadata, service_name=service_name
)
if not source_table:
logger.warning(f"Can't find table for source [{source}]")
continue
yield Either(
right=AddLineageRequest(
edge=EntitiesEdge(
fromEntity=EntityReference(
id=source_table.id,
type=ENTITY_REFERENCE_TYPE_MAP[Table.__name__],
),
toEntity=EntityReference(
id=to_entity.id,
type=ENTITY_REFERENCE_TYPE_MAP[Table.__name__],
),
lineageDetails=LineageDetails(
source=Source.ViewLineage,
columnsLineage=[
ColumnLineage(
fromColumns=[
FullyQualifiedEntityName(
get_column_fqn(
table_entity=source_table,
column=source_col,
)
)
for source_col in mapping.sources
],
toColumn=FullyQualifiedEntityName(
get_column_fqn(
table_entity=to_entity,
column=mapping.target,
)
),
function=mapping.formula
if mapping.formula
else None,
)
for mapping in self.mappings
if mapping.data_source == source
],
),
)
)
)
except Exception as exc:
yield Either(
left=StackTraceError(
name=to_entity.fullyQualifiedName.root,
error=f"Error trying to get lineage for [{source}] due to [{exc}]",
stackTrace=traceback.format_exc(),
)
)
def _get_column_datasources(
entry: ET.Element, datasource_map: Optional[DataSourceMap] = None
) -> Set[DataSource]:
"""Read a DataSource from the CDATA XML"""
if (
datasource_map
and entry.get(CDATAKeys.COLUMN_OBJECT_NAME.value) in datasource_map
):
# If the datasource is in the map, we'll traverse all intermediate logical
# datasources until we arrive to a table or view.
# Note that we can have multiple sources for a single column, e.g., columns
# coming from a JOIN
return set(
_traverse_ds(
current_column=entry.get(CDATAKeys.COLUMN_NAME.value),
ds_origin_list=[],
current_ds=datasource_map[
entry.get(CDATAKeys.COLUMN_OBJECT_NAME.value)
],
datasource_map=datasource_map,
)
)
# If we don't have any logical sources (projections, aggregations, etc.) We'll stick to
# a single table origin
return {
DataSource(
name=entry.get(CDATAKeys.COLUMN_OBJECT_NAME.value),
location=entry.get(CDATAKeys.SCHEMA_NAME.value),
source_type=ViewType.DATA_BASE_TABLE,
)
}
def _traverse_ds(
current_column: str,
ds_origin_list: List[DataSource],
current_ds: DataSource,
datasource_map: Optional[DataSourceMap],
) -> List[DataSource]:
"""
Traverse the ds dict jumping from target -> source columns and getting the right parent.
We keep inspecting current datasources and will append to the origin list the ones
that are not LOGICAL
"""
if current_ds.source_type != ViewType.LOGICAL:
ds_origin_list.append(current_ds)
else:
# Based on our current column, find the parents from the mappings in the current_ds
current_ds_mapping: DataSourceMapping = current_ds.mapping.get(current_column)
if current_ds_mapping:
for parent in current_ds_mapping.parents:
parent_ds = datasource_map.get(parent.parent)
if not parent_ds:
raise CDATAParsingError(
f"Can't find parent [{parent.parent}] for column [{current_column}]"
)
# Traverse from the source column in the parent mapping
_traverse_ds(
current_column=parent.source,
ds_origin_list=ds_origin_list,
current_ds=parent_ds,
datasource_map=datasource_map,
)
else:
logger.info(
f"Can't find mapping for column [{current_column}] in [{current_ds}]. "
f"We still have to implement `calculatedViewAttributes`."
)
return ds_origin_list
def _read_attributes(
tree: ET.Element, ns: dict, datasource_map: Optional[DataSourceMap] = None
) -> ParsedLineage:
"""Compute the lineage based from the attributes"""
lineage = ParsedLineage()
attribute_list = tree.find(CDATAKeys.ATTRIBUTES.value, ns) if tree else None
if not attribute_list:
return lineage
for attribute in attribute_list.findall(CDATAKeys.ATTRIBUTE.value, ns):
key_mapping = attribute.find(CDATAKeys.KEY_MAPPING.value, ns)
data_sources = _get_column_datasources(
entry=key_mapping, datasource_map=datasource_map
)
attr_lineage = ParsedLineage(
mappings=[
ColumnMapping(
data_source=ds,
sources=[key_mapping.get(CDATAKeys.COLUMN_NAME.value)],
target=attribute.get(CDATAKeys.ID.value),
)
for ds in data_sources
]
)
lineage += attr_lineage
return lineage
def _read_calculated_attributes(
tree: ET.Element,
ns: dict,
base_lineage: ParsedLineage,
key: CalculatedAttrKey = CalculatedAttrKey.CALCULATED_ATTRIBUTE,
) -> ParsedLineage:
"""Compute the lineage based on the calculated attributes"""
lineage = ParsedLineage()
calculated_attrs = tree.find(CDATAKeys.CALCULATED_ATTRS.value, ns)
if not calculated_attrs:
return lineage
for calculated_attr in calculated_attrs.findall(key.value, ns):
formula = (
calculated_attr.find(CDATAKeys.KEY_CALCULATION.value, ns)
.find(CDATAKeys.FORMULA.value, ns)
.text
)
lineage += _explode_formula(
target=calculated_attr.get(CDATAKeys.ID.value),
formula=formula,
base_lineage=base_lineage,
)
return lineage
def _read_base_measures(
tree: ET.Element, ns: dict, datasource_map: Optional[DataSourceMap] = None
) -> ParsedLineage:
"""
Compute the lineage based on the base measures.
For CalculationViews, we have a dictionary of pre-defined DataSources. For the rest,
we'll default to Table DataSources with the given information in the measure.
See examples cdata_calculation_view.xml and cdata_attribute_view.xml in test resources.
"""
lineage = ParsedLineage()
base_measures = tree.find(CDATAKeys.BASE_MEASURES.value, ns)
if not base_measures:
return lineage
for measure in base_measures.findall(CDATAKeys.MEASURE.value, ns):
measure_mapping = measure.find(CDATAKeys.MEASURE_MAPPING.value, ns)
data_sources = _get_column_datasources(
entry=measure_mapping, datasource_map=datasource_map
)
measure_lineage = ParsedLineage(
mappings=[
ColumnMapping(
data_source=ds,
sources=[measure_mapping.get(CDATAKeys.COLUMN_NAME.value)],
target=measure.get(CDATAKeys.ID.value),
)
for ds in data_sources
]
)
lineage += measure_lineage
return lineage
def _explode_formula(
target: str, formula: str, base_lineage: ParsedLineage
) -> ParsedLineage:
"""
Explode the formula and extract the columns
Args:
formula: formula to extract involved columns from
base_lineage: parsed lineage of the main attributes. We'll use this to pick up the original lineage columns
Returns:
Parsed Lineage from the formula
"""
column_ds = {
match.group(1): base_lineage.find_target(match.group(1)).data_source
for match in FORMULA_PATTERN.finditer(formula)
}
# Group every datasource (key) with a list of the involved columns (values)
ds_columns = defaultdict(list)
for column, ds in column_ds.items():
ds_columns[ds].append(column)
return ParsedLineage(
mappings=[
ColumnMapping(
# We get the source once we find the mapping of the target
data_source=data_source,
sources=columns,
target=target,
formula=formula,
)
for data_source, columns in ds_columns.items()
]
)
parse_registry = enum_register()
@parse_registry.add(ViewType.ANALYTIC_VIEW.value)
def _(cdata: str) -> ParsedLineage:
"""Parse the CDATA XML for Analytics View"""
ns = NAMESPACE_DICT[ViewType.ANALYTIC_VIEW.value]
tree = ET.fromstring(cdata)
measure_group = tree.find(CDATAKeys.PRIVATE_MEASURE_GROUP.value, ns)
# TODO: Handle lineage from calculatedMeasures, restrictedMeasures and sharedDimensions
return _read_attributes(measure_group, ns)
@parse_registry.add(ViewType.ATTRIBUTE_VIEW.value)
def _(cdata: str) -> ParsedLineage:
"""Parse the CDATA XML for Analytics View"""
ns = NAMESPACE_DICT[ViewType.ATTRIBUTE_VIEW.value]
tree = ET.fromstring(cdata)
attribute_lineage = _read_attributes(tree=tree, ns=ns)
calculated_attrs_lineage = _read_calculated_attributes(
tree=tree, ns=ns, base_lineage=attribute_lineage
)
base_measure_lineage = _read_base_measures(tree=tree, ns=ns, datasource_map=None)
return attribute_lineage + calculated_attrs_lineage + base_measure_lineage
@parse_registry.add(ViewType.CALCULATION_VIEW.value)
def _(cdata: str) -> ParsedLineage:
"""
Parse the CDATA XML for Calculation View
We can think of the DataSources in a CV as:
- "real" sources: they are tables or other views that are used in the calculation
- "logical" sources: as internal projections, aggregations, etc.
The behavior is:
- The LogicalModel attributes can be linked - transitively - to either a "real" or a "logical" source.
- The "logical" sources are defined inside each `calculationView`
- Each `calculationView` can have an `input` node that can either be "real" or "logical"
- We can identify "real" inputs as their `<input node="#<ID>">` will match the <DataSource id="ID" type="...">
When building the dataSources here we need to ensure we iterate over the calculationViews, and during
the _read_ds we'll iteratively traverse the dataSources to find the actual table.
Internally, we'll identify "logical" dataSources by giving them a list of column mappings, which we'll
use to identify the actual source.
"""
# TODO: Handle lineage from calculatedMeasure, restrictedMeasure and sharedDimesions
ns = NAMESPACE_DICT[ViewType.CALCULATION_VIEW.value]
tree = ET.fromstring(cdata)
# Prepare a dictionary of defined data sources
datasource_map = _parse_cv_data_sources(tree=tree, ns=ns)
# Iterate over the Logical Model attributes
logical_model = tree.find(CDATAKeys.LOGICAL_MODEL.value, ns)
attribute_lineage = _read_attributes(
tree=logical_model, ns=ns, datasource_map=datasource_map
)
calculated_attrs_lineage = _read_calculated_attributes(
tree=tree,
ns=ns,
base_lineage=attribute_lineage,
key=CalculatedAttrKey.CALCULATED_VIEW_ATTRIBUTE,
)
base_measure_lineage = _read_base_measures(
tree=logical_model, ns=ns, datasource_map=datasource_map
)
return attribute_lineage + calculated_attrs_lineage + base_measure_lineage
def _parse_cv_data_sources(tree: ET.Element, ns: dict) -> DataSourceMap:
"""
Parse the real and logical data sources of a CV
The logical (`calculationViews`) have the following shape:
```
<calculationViews>
<calculationView xsi:type="Calculation:AggregationView" id="Aggregation_1">
<descriptions/>
<viewAttributes>
<viewAttribute id="MANDT"/>
...
</viewAttributes>
<calculatedViewAttributes>
<calculatedViewAttribute datatype="INTEGER" id="USAGE_PCT" expressionLanguage="COLUMN_ENGINE">
<formula>&quot;SEATSOCC_ALL&quot;/&quot;SEATSMAX_ALL&quot;</formula>
</calculatedViewAttribute>
...
</calculatedViewAttributes>
<input node="#AT_SFLIGHT">
<mapping xsi:type="Calculation:AttributeMapping" target="MANDT" source="MANDT"/>
...
</input>
<input emptyUnionBehavior="NO_ROW" node="#Projection_1">
<mapping xsi:type="Calculation:ConstantAttributeMapping" target="CARRNAME" null="true" value=""/>
...
</input>
...
</calculationView>
...
</calculationViews>
```
"""
datasource_map = DataSourceMap({})
for ds in tree.find(CDATAKeys.DATA_SOURCES.value, ns).findall(
CDATAKeys.DATA_SOURCE.value, ns
):
column_object = ds.find(CDATAKeys.COLUMN_OBJECT.value, ns)
# we can't rely on the falsy value of the object even if present in the XML
# If columnObject is informed, we're talking about a table
if column_object is not None:
ds_value = DataSource(
name=column_object.get(CDATAKeys.COLUMN_OBJECT_NAME.value),
location=column_object.get(CDATAKeys.SCHEMA_NAME.value),
source_type=ViewType.DATA_BASE_TABLE,
)
# or a package object
else:
ds_value = DataSource(
name=ds.get(CDATAKeys.ID.value),
location=ds.find(CDATAKeys.RESOURCE_URI.value).text,
source_type=ViewType.__members__[ds.get(CDATAKeys.TYPE.value)],
)
datasource_map[ds.get(CDATAKeys.ID.value)] = ds_value
calculation_views = tree.find(CDATAKeys.CALCULATION_VIEWS.value, ns)
if calculation_views is None:
return datasource_map
for cv in calculation_views.findall(CDATAKeys.CALCULATION_VIEW.value, ns):
mappings = _build_mappings(calculation_view=cv, ns=ns)
datasource_map[cv.get(CDATAKeys.ID.value)] = DataSource(
name=cv.get(CDATAKeys.ID.value),
location=None,
mapping={mapping.target: mapping for mapping in mappings},
source_type=ViewType.LOGICAL,
)
return datasource_map
def _build_mappings(calculation_view: ET.Element, ns: dict) -> List[DataSourceMapping]:
"""
Build the DataSourceMappings from each `input` inside a Calculation View tree.
Note how we can have:
```
<input emptyUnionBehavior="NO_ROW" node="#Aggregation_1">
<mapping xsi:type="Calculation:AttributeMapping" target="MANDT" source="MANDT"/>
...
</input>
<input emptyUnionBehavior="NO_ROW" node="#Projection_1">
<mapping xsi:type="Calculation:AttributeMapping" target="MANDT" source="MANDT"/>
...
</input>
```
Where a single target column `MANDT` comes from multiple sources. We need to consider
this when building the `parent` field in DataSourceMapping.
1. First, create a single list of all the mappings from all the inputs independently
2. Then, group by `target` and listagg the `parent`s
TODO: We still need to take care of mappings without source, since those come from
`calculatedViewAttributes` where we should handle the formula. Check `cdata_calculation_view.xml`
and take the `USAGE_PCT` as an example.
"""
input_mappings = _build_input_mappings(calculation_view=calculation_view, ns=ns)
# calculated_view_attrs = _build_cv_attributes(
# calculation_view=calculation_view, ns=ns, input_mappings=input_mappings
# )
return input_mappings
def _build_input_mappings(
calculation_view: ET.Element, ns: dict
) -> List[DataSourceMapping]:
"""Map input nodes"""
mappings = []
for input_node in calculation_view.findall(CDATAKeys.INPUT.value, ns):
for mapping in input_node.findall(CDATAKeys.MAPPING.value, ns):
if mapping.get(CDATAKeys.SOURCE.value) and mapping.get(
CDATAKeys.TARGET.value
):
mappings.append(
DataSourceMapping(
target=mapping.get(CDATAKeys.TARGET.value),
parents=[
ParentSource(
source=mapping.get(CDATAKeys.SOURCE.value),
parent=input_node.get(CDATAKeys.NODE.value).replace(
"#", ""
),
)
],
)
)
return _group_mappings(mappings)
def _build_cv_attributes(
calculation_view: ET.Element, ns: dict, input_mappings: List[DataSourceMapping]
) -> List[DataSourceMapping]:
"""Extract mapping from `calculatedViewAttribute` formulas"""
mappings = []
view_attrs = calculation_view.find(CDATAKeys.CALCULATION_VIEW_ATTRIBUTES.value, ns)
if view_attrs is None:
return mappings
for view_attr in view_attrs.findall(CDATAKeys.CALCULATION_VIEW_ATTRIBUTE.value, ns):
formula = (
view_attr.find(CDATAKeys.FORMULA.value, ns).text
if view_attr.find(CDATAKeys.FORMULA.value, ns) is not None
else None
)
if not formula:
logger.debug(f"Skipping formula without expression at {view_attr}")
continue
involved_columns = FORMULA_PATTERN.findall(formula)
for col in involved_columns:
# Find the mapping for the involved column
mapping = next(
(mapping for mapping in input_mappings if mapping.target == col), None
)
if not mapping:
logger.debug(
f"Can't find mapping for column [{col}] in [{input_mappings}]"
)
continue
mappings.append(
DataSourceMapping(
target=view_attr.get(CDATAKeys.ID.value),
parents=mapping.parents,
)
)
return _group_mappings(mappings)
def _group_mappings(mappings: List[DataSourceMapping]) -> List[DataSourceMapping]:
"""Group the mappings by target column and listagg the parents"""
# Sort the data by the target field
mappings.sort(key=lambda x: x.target)
# Use groupby to group by the target field
grouped_data = [
DataSourceMapping(
target=target,
parents=list(itertools.chain.from_iterable(item.parents for item in group)),
)
for target, group in itertools.groupby(mappings, key=lambda x: x.target)
]
return grouped_data

View File

@ -0,0 +1,159 @@
# 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.
"""
SAP Hana lineage module
"""
import traceback
from typing import Iterable, Optional
from sqlalchemy import text
from metadata.generated.schema.api.lineage.addLineage import AddLineageRequest
from metadata.generated.schema.entity.data.table import Table
from metadata.generated.schema.entity.services.connections.database.sapHanaConnection import (
SapHanaConnection,
)
from metadata.generated.schema.entity.services.ingestionPipelines.status import (
StackTraceError,
)
from metadata.generated.schema.metadataIngestion.workflow import (
Source as WorkflowSource,
)
from metadata.ingestion.api.models import Either
from metadata.ingestion.api.steps import InvalidSourceException, Source
from metadata.ingestion.ometa.ometa_api import OpenMetadata
from metadata.ingestion.source.connections import get_test_connection_fn
from metadata.ingestion.source.database.saphana.cdata_parser import (
ParsedLineage,
parse_registry,
)
from metadata.ingestion.source.database.saphana.models import SapHanaLineageModel
from metadata.ingestion.source.database.saphana.queries import SAPHANA_LINEAGE
from metadata.utils.logger import ingestion_logger
from metadata.utils.ssl_manager import get_ssl_connection
logger = ingestion_logger()
class SaphanaLineageSource(Source):
"""
Get the lineage information of:
- calculationview
- analyticview
- attributeview
We support the following relationships:
- Analytic View and Attribute View based on a Table
- Calculation View based on an Analytic View, Attribute View, Calculation View or Table
Parse the CDATA XML definition from _SYS_REPO.ACTIVE_OBJECT
"""
def __init__(
self,
config: WorkflowSource,
metadata: OpenMetadata,
get_engine: bool = True,
):
super().__init__()
self.config = config
self.metadata = metadata
self.service_connection = self.config.serviceConnection.root.config
self.source_config = self.config.sourceConfig.config
self.engine = (
get_ssl_connection(self.service_connection) if get_engine else None
)
logger.info(
"Initializing SAP Hana Lineage Source. Note that we'll parse the lineage from CDATA XML definition "
+ "from _SYS_REPO.ACTIVE_OBJECT and we won't use the time-specific input parameters."
)
def prepare(self):
"""By default, there's nothing to prepare"""
@classmethod
def create(
cls, config_dict, metadata: OpenMetadata, pipeline_name: Optional[str] = None
):
config: WorkflowSource = WorkflowSource.model_validate(config_dict)
connection: SapHanaConnection = config.serviceConnection.root.config
if not isinstance(connection, SapHanaConnection):
raise InvalidSourceException(
f"Expected SapHanaConnection, but got {connection}"
)
return cls(config, metadata)
def close(self) -> None:
self.engine.dispose()
def _iter(self, *_, **__) -> Iterable[Either[AddLineageRequest]]:
"""
Based on the query logs, prepare the lineage
and send it to the sink
"""
with self.engine.connect() as conn:
result = conn.execution_options(
stream_results=True, max_row_buffer=100
).execute(text(SAPHANA_LINEAGE))
for row in result:
try:
lineage_model = SapHanaLineageModel.validate(dict(row))
yield from self.parse_cdata(
metadata=self.metadata, lineage_model=lineage_model
)
except Exception as exc:
self.status.failed(
error=StackTraceError(
name=row["OBJECT_NAME"],
error=f"Error validating lineage model due to [{exc}]",
stackTrace=traceback.format_exc(),
)
)
def parse_cdata(
self, metadata: OpenMetadata, lineage_model: SapHanaLineageModel
) -> Iterable[Either[AddLineageRequest]]:
"""Parse the CDATA XML definition from _SYS_REPO.ACTIVE_OBJECT"""
parse_fn = parse_registry.registry.get(lineage_model.object_suffix.value)
try:
parsed_lineage: ParsedLineage = parse_fn(lineage_model.cdata)
to_entity: Table = metadata.get_by_name(
entity=Table,
fqn=lineage_model.get_fqn(
metadata=metadata,
service_name=self.config.serviceName,
),
)
if to_entity:
yield from parsed_lineage.to_request(
metadata=metadata,
service_name=self.config.serviceName,
to_entity=to_entity,
)
except Exception as exc:
error = (
f"Error parsing CDATA XML for {lineage_model.object_suffix} at "
+ f"{lineage_model.package_id}/{lineage_model.object_name} due to [{exc}]"
)
self.status.failed(
error=StackTraceError(
name=lineage_model.object_name,
error=error,
stackTrace=traceback.format_exc(),
)
)
def test_connection(self) -> None:
test_connection_fn = get_test_connection_fn(self.service_connection)
test_connection_fn(self.engine)

View File

@ -0,0 +1,66 @@
# 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.
"""
SAP Hana lineage module
"""
from enum import Enum
from pydantic import Field, computed_field
from typing_extensions import Annotated
from metadata.generated.schema.entity.data.table import Table
from metadata.ingestion.models.custom_pydantic import BaseModel
from metadata.ingestion.ometa.ometa_api import OpenMetadata
from metadata.utils import fqn
SYS_BIC_SCHEMA_NAME = "_SYS_BIC"
class ViewType(Enum):
"""Supported SAP Hana Views"""
CALCULATION_VIEW = "calculationview"
ANALYTIC_VIEW = "analyticview"
ATTRIBUTE_VIEW = "attributeview"
# Artificially set to define calculationView internal models. This won't come from the ACTIVE_OBJECT table
LOGICAL = "logical"
DATA_BASE_TABLE = "table"
class SapHanaLineageModel(BaseModel):
"""SAP Hana Lineage model from _SYS_REPO.ACTIVE_OBJECT"""
package_id: Annotated[
str, Field(..., description="Package ID that hosts the model code")
]
object_name: Annotated[str, Field(..., description="View Name")]
object_suffix: Annotated[ViewType, Field(..., description="View Type")]
cdata: Annotated[str, Field(..., description="XML representation of the model")]
@computed_field
@property
def name(self) -> str:
"""
Actual name representation of the view in _SYS_BIC.VIEWS.
This is the name in OpenMetadata
"""
return f"{self.package_id}/{self.object_name}"
def get_fqn(self, metadata: OpenMetadata, service_name: str) -> str:
"""Build OM's FQN with the static schema name from _SYS_BIC"""
return fqn.build(
metadata,
entity_type=Table,
service_name=service_name,
database_name=None,
schema_name=SYS_BIC_SCHEMA_NAME,
table_name=self.name,
)

View File

@ -0,0 +1,23 @@
# 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.
"""
SAP Hana DB queries for metadata extraction
"""
SAPHANA_LINEAGE = """
SELECT
PACKAGE_ID,
OBJECT_NAME,
OBJECT_SUFFIX,
CDATA
FROM _SYS_REPO.ACTIVE_OBJECT
WHERE OBJECT_SUFFIX IN ('analyticview', 'attributeview', 'calculationview');
"""

View File

@ -0,0 +1,79 @@
<?xml version="1.0" encoding="UTF-8"?>
<Cube:cube xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns:AccessControl="http://www.sap.com/ndb/SQLCoreModelAccessControl.ecore" xmlns:Cube="http://www.sap.com/ndb/BiModelCube.ecore" schemaVersion="1.5" id="AN_SBOOK" applyPrivilegeType="ANALYTIC_PRIVILEGE" checkAnalyticPrivileges="true" defaultClient="$$client$$" defaultLanguage="$$language$$" hierarchiesSQLEnabled="false" translationRelevant="true" visibility="reportingEnabled">
<origin/>
<descriptions defaultDescription="AN_SBOOK"/>
<metadata changedAt="2024-08-22 08:56:08.244"/>
<localVariables/>
<informationModelLayout relativeWidthScenario="27"/>
<privateMeasureGroup id="MeasureGroup">
<attributes>
<attribute id="MANDT" order="1" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="MANDT"/>
<keyMapping schemaName="SFLIGHT" columnObjectName="SBOOK" columnName="MANDT"/>
</attribute>
<attribute id="CARRID" order="2" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="CARRID"/>
<keyMapping schemaName="SFLIGHT" columnObjectName="SBOOK" columnName="CARRID"/>
</attribute>
<attribute id="CONNID" order="3" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="CONNID"/>
<keyMapping schemaName="SFLIGHT" columnObjectName="SBOOK" columnName="CONNID"/>
</attribute>
<attribute id="FLDATE" order="4" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="FLDATE"/>
<keyMapping schemaName="SFLIGHT" columnObjectName="SBOOK" columnName="FLDATE"/>
</attribute>
<attribute id="FORCURKEY" order="6" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="FORCURKEY"/>
<keyMapping schemaName="SFLIGHT" columnObjectName="SBOOK" columnName="FORCURKEY"/>
</attribute>
<attribute id="LOCCURKEY" order="8" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="LOCCURKEY"/>
<keyMapping schemaName="SFLIGHT" columnObjectName="SBOOK" columnName="LOCCURKEY"/>
</attribute>
</attributes>
<calculatedAttributes/>
<privateDataFoundation>
<tableProxies>
<tableProxy centralTable="true">
<table schemaName="SFLIGHT" columnObjectName="SBOOK"/>
<columnFilter columnName="CANCELLED">
<valueFilter xsi:type="AccessControl:SingleValueFilter" including="false" value="X"/>
</columnFilter>
</tableProxy>
</tableProxies>
<joins/>
<layout>
<shapes>
<shape modelObjectName="SBOOK" modelObjectNameSpace="SFLIGHT" modelObjectType="catalog">
<upperLeftCorner x="70" y="30"/>
</shape>
</shapes>
</layout>
</privateDataFoundation>
<baseMeasures>
<measure id="FORCURAM" order="5" aggregationType="sum" engineAggregation="sum" measureType="simple">
<descriptions defaultDescription="FORCURAM"/>
<measureMapping schemaName="SFLIGHT" columnObjectName="SBOOK" columnName="FORCURAM"/>
</measure>
<measure id="LOCCURAM" order="7" aggregationType="sum" engineAggregation="sum" measureType="simple">
<descriptions defaultDescription="LOCCURAM"/>
<measureMapping schemaName="SFLIGHT" columnObjectName="SBOOK" columnName="LOCCURAM"/>
</measure>
</baseMeasures>
<calculatedMeasures/>
<restrictedMeasures/>
<sharedDimensions/>
<layout>
<shapes>
<shape modelObjectName="MEASURE_GROUP" modelObjectType="repository">
<upperLeftCorner x="300" y="200"/>
</shape>
<shape modelObjectName="LogicalView" modelObjectNameSpace="MeasureGroup" modelObjectType="repository">
<upperLeftCorner x="40" y="85"/>
<rectangleSize/>
</shape>
</shapes>
</layout>
</privateMeasureGroup>
</Cube:cube>

View File

@ -0,0 +1,135 @@
<?xml version="1.0" encoding="UTF-8"?>
<Dimension:dimension xmlns:Dimension="http://www.sap.com/ndb/BiModelDimension.ecore" schemaVersion="1.2" id="AT_SFLIGHT" applyPrivilegeType="ANALYTIC_PRIVILEGE" checkAnalyticPrivileges="true" defaultClient="$$client$$" defaultLanguage="$$language$$" hierarchiesSQLEnabled="false" translationRelevant="true" visibility="internal" dimensionType="Standard">
<origin/>
<descriptions defaultDescription="AT_SFLIGHT"/>
<metadata changedAt="2024-08-28 17:28:04.084"/>
<informationModelLayout relativeWidthScenario="27"/>
<attributes>
<attribute id="MANDT" key="true" order="1" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="MANDT"/>
<keyMapping schemaName="SFLIGHT" columnObjectName="SFLIGHT" columnName="MANDT"/>
</attribute>
<attribute id="CARRID" key="true" order="2" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="CARRID"/>
<keyMapping schemaName="SFLIGHT" columnObjectName="SFLIGHT" columnName="CARRID"/>
</attribute>
<attribute id="CONNID" key="true" order="3" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="CONNID"/>
<keyMapping schemaName="SFLIGHT" columnObjectName="SFLIGHT" columnName="CONNID"/>
</attribute>
<attribute id="FLDATE" key="true" order="4" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="FLDATE"/>
<keyMapping schemaName="SFLIGHT" columnObjectName="SFLIGHT" columnName="FLDATE"/>
</attribute>
<attribute id="PRICE" order="5" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="PRICE"/>
<keyMapping schemaName="SFLIGHT" columnObjectName="SFLIGHT" columnName="PRICE"/>
</attribute>
<attribute id="CURRENCY" order="6" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="CURRENCY"/>
<keyMapping schemaName="SFLIGHT" columnObjectName="SFLIGHT" columnName="CURRENCY"/>
</attribute>
<attribute id="PLANETYPE" order="7" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="PLANETYPE"/>
<keyMapping schemaName="SFLIGHT" columnObjectName="SFLIGHT" columnName="PLANETYPE"/>
</attribute>
<attribute id="SEATSMAX" order="8" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="SEATSMAX"/>
<keyMapping schemaName="SFLIGHT" columnObjectName="SFLIGHT" columnName="SEATSMAX"/>
</attribute>
<attribute id="SEATSOCC" order="9" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="SEATSOCC"/>
<keyMapping schemaName="SFLIGHT" columnObjectName="SFLIGHT" columnName="SEATSOCC"/>
</attribute>
<attribute id="PAYMENTSUM" order="10" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="PAYMENTSUM"/>
<keyMapping schemaName="SFLIGHT" columnObjectName="SFLIGHT" columnName="PAYMENTSUM"/>
</attribute>
<attribute id="SEATSMAX_B" order="11" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="SEATSMAX_B"/>
<keyMapping schemaName="SFLIGHT" columnObjectName="SFLIGHT" columnName="SEATSMAX_B"/>
</attribute>
<attribute id="SEATSOCC_B" order="12" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="SEATSOCC_B"/>
<keyMapping schemaName="SFLIGHT" columnObjectName="SFLIGHT" columnName="SEATSOCC_B"/>
</attribute>
<attribute id="SEATSMAX_F" order="13" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="SEATSMAX_F"/>
<keyMapping schemaName="SFLIGHT" columnObjectName="SFLIGHT" columnName="SEATSMAX_F"/>
</attribute>
<attribute id="SEATSOCC_F" order="14" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="SEATSOCC_F"/>
<keyMapping schemaName="SFLIGHT" columnObjectName="SFLIGHT" columnName="SEATSOCC_F"/>
</attribute>
<attribute id="CARRNAME" order="18" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="CARRNAME"/>
<keyMapping schemaName="SFLIGHT" columnObjectName="SCARR" columnName="CARRNAME"/>
</attribute>
</attributes>
<calculatedAttributes>
<calculatedAttribute id="SEATSMAX_ALL" hidden="false" order="15" semanticType="empty" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription=""/>
<keyCalculation datatype="INTEGER" expressionLanguage="COLUMN_ENGINE">
<formula>&quot;SEATSMAX&quot;+&quot;SEATSMAX_B&quot;+&quot;SEATSMAX_F&quot;</formula>
</keyCalculation>
</calculatedAttribute>
<calculatedAttribute id="SEATSOCC_ALL" hidden="false" order="16" semanticType="empty" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="SEATSOCC_ALL"/>
<keyCalculation datatype="INTEGER" expressionLanguage="COLUMN_ENGINE">
<formula>&quot;SEATSOCC&quot;+&quot;SEATSOCC_B&quot;+&quot;SEATSOCC_F&quot;</formula>
</keyCalculation>
</calculatedAttribute>
<calculatedAttribute id="RETURN_INDEX" hidden="false" order="17" semanticType="empty" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="RETURN_INDEX"/>
<keyCalculation datatype="INTEGER" expressionLanguage="COLUMN_ENGINE">
<formula>&quot;PRICE&quot;*&quot;SEATSMAX&quot;</formula>
</keyCalculation>
</calculatedAttribute>
<calculatedAttribute id="FULL_CARRNAME" hidden="false" order="19" semanticType="empty" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="FULL_CARRNAME"/>
<keyCalculation datatype="VARCHAR" expressionLanguage="COLUMN_ENGINE" length="256">
<formula>&quot;MANDT&quot;+&quot;CARRNAME&quot;</formula>
</keyCalculation>
</calculatedAttribute>
</calculatedAttributes>
<privateDataFoundation>
<tableProxies>
<tableProxy>
<table schemaName="SFLIGHT" columnObjectName="SFLIGHT"/>
</tableProxy>
<tableProxy>
<table schemaName="SFLIGHT" columnObjectName="SCARR"/>
</tableProxy>
</tableProxies>
<joins>
<join>
<leftTable schemaName="SFLIGHT" columnObjectName="SFLIGHT"/>
<rightTable schemaName="SFLIGHT" columnObjectName="SCARR"/>
<leftColumns>
<columnName>MANDT</columnName>
<columnName>CARRID</columnName>
</leftColumns>
<rightColumns>
<columnName>MANDT</columnName>
<columnName>CARRID</columnName>
</rightColumns>
<properties joinOperator="Equal" joinType="referential"/>
</join>
</joins>
<layout>
<shapes>
<shape modelObjectName="SFLIGHT" modelObjectNameSpace="SFLIGHT" modelObjectType="catalog">
<upperLeftCorner x="70" y="30"/>
</shape>
<shape modelObjectName="SCARR" modelObjectNameSpace="SFLIGHT" modelObjectType="catalog">
<upperLeftCorner x="310" y="30"/>
</shape>
<shape modelObjectName="DataFoundation" modelObjectNameSpace="DataFoundation" modelObjectType="repository">
<upperLeftCorner x="40" y="85"/>
<rectangleSize height="0" width="0"/>
</shape>
</shapes>
</layout>
</privateDataFoundation>
<hierarchies/>
</Dimension:dimension>

View File

@ -0,0 +1,193 @@
<?xml version="1.0" encoding="UTF-8"?>
<Calculation:scenario xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns:Calculation="http://www.sap.com/ndb/BiModelCalculation.ecore" schemaVersion="2.3" id="CV_SFLIGHT_SBOOK" applyPrivilegeType="SQL_ANALYTIC_PRIVILEGE" defaultClient="$$client$$" defaultLanguage="$$language$$" hierarchiesSQLEnabled="false" translationRelevant="true" visibility="reportingEnabled" calculationScenarioType="TREE_BASED" dataCategory="CUBE" enforceSqlExecution="false" executionSemantic="UNDEFINED" outputViewType="Projection">
<origin/>
<descriptions defaultDescription="CV_SFLIGHT_SBOOK"/>
<metadata changedAt="2024-08-22 09:27:52.856"/>
<localVariables/>
<variableMappings/>
<informationModelLayout relativeWidthScenario="27"/>
<dataSources>
<DataSource id="AT_SFLIGHT" type="ATTRIBUTE_VIEW">
<viewAttributes allViewAttributes="true"/>
<resourceUri>/SFLIGHT.MODELING/attributeviews/AT_SFLIGHT</resourceUri>
</DataSource>
<DataSource id="AN_SBOOK" type="ANALYTIC_VIEW">
<viewAttributes allViewAttributes="true"/>
<resourceUri>/SFLIGHT.MODELING/analyticviews/AN_SBOOK</resourceUri>
</DataSource>
</dataSources>
<calculationViews>
<calculationView xsi:type="Calculation:AggregationView" id="Aggregation_1">
<descriptions/>
<viewAttributes>
<viewAttribute id="MANDT"/>
<viewAttribute id="CARRID"/>
<viewAttribute id="CONNID"/>
<viewAttribute id="FLDATE"/>
<viewAttribute id="CARRNAME"/>
<viewAttribute id="SEATSMAX_ALL"/>
<viewAttribute id="SEATSOCC_ALL"/>
<viewAttribute id="PAYMENTSUM"/>
<viewAttribute id="RETURN_INDEX"/>
</viewAttributes>
<calculatedViewAttributes>
<calculatedViewAttribute datatype="INTEGER" id="USAGE_PCT" expressionLanguage="COLUMN_ENGINE">
<formula>&quot;SEATSOCC_ALL&quot;/&quot;SEATSMAX_ALL&quot;</formula>
</calculatedViewAttribute>
</calculatedViewAttributes>
<input node="#AT_SFLIGHT">
<mapping xsi:type="Calculation:AttributeMapping" target="MANDT" source="MANDT"/>
<mapping xsi:type="Calculation:AttributeMapping" target="CARRID" source="CARRID"/>
<mapping xsi:type="Calculation:AttributeMapping" target="CONNID" source="CONNID"/>
<mapping xsi:type="Calculation:AttributeMapping" target="FLDATE" source="FLDATE"/>
<mapping xsi:type="Calculation:AttributeMapping" target="CARRNAME" source="CARRNAME"/>
<mapping xsi:type="Calculation:AttributeMapping" target="SEATSMAX_ALL" source="SEATSMAX_ALL"/>
<mapping xsi:type="Calculation:AttributeMapping" target="SEATSOCC_ALL" source="SEATSOCC_ALL"/>
<mapping xsi:type="Calculation:AttributeMapping" target="PAYMENTSUM" source="PAYMENTSUM"/>
<mapping xsi:type="Calculation:AttributeMapping" target="RETURN_INDEX" source="RETURN_INDEX"/>
</input>
</calculationView>
<calculationView xsi:type="Calculation:ProjectionView" id="Projection_1">
<descriptions/>
<viewAttributes>
<viewAttribute id="MANDT"/>
<viewAttribute id="CARRID"/>
<viewAttribute id="CONNID"/>
<viewAttribute id="FLDATE"/>
<viewAttribute id="LOCCURAM"/>
<viewAttribute id="LOCCURKEY"/>
<viewAttribute id="FORCURAM"/>
<viewAttribute id="FORCURKEY"/>
</viewAttributes>
<calculatedViewAttributes/>
<input node="#AN_SBOOK">
<mapping xsi:type="Calculation:AttributeMapping" target="MANDT" source="MANDT"/>
<mapping xsi:type="Calculation:AttributeMapping" target="CARRID" source="CARRID"/>
<mapping xsi:type="Calculation:AttributeMapping" target="CONNID" source="CONNID"/>
<mapping xsi:type="Calculation:AttributeMapping" target="FLDATE" source="FLDATE"/>
<mapping xsi:type="Calculation:AttributeMapping" target="LOCCURAM" source="LOCCURAM"/>
<mapping xsi:type="Calculation:AttributeMapping" target="LOCCURKEY" source="LOCCURKEY"/>
<mapping xsi:type="Calculation:AttributeMapping" target="FORCURAM" source="FORCURAM"/>
<mapping xsi:type="Calculation:AttributeMapping" target="FORCURKEY" source="FORCURKEY"/>
</input>
</calculationView>
<calculationView xsi:type="Calculation:UnionView" id="Union_1">
<descriptions/>
<viewAttributes>
<viewAttribute id="MANDT" transparentFilter="false"/>
<viewAttribute id="CARRID" transparentFilter="false"/>
<viewAttribute id="CONNID" transparentFilter="false"/>
<viewAttribute id="FLDATE" transparentFilter="false"/>
<viewAttribute id="CARRNAME" transparentFilter="false"/>
<viewAttribute id="SEATSMAX_ALL" transparentFilter="false"/>
<viewAttribute id="SEATSOCC_ALL" transparentFilter="false"/>
<viewAttribute id="PAYMENTSUM" transparentFilter="false"/>
<viewAttribute id="RETURN_INDEX" transparentFilter="false"/>
<viewAttribute id="USAGE_PCT" transparentFilter="false"/>
</viewAttributes>
<calculatedViewAttributes/>
<input emptyUnionBehavior="NO_ROW" node="#Aggregation_1">
<mapping xsi:type="Calculation:AttributeMapping" target="MANDT" source="MANDT"/>
<mapping xsi:type="Calculation:AttributeMapping" target="CARRID" source="CARRID"/>
<mapping xsi:type="Calculation:AttributeMapping" target="CONNID" source="CONNID"/>
<mapping xsi:type="Calculation:AttributeMapping" target="FLDATE" source="FLDATE"/>
<mapping xsi:type="Calculation:AttributeMapping" target="CARRNAME" source="CARRNAME"/>
<mapping xsi:type="Calculation:AttributeMapping" target="SEATSMAX_ALL" source="SEATSMAX_ALL"/>
<mapping xsi:type="Calculation:AttributeMapping" target="SEATSOCC_ALL" source="SEATSOCC_ALL"/>
<mapping xsi:type="Calculation:AttributeMapping" target="PAYMENTSUM" source="PAYMENTSUM"/>
<mapping xsi:type="Calculation:AttributeMapping" target="RETURN_INDEX" source="RETURN_INDEX"/>
<mapping xsi:type="Calculation:AttributeMapping" target="USAGE_PCT" source="USAGE_PCT"/>
</input>
<input emptyUnionBehavior="NO_ROW" node="#Projection_1">
<mapping xsi:type="Calculation:ConstantAttributeMapping" target="CARRNAME" null="true" value=""/>
<mapping xsi:type="Calculation:ConstantAttributeMapping" target="SEATSMAX_ALL" null="true" value=""/>
<mapping xsi:type="Calculation:ConstantAttributeMapping" target="SEATSOCC_ALL" null="true" value=""/>
<mapping xsi:type="Calculation:ConstantAttributeMapping" target="PAYMENTSUM" null="true" value=""/>
<mapping xsi:type="Calculation:AttributeMapping" target="MANDT" source="MANDT"/>
<mapping xsi:type="Calculation:AttributeMapping" target="CARRID" source="CARRID"/>
<mapping xsi:type="Calculation:AttributeMapping" target="CONNID" source="CONNID"/>
<mapping xsi:type="Calculation:AttributeMapping" target="FLDATE" source="FLDATE"/>
<mapping xsi:type="Calculation:ConstantAttributeMapping" target="RETURN_INDEX" null="true" value=""/>
<mapping xsi:type="Calculation:ConstantAttributeMapping" target="USAGE_PCT" null="true" value=""/>
</input>
</calculationView>
</calculationViews>
<logicalModel id="Union_1">
<descriptions/>
<attributes>
<attribute id="MANDT" order="1" attributeHierarchyActive="false" displayAttribute="false" transparentFilter="false">
<descriptions defaultDescription="MANDT"/>
<keyMapping columnObjectName="Union_1" columnName="MANDT"/>
</attribute>
<attribute id="CARRID" order="2" attributeHierarchyActive="false" displayAttribute="false" transparentFilter="false">
<descriptions defaultDescription="CARRID"/>
<keyMapping columnObjectName="Union_1" columnName="CARRID"/>
</attribute>
<attribute id="CARRNAME" order="3" attributeHierarchyActive="false" displayAttribute="false" transparentFilter="false">
<descriptions defaultDescription="CARRNAME"/>
<keyMapping columnObjectName="Union_1" columnName="CARRNAME"/>
</attribute>
<attribute id="FLDATE" order="4" attributeHierarchyActive="false" displayAttribute="false" transparentFilter="false">
<descriptions defaultDescription="FLDATE"/>
<keyMapping columnObjectName="Union_1" columnName="FLDATE"/>
</attribute>
<attribute id="CONNID" order="5" attributeHierarchyActive="false" displayAttribute="false" transparentFilter="false">
<descriptions defaultDescription="CONNID"/>
<keyMapping columnObjectName="Union_1" columnName="CONNID"/>
</attribute>
<attribute id="SEATSMAX_ALL" order="6" attributeHierarchyActive="false" displayAttribute="false" transparentFilter="false">
<descriptions/>
<keyMapping columnObjectName="Union_1" columnName="SEATSMAX_ALL"/>
</attribute>
<attribute id="SEATSOCC_ALL" order="7" attributeHierarchyActive="false" displayAttribute="false" transparentFilter="false">
<descriptions defaultDescription="SEATSOCC_ALL"/>
<keyMapping columnObjectName="Union_1" columnName="SEATSOCC_ALL"/>
</attribute>
<attribute id="PAYMENTSUM" order="8" attributeHierarchyActive="false" displayAttribute="false" transparentFilter="false">
<descriptions defaultDescription="PAYMENTSUM"/>
<keyMapping columnObjectName="Union_1" columnName="PAYMENTSUM"/>
</attribute>
<attribute id="RETURN_INDEX" order="9" attributeHierarchyActive="false" displayAttribute="false" transparentFilter="false">
<descriptions defaultDescription="RETURN_INDEX"/>
<keyMapping columnObjectName="Union_1" columnName="RETURN_INDEX"/>
</attribute>
</attributes>
<calculatedAttributes/>
<privateDataFoundation>
<tableProxies/>
<joins/>
<layout>
<shapes/>
</layout>
</privateDataFoundation>
<baseMeasures>
<measure id="USAGE_PCT" order="10" aggregationType="sum" measureType="simple">
<descriptions defaultDescription="USAGE_PCT"/>
<measureMapping columnObjectName="Union_1" columnName="USAGE_PCT"/>
</measure>
</baseMeasures>
<calculatedMeasures/>
<restrictedMeasures/>
<localDimensions/>
</logicalModel>
<layout>
<shapes>
<shape expanded="true" modelObjectName="Output" modelObjectNameSpace="MeasureGroup">
<upperLeftCorner x="40" y="85"/>
<rectangleSize/>
</shape>
<shape expanded="true" modelObjectName="Aggregation_1" modelObjectNameSpace="CalculationView">
<upperLeftCorner x="20" y="310"/>
<rectangleSize height="-1" width="-1"/>
</shape>
<shape expanded="true" modelObjectName="Projection_1" modelObjectNameSpace="CalculationView">
<upperLeftCorner x="230" y="320"/>
<rectangleSize height="-1" width="-1"/>
</shape>
<shape expanded="true" modelObjectName="Union_1" modelObjectNameSpace="CalculationView">
<upperLeftCorner x="60" y="180"/>
<rectangleSize height="-1" width="-1"/>
</shape>
</shapes>
</layout>
</Calculation:scenario>

View File

@ -0,0 +1,62 @@
<?xml version="1.0" encoding="UTF-8"?>
<Calculation:scenario xmlns:Calculation="http://www.sap.com/ndb/BiModelCalculation.ecore" schemaVersion="2.3" id="CV_CV" applyPrivilegeType="SQL_ANALYTIC_PRIVILEGE" defaultClient="$$client$$" defaultLanguage="$$language$$" hierarchiesSQLEnabled="false" translationRelevant="true" visibility="reportingEnabled" calculationScenarioType="TREE_BASED" dataCategory="CUBE" enforceSqlExecution="false" executionSemantic="UNDEFINED" outputViewType="Projection">
<origin/>
<descriptions defaultDescription="CV_CV"/>
<metadata changedAt="2024-08-28 11:22:50.623"/>
<localVariables/>
<variableMappings/>
<informationModelLayout relativeWidthScenario="27"/>
<dataSources>
<DataSource id="CV_SFLIGHT_SBOOK" type="CALCULATION_VIEW">
<viewAttributes allViewAttributes="true"/>
<resourceUri>/SFLIGHT.MODELING/calculationviews/CV_SFLIGHT_SBOOK</resourceUri>
</DataSource>
</dataSources>
<calculationViews/>
<logicalModel id="CV_SFLIGHT_SBOOK">
<descriptions/>
<attributes>
<attribute id="MANDT" order="1" attributeHierarchyActive="false" displayAttribute="false" transparentFilter="false">
<descriptions defaultDescription="MANDT"/>
<keyMapping columnObjectName="CV_SFLIGHT_SBOOK" columnName="MANDT"/>
</attribute>
<attribute id="CARRID" order="2" attributeHierarchyActive="false" displayAttribute="false" transparentFilter="false">
<descriptions defaultDescription="CARRID"/>
<keyMapping columnObjectName="CV_SFLIGHT_SBOOK" columnName="CARRID"/>
</attribute>
<attribute id="CARRNAME" order="3" attributeHierarchyActive="false" displayAttribute="false" transparentFilter="false">
<descriptions defaultDescription="CARRNAME"/>
<keyMapping columnObjectName="CV_SFLIGHT_SBOOK" columnName="CARRNAME"/>
</attribute>
<attribute id="RETURN_INDEX" order="5" attributeHierarchyActive="false" displayAttribute="false" transparentFilter="false">
<descriptions defaultDescription="RETURN_INDEX"/>
<keyMapping columnObjectName="CV_SFLIGHT_SBOOK" columnName="RETURN_INDEX"/>
</attribute>
</attributes>
<calculatedAttributes/>
<privateDataFoundation>
<tableProxies/>
<joins/>
<layout>
<shapes/>
</layout>
</privateDataFoundation>
<baseMeasures>
<measure id="USAGE_PCT" order="4" aggregationType="sum" measureType="simple">
<descriptions defaultDescription="USAGE_PCT"/>
<measureMapping columnObjectName="CV_SFLIGHT_SBOOK" columnName="USAGE_PCT"/>
</measure>
</baseMeasures>
<calculatedMeasures/>
<restrictedMeasures/>
<localDimensions/>
</logicalModel>
<layout>
<shapes>
<shape expanded="true" modelObjectName="Output" modelObjectNameSpace="MeasureGroup">
<upperLeftCorner x="40" y="85"/>
<rectangleSize/>
</shape>
</shapes>
</layout>
</Calculation:scenario>

View File

@ -0,0 +1,70 @@
<?xml version="1.0" encoding="UTF-8"?>
<Calculation:scenario xmlns:Calculation="http://www.sap.com/ndb/BiModelCalculation.ecore" schemaVersion="2.3" id="CV_TAB" applyPrivilegeType="SQL_ANALYTIC_PRIVILEGE" defaultClient="$$client$$" defaultLanguage="$$language$$" hierarchiesSQLEnabled="false" translationRelevant="true" visibility="reportingEnabled" calculationScenarioType="TREE_BASED" dataCategory="CUBE" enforceSqlExecution="false" executionSemantic="UNDEFINED" outputViewType="Projection">
<origin/>
<descriptions defaultDescription="CV_TAB"/>
<metadata changedAt="2024-08-28 10:51:46.367"/>
<localVariables/>
<variableMappings/>
<informationModelLayout relativeWidthScenario="27"/>
<dataSources>
<DataSource id="SFLIGHT" type="DATA_BASE_TABLE">
<viewAttributes allViewAttributes="true"/>
<columnObject schemaName="SFLIGHT" columnObjectName="SFLIGHT"/>
</DataSource>
</dataSources>
<calculationViews/>
<logicalModel id="SFLIGHT">
<descriptions/>
<attributes>
<attribute id="MANDT" order="1" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="MANDT"/>
<keyMapping schemaName="SFLIGHT" columnObjectName="SFLIGHT" columnName="MANDT"/>
</attribute>
<attribute id="CARRID" order="2" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="CARRID"/>
<keyMapping schemaName="SFLIGHT" columnObjectName="SFLIGHT" columnName="CARRID"/>
</attribute>
<attribute id="CONNID" order="3" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="CONNID"/>
<keyMapping schemaName="SFLIGHT" columnObjectName="SFLIGHT" columnName="CONNID"/>
</attribute>
<attribute id="FLDATE" order="4" attributeHierarchyActive="false" displayAttribute="false">
<descriptions defaultDescription="FLDATE"/>
<keyMapping schemaName="SFLIGHT" columnObjectName="SFLIGHT" columnName="FLDATE"/>
</attribute>
</attributes>
<calculatedAttributes/>
<privateDataFoundation>
<tableProxies/>
<joins/>
<layout>
<shapes/>
</layout>
</privateDataFoundation>
<baseMeasures>
<measure id="PRICE" order="5" aggregationType="sum" measureType="simple">
<descriptions defaultDescription="PRICE"/>
<measureMapping schemaName="SFLIGHT" columnObjectName="SFLIGHT" columnName="PRICE"/>
</measure>
<measure id="SEATSMAX_B" order="6" aggregationType="sum" measureType="simple">
<descriptions defaultDescription="SEATSMAX_B"/>
<measureMapping schemaName="SFLIGHT" columnObjectName="SFLIGHT" columnName="SEATSMAX_B"/>
</measure>
<measure id="PAYMENTSUM" order="7" aggregationType="sum" measureType="simple">
<descriptions defaultDescription="PAYMENTSUM"/>
<measureMapping schemaName="SFLIGHT" columnObjectName="SFLIGHT" columnName="PAYMENTSUM"/>
</measure>
</baseMeasures>
<calculatedMeasures/>
<restrictedMeasures/>
<localDimensions/>
</logicalModel>
<layout>
<shapes>
<shape expanded="true" modelObjectName="Output" modelObjectNameSpace="MeasureGroup">
<upperLeftCorner x="40" y="85"/>
<rectangleSize/>
</shape>
</shapes>
</layout>
</Calculation:scenario>

View File

@ -0,0 +1,163 @@
# Copyright 2021 Collate
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
# You may obtain a copy of the License at
# http://www.apache.org/licenses/LICENSE-2.0
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
"""
Test SAP Hana source
"""
from pathlib import Path
from metadata.ingestion.source.database.saphana.cdata_parser import (
ColumnMapping,
DataSource,
ParsedLineage,
ViewType,
parse_registry,
)
RESOURCES_DIR = Path(__file__).parent.parent.parent / "resources" / "saphana"
def test_parse_analytic_view() -> None:
"""Read the resource and parse the file"""
with open(RESOURCES_DIR / "cdata_analytic_view.xml") as file:
cdata = file.read()
parse_fn = parse_registry.registry.get(ViewType.ANALYTIC_VIEW.value)
parsed_lineage: ParsedLineage = parse_fn(cdata)
ds = DataSource(
name="SBOOK", location="SFLIGHT", source_type=ViewType.DATA_BASE_TABLE
)
assert parsed_lineage
assert len(parsed_lineage.mappings) == 6
assert parsed_lineage.sources == {ds}
assert parsed_lineage.mappings[0] == ColumnMapping(
data_source=ds,
sources=["MANDT"],
target="MANDT",
)
def test_parse_attribute_view() -> None:
"""Read the resource and parse the file"""
with open(RESOURCES_DIR / "cdata_attribute_view.xml") as file:
cdata = file.read()
parse_fn = parse_registry.registry.get(ViewType.ATTRIBUTE_VIEW.value)
parsed_lineage: ParsedLineage = parse_fn(cdata)
ds = DataSource(
name="SFLIGHT", location="SFLIGHT", source_type=ViewType.DATA_BASE_TABLE
)
assert parsed_lineage
assert len(parsed_lineage.mappings) == 20 # 15 columns + 5 derived from formulas
assert parsed_lineage.sources == {
DataSource(
name="SCARR", location="SFLIGHT", source_type=ViewType.DATA_BASE_TABLE
),
ds,
}
assert parsed_lineage.mappings[0] == ColumnMapping(
data_source=ds,
sources=["MANDT"],
target="MANDT",
)
def test_parse_cv_tab() -> None:
"""Read the resource and parse the file"""
with open(RESOURCES_DIR / "cdata_calculation_view_tab.xml") as file:
cdata = file.read()
parse_fn = parse_registry.registry.get(ViewType.CALCULATION_VIEW.value)
parsed_lineage: ParsedLineage = parse_fn(cdata)
ds = DataSource(
name="SFLIGHT", location="SFLIGHT", source_type=ViewType.DATA_BASE_TABLE
)
assert parsed_lineage
assert len(parsed_lineage.mappings) == 7 # 4 attributes, 3 measures
assert parsed_lineage.sources == {ds}
# Attribute
assert parsed_lineage.mappings[0] == ColumnMapping(
data_source=ds,
sources=["MANDT"],
target="MANDT",
)
# Measure
assert parsed_lineage.mappings[-1] == ColumnMapping(
data_source=ds,
sources=["PAYMENTSUM"],
target="PAYMENTSUM",
)
def test_parse_cv_view() -> None:
"""Read the resource and parse the file"""
with open(RESOURCES_DIR / "cdata_calculation_view_cv.xml") as file:
cdata = file.read()
parse_fn = parse_registry.registry.get(ViewType.CALCULATION_VIEW.value)
parsed_lineage: ParsedLineage = parse_fn(cdata)
ds = DataSource(
name="CV_SFLIGHT_SBOOK",
location="/SFLIGHT.MODELING/calculationviews/CV_SFLIGHT_SBOOK",
source_type=ViewType.CALCULATION_VIEW,
)
assert parsed_lineage
assert len(parsed_lineage.mappings) == 5 # 4 attributes, 1 measure
assert parsed_lineage.sources == {ds}
# Attribute
assert parsed_lineage.mappings[0] == ColumnMapping(
data_source=ds,
sources=["MANDT"],
target="MANDT",
)
# Measure
assert parsed_lineage.mappings[-1] == ColumnMapping(
data_source=ds,
sources=["USAGE_PCT"],
target="USAGE_PCT",
)
def test_parse_cv() -> None:
"""Read the resource and parse the file"""
with open(RESOURCES_DIR / "cdata_calculation_view.xml") as file:
cdata = file.read()
parse_fn = parse_registry.registry.get(ViewType.CALCULATION_VIEW.value)
parsed_lineage: ParsedLineage = parse_fn(cdata)
ds_sbook = DataSource(
name="AN_SBOOK",
location="/SFLIGHT.MODELING/analyticviews/AN_SBOOK",
source_type=ViewType.ANALYTIC_VIEW,
)
ds_sflight = DataSource(
name="AT_SFLIGHT",
location="/SFLIGHT.MODELING/attributeviews/AT_SFLIGHT",
source_type=ViewType.ATTRIBUTE_VIEW,
)
assert parsed_lineage
# Even though we have 9 unique columns, some come from 2 tables, so we have two mappings
assert len(parsed_lineage.mappings) == 13
assert parsed_lineage.sources == {ds_sbook, ds_sflight}
# We can validate that MANDT comes from 2 sources
mandt_mappings = [
mapping for mapping in parsed_lineage.mappings if mapping.target == "MANDT"
]
assert len(mandt_mappings) == 2
assert {mapping.data_source for mapping in mandt_mappings} == {ds_sbook, ds_sflight}

View File

@ -46,6 +46,15 @@ And, if you have password policies forcing users to reset the password, you can
ALTER USER openmetadata DISABLE PASSWORD LIFETIME;
```
Note that in order to get the metadata for **Calculation Views**, **Analytics Views** and **Attribute Views**, you need to have enough
permissions on the `_SYS_BIC` schema. You can grant the required permissions to the user by running the following SQL commands:
```SQL
GRANT SELECT ON SCHEMA _SYS_BIC TO <user_or_role>;
```
The same applies to the `_SYS_REPO` schema, required for lineage extraction.
### Profiler & Data Quality
Executing the profiler Workflow or data quality tests, will require the user to have `SELECT` permission on the tables/schemas where the profiler/tests will be executed. The user should also be allowed to view information in `tables` for all objects in the database. More information on the profiler workflow setup can be found [here](/connectors/ingestion/workflows/profiler) and data quality tests [here](/connectors/ingestion/workflows/data-quality).

View File

@ -29,8 +29,6 @@ Configure and schedule SAP Hana metadata and profiler workflows from the OpenMet
To deploy OpenMetadata, check the Deployment guides.
{%/inlineCallout%}
{% note %}
The connector is compatible with HANA or HANA express versions since HANA SPS 2.
{% /note %}
@ -45,6 +43,35 @@ To run the SAP Hana ingestion, you will need to install:
pip3 install "openmetadata-ingestion[sap-hana]"
```
### Metadata
To extract metadata the user used in the connection needs to have access to the `SYS` schema.
You can create a new user to run the ingestion with:
```SQL
CREATE USER openmetadata PASSWORD Password123;
```
And, if you have password policies forcing users to reset the password, you can disable that policy for this technical user with:
```SQL
ALTER USER openmetadata DISABLE PASSWORD LIFETIME;
```
Note that in order to get the metadata for **Calculation Views**, **Analytics Views** and **Attribute Views**, you need to have enough
permissions on the `_SYS_BIC` schema. You can grant the required permissions to the user by running the following SQL commands:
```SQL
GRANT SELECT ON SCHEMA _SYS_BIC TO <user_or_role>;
```
The same applies to the `_SYS_REPO` schema, required for lineage extraction.
### Profiler & Data Quality
Executing the profiler Workflow or data quality tests, will require the user to have `SELECT` permission on the tables/schemas where the profiler/tests will be executed. The user should also be allowed to view information in `tables` for all objects in the database. More information on the profiler workflow setup can be found [here](/connectors/ingestion/workflows/profiler) and data quality tests [here](/connectors/ingestion/workflows/data-quality).
## Metadata Ingestion
All connectors are defined as JSON Schemas.

View File

@ -46,6 +46,15 @@ And, if you have password policies forcing users to reset the password, you can
ALTER USER openmetadata DISABLE PASSWORD LIFETIME;
```
Note that in order to get the metadata for **Calculation Views**, **Analytics Views** and **Attribute Views**, you need to have enough
permissions on the `_SYS_BIC` schema. You can grant the required permissions to the user by running the following SQL commands:
```SQL
GRANT SELECT ON SCHEMA _SYS_BIC TO <user_or_role>;
```
The same applies to the `_SYS_REPO` schema, required for lineage extraction.
### Profiler & Data Quality
Executing the profiler Workflow or data quality tests, will require the user to have `SELECT` permission on the tables/schemas where the profiler/tests will be executed. The user should also be allowed to view information in `tables` for all objects in the database. More information on the profiler workflow setup can be found [here](/connectors/ingestion/workflows/profiler) and data quality tests [here](/connectors/ingestion/workflows/data-quality).

View File

@ -30,7 +30,6 @@ To deploy OpenMetadata, check the Deployment guides.
{%/inlineCallout%}
{% note %}
The connector is compatible with HANA or HANA express versions since HANA SPS 2.
{% /note %}
@ -45,6 +44,35 @@ To run the SAP Hana ingestion, you will need to install:
pip3 install "openmetadata-ingestion[sap-hana]"
```
### Metadata
To extract metadata the user used in the connection needs to have access to the `SYS` schema.
You can create a new user to run the ingestion with:
```SQL
CREATE USER openmetadata PASSWORD Password123;
```
And, if you have password policies forcing users to reset the password, you can disable that policy for this technical user with:
```SQL
ALTER USER openmetadata DISABLE PASSWORD LIFETIME;
```
Note that in order to get the metadata for **Calculation Views**, **Analytics Views** and **Attribute Views**, you need to have enough
permissions on the `_SYS_BIC` schema. You can grant the required permissions to the user by running the following SQL commands:
```SQL
GRANT SELECT ON SCHEMA _SYS_BIC TO <user_or_role>;
```
The same applies to the `_SYS_REPO` schema, required for lineage extraction.
### Profiler & Data Quality
Executing the profiler Workflow or data quality tests, will require the user to have `SELECT` permission on the tables/schemas where the profiler/tests will be executed. The user should also be allowed to view information in `tables` for all objects in the database. More information on the profiler workflow setup can be found [here](/connectors/ingestion/workflows/profiler) and data quality tests [here](/connectors/ingestion/workflows/data-quality).
## Metadata Ingestion
All connectors are defined as JSON Schemas.

View File

@ -64,6 +64,9 @@
"title": "Supports Profiler",
"$ref": "../connectionBasicType.json#/definitions/supportsProfiler"
},
"supportsLineageExtraction": {
"$ref": "../connectionBasicType.json#/definitions/supportsLineageExtraction"
},
"supportsQueryComment": {
"title": "Supports Query Comment",
"$ref": "../connectionBasicType.json#/definitions/supportsQueryComment"