mirror of
https://github.com/open-metadata/OpenMetadata.git
synced 2025-09-25 08:50:18 +00:00
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:
parent
c0a23c667b
commit
72afde0aca
@ -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>"SEATSOCC_ALL"/"SEATSMAX_ALL"</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
|
@ -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)
|
@ -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,
|
||||
)
|
@ -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');
|
||||
"""
|
@ -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>
|
135
ingestion/tests/unit/resources/saphana/cdata_attribute_view.xml
Normal file
135
ingestion/tests/unit/resources/saphana/cdata_attribute_view.xml
Normal 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>"SEATSMAX"+"SEATSMAX_B"+"SEATSMAX_F"</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>"SEATSOCC"+"SEATSOCC_B"+"SEATSOCC_F"</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>"PRICE"*"SEATSMAX"</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>"MANDT"+"CARRNAME"</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>
|
@ -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>"SEATSOCC_ALL"/"SEATSMAX_ALL"</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>
|
@ -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>
|
@ -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>
|
163
ingestion/tests/unit/topology/database/test_sap_hana.py
Normal file
163
ingestion/tests/unit/topology/database/test_sap_hana.py
Normal 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}
|
@ -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).
|
||||
|
@ -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.
|
||||
|
@ -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).
|
||||
|
@ -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.
|
||||
|
@ -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"
|
||||
|
Loading…
x
Reference in New Issue
Block a user