diff --git a/datahub-web-react/src/images/deltalakelogo.png b/datahub-web-react/src/images/deltalakelogo.png new file mode 100644 index 00000000000000..28f9a9d28a0cb6 Binary files /dev/null and b/datahub-web-react/src/images/deltalakelogo.png differ diff --git a/metadata-ingestion/docs/sources/delta-lake/delta-lake.md b/metadata-ingestion/docs/sources/delta-lake/delta-lake.md new file mode 100644 index 00000000000000..9c620d9840e340 --- /dev/null +++ b/metadata-ingestion/docs/sources/delta-lake/delta-lake.md @@ -0,0 +1,142 @@ +## Usage Guide + +If you are new to [Delta Lake](https://delta.io/) and want to test out a simple integration with Delta Lake and DataHub, you can follow this guide. + +### Delta Table on Local File System + +#### Step 1 +Create a delta table using the sample PySpark code below if you don't have a delta table you can point to. + +```python +import uuid +import random +from pyspark.sql import SparkSession +from delta.tables import DeltaTable + +def generate_data(): + return [(y, m, d, str(uuid.uuid4()), str(random.randrange(10000) % 26 + 65) * 3, random.random()*10000) + for d in range(1, 29) + for m in range(1, 13) + for y in range(2000, 2021)] + +jar_packages = ["org.apache.hadoop:hadoop-aws:3.2.3", "io.delta:delta-core_2.12:1.2.1"] +spark = SparkSession.builder \ + .appName("quickstart") \ + .master("local[*]") \ + .config("spark.jars.packages", ",".join(jar_packages)) \ + .config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") \ + .config("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.delta.catalog.DeltaCatalog") \ + .getOrCreate() + +table_path = "quickstart/my-table" +columns = ["year", "month", "day", "sale_id", "customer", "total_cost"] +spark.sparkContext.parallelize(generate_data()).toDF(columns).repartition(1).write.format("delta").save(table_path) + +df = spark.read.format("delta").load(table_path) +df.show() + +``` + +#### Step 2 +Create a datahub ingestion yaml file (delta.dhub.yaml) to ingest metadata from the delta table you just created. + +```yaml +source: + type: "delta-lake" + config: + base_path: "quickstart/my-table" + +sink: + type: "datahub-rest" + config: + server: "http://localhost:8080" +``` + +Note: Make sure you run the Spark code as well as recipe from same folder otherwise use absolute paths. + +#### Step 3 +Execute the ingestion recipe: +```shell +datahub ingest -c delta.dhub.yaml +``` + +### Delta Table on S3 + +#### Step 1 +Set up your AWS credentials by creating an AWS credentials config file; typically in '$HOME/.aws/credentials'. +``` +[my-creds] +aws_access_key_id: ###### +aws_secret_access_key: ###### +``` +Step 2: Create a Delta Table using the PySpark sample code below unless you already have Delta Tables on your S3. +```python +from pyspark.sql import SparkSession +from delta.tables import DeltaTable +from configparser import ConfigParser +import uuid +import random +def generate_data(): + return [(y, m, d, str(uuid.uuid4()), str(random.randrange(10000) % 26 + 65) * 3, random.random()*10000) + for d in range(1, 29) + for m in range(1, 13) + for y in range(2000, 2021)] + +jar_packages = ["org.apache.hadoop:hadoop-aws:3.2.3", "io.delta:delta-core_2.12:1.2.1"] +spark = SparkSession.builder \ + .appName("quickstart") \ + .master("local[*]") \ + .config("spark.jars.packages", ",".join(jar_packages)) \ + .config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") \ + .config("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.delta.catalog.DeltaCatalog") \ + .getOrCreate() + + +config_object = ConfigParser() +config_object.read("$HOME/.aws/credentials") +profile_info = config_object["my-creds"] +access_id = profile_info["aws_access_key_id"] +access_key = profile_info["aws_secret_access_key"] + +hadoop_conf = spark._jsc.hadoopConfiguration() +hadoop_conf.set("fs.s3a.impl", "org.apache.hadoop.fs.s3a.S3AFileSystem") +hadoop_conf.set("fs.s3a.aws.credentials.provider", "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider") +hadoop_conf.set("fs.s3a.access.key", access_id) +hadoop_conf.set("fs.s3a.secret.key", access_key) + +table_path = "s3a://my-bucket/my-folder/sales-table" +columns = ["year", "month", "day", "sale_id", "customer", "total_cost"] +spark.sparkContext.parallelize(generate_data()).toDF(columns).repartition(1).write.format("delta").save(table_path) +df = spark.read.format("delta").load(table_path) +df.show() + +``` + +#### Step 3 +Create a datahub ingestion yaml file (delta.s3.dhub.yaml) to ingest metadata from the delta table you just created. + +```yml +source: + type: "delta-lake" + config: + base_path: "s3://my-bucket/my-folder/sales-table" + s3: + aws_config: + aws_access_key_id: <> + aws_secret_access_key: <> + +sink: + type: "datahub-rest" + config: + server: "http://localhost:8080" +``` + +#### Step 4 +Execute the ingestion recipe: +```shell +datahub ingest -c delta.s3.dhub.yaml +``` + +### Note + +The above recipes are minimal recipes. Please refer to [Config Details](#config-details) section for the full configuration. diff --git a/metadata-ingestion/docs/sources/delta-lake/delta-lake_recipe.yml b/metadata-ingestion/docs/sources/delta-lake/delta-lake_recipe.yml new file mode 100644 index 00000000000000..fac8dc4f637415 --- /dev/null +++ b/metadata-ingestion/docs/sources/delta-lake/delta-lake_recipe.yml @@ -0,0 +1,9 @@ +source: + type: delta-lake + config: + env: "PROD" + platform_instance: "my-delta-lake" + base_path: "/path/to/data/folder" + +sink: + # sink configs \ No newline at end of file diff --git a/metadata-ingestion/setup.py b/metadata-ingestion/setup.py index 8c940e9576f212..053aed2acf8f75 100644 --- a/metadata-ingestion/setup.py +++ b/metadata-ingestion/setup.py @@ -164,6 +164,11 @@ def get_long_description(): "wcmatch", } +delta_lake = { + *s3_base, + "deltalake", +} + usage_common = { "sqlparse", } @@ -196,6 +201,7 @@ def get_long_description(): "datahub-business-glossary": set(), "data-lake": {*data_lake_base, *data_lake_profiling}, "s3": {*s3_base, *data_lake_profiling}, + "delta-lake": {*data_lake_profiling, *delta_lake}, "dbt": {"requests"} | aws_common, "druid": sql_common | {"pydruid>=0.6.2"}, # Starting with 7.14.0 python client is checking if it is connected to elasticsearch client. If its not it throws @@ -350,6 +356,7 @@ def get_long_description(): "redshift", "redshift-usage", "data-lake", + "delta-lake", "s3", "tableau", "trino", @@ -447,7 +454,7 @@ def get_long_description(): entry_points = { "console_scripts": ["datahub = datahub.entrypoints:main"], "datahub.ingestion.source.plugins": [ - "csv-enricher = datahub.ingestion.source.csv_enricher:CSVEnricherSource", + "csv-enricher = datahub.ingestion.source.csv_enricher:CSVEnricherSource", "file = datahub.ingestion.source.file:GenericFileSource", "sqlalchemy = datahub.ingestion.source.sql.sql_generic:SQLAlchemyGenericSource", "athena = datahub.ingestion.source.sql.athena:AthenaSource", @@ -457,6 +464,7 @@ def get_long_description(): "clickhouse = datahub.ingestion.source.sql.clickhouse:ClickHouseSource", "clickhouse-usage = datahub.ingestion.source.usage.clickhouse_usage:ClickHouseUsageSource", "data-lake = datahub.ingestion.source.data_lake:DataLakeSource", + "delta-lake = datahub.ingestion.source.delta_lake:DeltaLakeSource", "s3 = datahub.ingestion.source.s3:S3Source", "dbt = datahub.ingestion.source.dbt:DBTSource", "druid = datahub.ingestion.source.sql.druid:DruidSource", diff --git a/metadata-ingestion/src/datahub/ingestion/source/aws/s3_util.py b/metadata-ingestion/src/datahub/ingestion/source/aws/s3_util.py index a8e4587a917831..0acde18d86e0a8 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/aws/s3_util.py +++ b/metadata-ingestion/src/datahub/ingestion/source/aws/s3_util.py @@ -1,7 +1,103 @@ +import logging import os +from typing import Iterable, Optional + +from datahub.emitter.mce_builder import make_tag_urn +from datahub.ingestion.api.common import PipelineContext +from datahub.ingestion.source.aws.aws_common import AwsSourceConfig +from datahub.metadata.schema_classes import GlobalTagsClass, TagAssociationClass S3_PREFIXES = ["s3://", "s3n://", "s3a://"] +logging.getLogger("py4j").setLevel(logging.ERROR) +logger: logging.Logger = logging.getLogger(__name__) + + +def get_s3_tags( + bucket_name: str, + key_name: Optional[str], + dataset_urn: str, + aws_config: Optional[AwsSourceConfig], + ctx: PipelineContext, + use_s3_bucket_tags: Optional[bool] = False, + use_s3_object_tags: Optional[bool] = False, +) -> Optional[GlobalTagsClass]: + if aws_config is None: + raise ValueError("aws_config not set. Cannot browse s3") + new_tags = GlobalTagsClass(tags=[]) + tags_to_add = [] + if use_s3_bucket_tags: + s3 = aws_config.get_s3_resource() + bucket = s3.Bucket(bucket_name) + try: + tags_to_add.extend( + [ + make_tag_urn(f"""{tag["Key"]}:{tag["Value"]}""") + for tag in bucket.Tagging().tag_set + ] + ) + except s3.meta.client.exceptions.ClientError: + logger.warn(f"No tags found for bucket={bucket_name}") + + if use_s3_object_tags and key_name is not None: + s3_client = aws_config.get_s3_client() + object_tagging = s3_client.get_object_tagging(Bucket=bucket_name, Key=key_name) + tag_set = object_tagging["TagSet"] + if tag_set: + tags_to_add.extend( + [make_tag_urn(f"""{tag["Key"]}:{tag["Value"]}""") for tag in tag_set] + ) + else: + # Unlike bucket tags, if an object does not have tags, it will just return an empty array + # as opposed to an exception. + logger.warn(f"No tags found for bucket={bucket_name} key={key_name}") + if len(tags_to_add) == 0: + return None + if ctx.graph is not None: + logger.debug("Connected to DatahubApi, grabbing current tags to maintain.") + current_tags: Optional[GlobalTagsClass] = ctx.graph.get_aspect_v2( + entity_urn=dataset_urn, + aspect="globalTags", + aspect_type=GlobalTagsClass, + ) + if current_tags: + tags_to_add.extend([current_tag.tag for current_tag in current_tags.tags]) + else: + logger.warn("Could not connect to DatahubApi. No current tags to maintain") + # Remove duplicate tags + tags_to_add = list(set(tags_to_add)) + new_tags = GlobalTagsClass( + tags=[TagAssociationClass(tag_to_add) for tag_to_add in tags_to_add] + ) + return new_tags + + +def list_folders_path( + s3_uri: str, aws_config: Optional[AwsSourceConfig] +) -> Iterable[str]: + if not is_s3_uri(s3_uri): + raise ValueError("Not a s3 URI: " + s3_uri) + if aws_config is None: + raise ValueError("aws_config not set. Cannot browse s3") + bucket_name = get_bucket_name(s3_uri) + prefix = get_bucket_relative_path(s3_uri) + yield from list_folders(bucket_name, prefix, aws_config) + + +def list_folders( + bucket_name: str, prefix: str, aws_config: Optional[AwsSourceConfig] +) -> Iterable[str]: + if aws_config is None: + raise ValueError("aws_config not set. Cannot browse s3") + s3_client = aws_config.get_s3_client() + paginator = s3_client.get_paginator("list_objects_v2") + for page in paginator.paginate(Bucket=bucket_name, Prefix=prefix, Delimiter="/"): + for o in page.get("CommonPrefixes", []): + folder: str = str(o.get("Prefix")) + if folder.endswith("/"): + folder = folder[:-1] + yield f"{folder}" + def is_s3_uri(uri: str) -> bool: return any(uri.startswith(prefix) for prefix in S3_PREFIXES) @@ -24,7 +120,6 @@ def get_bucket_relative_path(s3_uri: str) -> str: def make_s3_urn(s3_uri: str, env: str) -> str: - s3_name = strip_s3_prefix(s3_uri) if s3_name.endswith("/"): diff --git a/metadata-ingestion/src/datahub/ingestion/source/data_lake/data_lake_utils.py b/metadata-ingestion/src/datahub/ingestion/source/data_lake/data_lake_utils.py new file mode 100644 index 00000000000000..02e516c8a2d052 --- /dev/null +++ b/metadata-ingestion/src/datahub/ingestion/source/data_lake/data_lake_utils.py @@ -0,0 +1,118 @@ +import logging +from typing import Iterable, List, Optional + +from datahub.emitter.mcp_builder import ( + FolderKey, + KeyType, + PlatformKey, + S3BucketKey, + add_dataset_to_container, + gen_containers, +) +from datahub.ingestion.api.workunit import MetadataWorkUnit + +# hide annoying debug errors from py4j +from datahub.ingestion.source.aws.s3_util import ( + get_bucket_name, + get_bucket_relative_path, + is_s3_uri, +) + +logging.getLogger("py4j").setLevel(logging.ERROR) +logger: logging.Logger = logging.getLogger(__name__) + + +class ContainerWUCreator: + processed_containers: List[str] + + def __init__(self, platform, platform_instance, env): + self.processed_containers = [] + self.platform = platform + self.instance = env if platform_instance is None else platform_instance + + def create_emit_containers( + self, + container_key: KeyType, + name: str, + sub_types: List[str], + parent_container_key: Optional[PlatformKey] = None, + domain_urn: Optional[str] = None, + ) -> Iterable[MetadataWorkUnit]: + if container_key.guid() not in self.processed_containers: + container_wus = gen_containers( + container_key=container_key, + name=name, + sub_types=sub_types, + parent_container_key=parent_container_key, + domain_urn=domain_urn, + ) + self.processed_containers.append(container_key.guid()) + logger.debug(f"Creating container with key: {container_key}") + for wu in container_wus: + yield wu + + def gen_folder_key(self, abs_path): + return FolderKey( + platform=self.platform, + instance=self.instance, + folder_abs_path=abs_path, + ) + + def gen_bucket_key(self, name): + return S3BucketKey( + platform="s3", + instance=self.instance, + bucket_name=name, + ) + + def create_container_hierarchy( + self, path: str, is_s3: bool, dataset_urn: str + ) -> Iterable[MetadataWorkUnit]: + logger.debug(f"Creating containers for {dataset_urn}") + base_full_path = path + parent_key = None + if is_s3_uri(path): + bucket_name = get_bucket_name(path) + bucket_key = self.gen_bucket_key(bucket_name) + yield from self.create_emit_containers( + container_key=bucket_key, + name=bucket_name, + sub_types=["S3 bucket"], + parent_container_key=None, + ) + parent_key = bucket_key + base_full_path = get_bucket_relative_path(path) + + parent_folder_path = ( + base_full_path[: base_full_path.rfind("/")] + if base_full_path.rfind("/") != -1 + else "" + ) + + # Dataset is in the root folder + if not parent_folder_path and parent_key is None: + logger.warning( + f"Failed to associate Dataset ({dataset_urn}) with container" + ) + return + + for folder in parent_folder_path.split("/"): + abs_path = folder + if parent_key: + prefix: str = "" + if isinstance(parent_key, S3BucketKey): + prefix = parent_key.bucket_name + elif isinstance(parent_key, FolderKey): + prefix = parent_key.folder_abs_path + abs_path = prefix + "/" + folder + folder_key = self.gen_folder_key(abs_path) + yield from self.create_emit_containers( + container_key=folder_key, + name=folder, + sub_types=["Folder"], + parent_container_key=parent_key, + ) + parent_key = folder_key + + assert parent_key is not None + yield from add_dataset_to_container(parent_key, dataset_urn) diff --git a/metadata-ingestion/src/datahub/ingestion/source/delta_lake/__init__.py b/metadata-ingestion/src/datahub/ingestion/source/delta_lake/__init__.py new file mode 100644 index 00000000000000..2d2a70707d354e --- /dev/null +++ b/metadata-ingestion/src/datahub/ingestion/source/delta_lake/__init__.py @@ -0,0 +1 @@ +from datahub.ingestion.source.delta_lake.source import DeltaLakeSource diff --git a/metadata-ingestion/src/datahub/ingestion/source/delta_lake/config.py b/metadata-ingestion/src/datahub/ingestion/source/delta_lake/config.py new file mode 100644 index 00000000000000..28abad5d918bf5 --- /dev/null +++ b/metadata-ingestion/src/datahub/ingestion/source/delta_lake/config.py @@ -0,0 +1,86 @@ +import logging +from typing import Any, Dict, Optional + +import pydantic +from pydantic import Field + +from datahub.configuration.common import AllowDenyPattern +from datahub.configuration.source_common import ( + ConfigModel, + EnvBasedSourceConfigBase, + PlatformSourceConfigBase, +) +from datahub.ingestion.source.aws.aws_common import AwsSourceConfig +from datahub.ingestion.source.aws.s3_util import is_s3_uri + +# hide annoying debug errors from py4j +logging.getLogger("py4j").setLevel(logging.ERROR) +logger: logging.Logger = logging.getLogger(__name__) + + +class S3(ConfigModel): + aws_config: AwsSourceConfig = Field(default=None, description="AWS configuration") + + # Whether or not to create in datahub from the s3 bucket + use_s3_bucket_tags: Optional[bool] = Field( + False, description="Whether or not to create tags in datahub from the s3 bucket" + ) + # Whether or not to create in datahub from the s3 object + use_s3_object_tags: Optional[bool] = Field( + False, + description="# Whether or not to create tags in datahub from the s3 object", + ) + + +class DeltaLakeSourceConfig(PlatformSourceConfigBase, EnvBasedSourceConfigBase): + class Config: + arbitrary_types_allowed = True + + base_path: str = Field( + description="Path to table (s3 or local file system). If path is not a delta table path " + "then all subfolders will be scanned to detect and ingest delta tables." + ) + relative_path: str = Field( + default=None, + description="If set, delta-tables will be searched at location " + "'/' and URNs will be created using " + "relative_path only.", + ) + platform: str = Field( + default="delta-lake", + description="The platform that this source connects to", + const=True, + ) + platform_instance: Optional[str] = Field( + default=None, + description="The instance of the platform that all assets produced by this recipe belong to", + ) + table_pattern: AllowDenyPattern = Field( + default=AllowDenyPattern.allow_all(), + description="regex patterns for tables to filter in ingestion.", + ) + + s3: Optional[S3] = Field() + + # to be set internally + _is_s3: bool + _complete_path: str + + def is_s3(self): + return self._is_s3 + + def get_complete_path(self): + return self._complete_path + + @pydantic.root_validator() + def validate_config(cls, values: Dict) -> Dict[str, Any]: + values["_is_s3"] = is_s3_uri(values["base_path"]) + if values["_is_s3"]: + if values["s3"] is None: + raise ValueError("s3 config must be set for s3 path") + values["_complete_path"] = values["base_path"] + if values["relative_path"] is not None: + values[ + "_complete_path" + ] = f"{values['_complete_path'].rstrip('/')}/{values['relative_path'].lstrip('/')}" + return values diff --git a/metadata-ingestion/src/datahub/ingestion/source/delta_lake/delta_lake_utils.py b/metadata-ingestion/src/datahub/ingestion/source/delta_lake/delta_lake_utils.py new file mode 100644 index 00000000000000..f1be3dc59c6929 --- /dev/null +++ b/metadata-ingestion/src/datahub/ingestion/source/delta_lake/delta_lake_utils.py @@ -0,0 +1,36 @@ +from typing import Optional + +from deltalake import DeltaTable, PyDeltaTableError + +from datahub.ingestion.source.delta_lake.config import DeltaLakeSourceConfig + + +def read_delta_table( + path: str, delta_lake_config: DeltaLakeSourceConfig +) -> Optional[DeltaTable]: + delta_table = None + try: + opts = {} + if delta_lake_config.is_s3(): + if delta_lake_config.s3 is None: + raise ValueError("aws_config not set. Cannot browse s3") + if delta_lake_config.s3.aws_config is None: + raise ValueError("aws_config not set. Cannot browse s3") + opts = { + "AWS_ACCESS_KEY_ID": delta_lake_config.s3.aws_config.aws_access_key_id, + "AWS_SECRET_ACCESS_KEY": delta_lake_config.s3.aws_config.aws_secret_access_key, + } + delta_table = DeltaTable(path, storage_options=opts) + + except PyDeltaTableError as e: + if "Not a Delta table" not in str(e): + import pdb + + pdb.set_trace() + raise e + + return delta_table + + +def get_file_count(delta_table: DeltaTable) -> int: + return len(delta_table.files()) diff --git a/metadata-ingestion/src/datahub/ingestion/source/delta_lake/report.py b/metadata-ingestion/src/datahub/ingestion/source/delta_lake/report.py new file mode 100644 index 00000000000000..e8690a24f8b85e --- /dev/null +++ b/metadata-ingestion/src/datahub/ingestion/source/delta_lake/report.py @@ -0,0 +1,17 @@ +import dataclasses +from dataclasses import field as dataclass_field +from typing import List + +from datahub.ingestion.api.source import SourceReport + + +@dataclasses.dataclass +class DeltaLakeSourceReport(SourceReport): + files_scanned = 0 + filtered: List[str] = dataclass_field(default_factory=list) + + def report_file_scanned(self) -> None: + self.files_scanned += 1 + + def report_file_dropped(self, file: str) -> None: + self.filtered.append(file) diff --git a/metadata-ingestion/src/datahub/ingestion/source/delta_lake/source.py b/metadata-ingestion/src/datahub/ingestion/source/delta_lake/source.py new file mode 100644 index 00000000000000..c292c16764ec3a --- /dev/null +++ b/metadata-ingestion/src/datahub/ingestion/source/delta_lake/source.py @@ -0,0 +1,273 @@ +import logging +import os +from typing import Callable, Iterable, List + +from deltalake import DeltaTable + +from datahub.emitter.mce_builder import ( + make_data_platform_urn, + make_dataset_urn_with_platform_instance, +) +from datahub.ingestion.api.common import PipelineContext, WorkUnit +from datahub.ingestion.api.decorators import ( + SourceCapability, + SupportStatus, + capability, + config_class, + platform_name, + support_status, +) +from datahub.ingestion.api.source import Source, SourceReport +from datahub.ingestion.api.workunit import MetadataWorkUnit +from datahub.ingestion.source.aws.s3_util import ( + get_bucket_name, + get_key_prefix, + get_s3_tags, + list_folders_path, + strip_s3_prefix, +) +from datahub.ingestion.source.data_lake.data_lake_utils import ContainerWUCreator +from datahub.ingestion.source.delta_lake.config import DeltaLakeSourceConfig +from datahub.ingestion.source.delta_lake.delta_lake_utils import ( + get_file_count, + read_delta_table, +) +from datahub.ingestion.source.delta_lake.report import DeltaLakeSourceReport +from datahub.ingestion.source.schema_inference.csv_tsv import tableschema_type_map +from datahub.metadata.com.linkedin.pegasus2avro.metadata.snapshot import DatasetSnapshot +from datahub.metadata.com.linkedin.pegasus2avro.mxe import MetadataChangeEvent +from datahub.metadata.com.linkedin.pegasus2avro.schema import ( + SchemaField, + SchemaFieldDataType, + SchemaMetadata, +) +from datahub.metadata.schema_classes import ( + DatasetPropertiesClass, + NullTypeClass, + OtherSchemaClass, +) +from datahub.telemetry import telemetry + +logging.getLogger("py4j").setLevel(logging.ERROR) +logger: logging.Logger = logging.getLogger(__name__) + +config_options_to_report = [ + "platform", +] + + +@platform_name("Delta Lake", id="delta-lake") +@config_class(DeltaLakeSourceConfig) +@support_status(SupportStatus.INCUBATING) +@capability(SourceCapability.TAGS, "Can extract S3 object/bucket tags if enabled") +class DeltaLakeSource(Source): + """ + This plugin extracts: + - Column types and schema associated with each delta table + - Custom properties: number_of_files, partition_columns, table_creation_time, location, version etc. + + :::caution + + If you are ingesting datasets from AWS S3, we recommend running the ingestion on a server in the same region to avoid high egress costs. + + ::: + + """ + + source_config: DeltaLakeSourceConfig + report: DeltaLakeSourceReport + profiling_times_taken: List[float] + container_WU_creator: ContainerWUCreator + + def __init__(self, config: DeltaLakeSourceConfig, ctx: PipelineContext): + super().__init__(ctx) + self.source_config = config + self.report = DeltaLakeSourceReport() + # self.profiling_times_taken = [] + config_report = { + config_option: config.dict().get(config_option) + for config_option in config_options_to_report + } + config_report = config_report + + telemetry.telemetry_instance.ping( + "delta_lake_config", + config_report, + ) + + @classmethod + def create(cls, config_dict: dict, ctx: PipelineContext) -> "Source": + config = DeltaLakeSourceConfig.parse_obj(config_dict) + return cls(config, ctx) + + def get_fields(self, delta_table: DeltaTable) -> List[SchemaField]: + + fields: List[SchemaField] = [] + + for raw_field in delta_table.schema().fields: + field = SchemaField( + fieldPath=raw_field.name, + type=SchemaFieldDataType( + tableschema_type_map.get(raw_field.type.type, NullTypeClass)() + ), + nativeDataType=raw_field.type.type, + recursive=False, + nullable=raw_field.nullable, + description=str(raw_field.metadata), + isPartitioningKey=True + if raw_field.name in delta_table.metadata().partition_columns + else False, + ) + fields.append(field) + fields = sorted(fields, key=lambda f: f.fieldPath) + + return fields + + def ingest_table( + self, delta_table: DeltaTable, path: str + ) -> Iterable[MetadataWorkUnit]: + table_name = ( + delta_table.metadata().name + if delta_table.metadata().name + else path.split("/")[-1] + ) + if not self.source_config.table_pattern.allowed(table_name): + logger.debug( + f"Skipping table ({table_name}) present at location {path} as table pattern does not match" + ) + + logger.debug(f"Ingesting table {table_name} from location {path}") + if self.source_config.relative_path is None: + browse_path: str = ( + strip_s3_prefix(path) if self.source_config.is_s3() else path.strip("/") + ) + else: + browse_path = path.split(self.source_config.base_path)[1].strip("/") + + data_platform_urn = make_data_platform_urn(self.source_config.platform) + logger.info(f"Creating dataset urn with name: {browse_path}") + dataset_urn = make_dataset_urn_with_platform_instance( + self.source_config.platform, + browse_path, + self.source_config.platform_instance, + self.source_config.env, + ) + dataset_snapshot = DatasetSnapshot( + urn=dataset_urn, + aspects=[], + ) + + customProperties = { + "number_of_files": str(get_file_count(delta_table)), + "partition_columns": str(delta_table.metadata().partition_columns), + "table_creation_time": str(delta_table.metadata().created_time), + "id": str(delta_table.metadata().id), + "version": str(delta_table.version()), + "location": self.source_config.get_complete_path(), + } + customProperties.update(delta_table.history()[-1]) + customProperties["version_creation_time"] = customProperties["timestamp"] + del customProperties["timestamp"] + for key in customProperties.keys(): + customProperties[key] = str(customProperties[key]) + + dataset_properties = DatasetPropertiesClass( + description=delta_table.metadata().description, + name=table_name, + customProperties=customProperties, + ) + dataset_snapshot.aspects.append(dataset_properties) + + fields = self.get_fields(delta_table) + schema_metadata = SchemaMetadata( + schemaName=table_name, + platform=data_platform_urn, + version=delta_table.version(), + hash="", + fields=fields, + platformSchema=OtherSchemaClass(rawSchema=""), + ) + dataset_snapshot.aspects.append(schema_metadata) + + if ( + self.source_config.is_s3() + and self.source_config.s3 + and ( + self.source_config.s3.use_s3_bucket_tags + or self.source_config.s3.use_s3_object_tags + ) + ): + bucket = get_bucket_name(path) + key_prefix = get_key_prefix(path) + s3_tags = get_s3_tags( + bucket, + key_prefix, + dataset_urn, + self.source_config.s3.aws_config, + self.ctx, + self.source_config.s3.use_s3_bucket_tags, + self.source_config.s3.use_s3_object_tags, + ) + if s3_tags is not None: + dataset_snapshot.aspects.append(s3_tags) + mce = MetadataChangeEvent(proposedSnapshot=dataset_snapshot) + wu = MetadataWorkUnit(id=delta_table.metadata().id, mce=mce) + self.report.report_workunit(wu) + yield wu + + container_wus = self.container_WU_creator.create_container_hierarchy( + browse_path, self.source_config.is_s3(), dataset_urn + ) + for wu in container_wus: + self.report.report_workunit(wu) + yield wu + + def process_folder( + self, path: str, get_folders: Callable[[str], Iterable[str]] + ) -> Iterable[MetadataWorkUnit]: + logger.debug(f"Processing folder: {path}") + delta_table = read_delta_table(path, self.source_config) + if delta_table: + logger.debug(f"Delta table found at: {path}") + for wu in self.ingest_table(delta_table, path): + yield wu + else: + for folder in get_folders(path): + yield from self.process_folder(path + "/" + folder, get_folders) + + def s3_get_folders(self, path: str) -> Iterable[str]: + if self.source_config.s3 is not None: + yield from list_folders_path(path, self.source_config.s3.aws_config) + + def local_get_folders(self, path: str) -> Iterable[str]: + if not os.path.isdir(path): + raise Exception( + f"{path} does not exist. Please check base_path configuration." + ) + for _, folders, _ in os.walk(path): + for folder in folders: + yield folder + break + return + + def get_workunits(self) -> Iterable[WorkUnit]: + self.container_WU_creator = ContainerWUCreator( + self.source_config.platform, + self.source_config.platform_instance, + self.source_config.env, + ) + get_folders = ( + self.s3_get_folders + if self.source_config.is_s3() + else self.local_get_folders + ) + for wu in self.process_folder( + self.source_config.get_complete_path(), get_folders + ): + yield wu + + def get_report(self) -> SourceReport: + return self.report + + def close(self): + pass diff --git a/metadata-ingestion/src/datahub/ingestion/source/s3/source.py b/metadata-ingestion/src/datahub/ingestion/source/s3/source.py index dcdce5f4ff59da..4e1dda628f1dad 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/s3/source.py +++ b/metadata-ingestion/src/datahub/ingestion/source/s3/source.py @@ -38,17 +38,8 @@ from datahub.emitter.mce_builder import ( make_data_platform_urn, make_dataset_urn_with_platform_instance, - make_tag_urn, ) from datahub.emitter.mcp import MetadataChangeProposalWrapper -from datahub.emitter.mcp_builder import ( - FolderKey, - KeyType, - PlatformKey, - S3BucketKey, - add_dataset_to_container, - gen_containers, -) from datahub.ingestion.api.common import PipelineContext from datahub.ingestion.api.decorators import ( SourceCapability, @@ -60,14 +51,16 @@ ) from datahub.ingestion.api.source import Source, SourceReport from datahub.ingestion.api.workunit import MetadataWorkUnit -from datahub.ingestion.source.aws.path_spec import PathSpec from datahub.ingestion.source.aws.s3_util import ( get_bucket_name, get_bucket_relative_path, get_key_prefix, + get_s3_tags, + list_folders, strip_s3_prefix, ) -from datahub.ingestion.source.s3.config import DataLakeSourceConfig +from datahub.ingestion.source.data_lake.data_lake_utils import ContainerWUCreator +from datahub.ingestion.source.s3.config import DataLakeSourceConfig, PathSpec from datahub.ingestion.source.s3.profiling import _SingleTableProfiler from datahub.ingestion.source.s3.report import DataLakeSourceReport from datahub.ingestion.source.schema_inference import avro, csv_tsv, json, parquet @@ -88,10 +81,8 @@ from datahub.metadata.schema_classes import ( ChangeTypeClass, DatasetPropertiesClass, - GlobalTagsClass, MapTypeClass, OtherSchemaClass, - TagAssociationClass, ) from datahub.telemetry import stats, telemetry from datahub.utilities.perf_timer import PerfTimer @@ -230,7 +221,7 @@ class S3Source(Source): source_config: DataLakeSourceConfig report: DataLakeSourceReport profiling_times_taken: List[float] - processed_containers: List[str] + container_WU_creator: ContainerWUCreator def __init__(self, config: DataLakeSourceConfig, ctx: PipelineContext): super().__init__(ctx) @@ -383,80 +374,6 @@ def read_file_spark(self, file: str, ext: str) -> Optional[DataFrame]: # see https://mungingdata.com/pyspark/avoid-dots-periods-column-names/ return df.toDF(*(c.replace(".", "_") for c in df.columns)) - def create_emit_containers( - self, - container_key: KeyType, - name: str, - sub_types: List[str], - parent_container_key: Optional[PlatformKey] = None, - domain_urn: Optional[str] = None, - ) -> Iterable[MetadataWorkUnit]: - if container_key.guid() not in self.processed_containers: - container_wus = gen_containers( - container_key=container_key, - name=name, - sub_types=sub_types, - parent_container_key=parent_container_key, - domain_urn=domain_urn, - ) - self.processed_containers.append(container_key.guid()) - logger.debug(f"Creating container with key: {container_key}") - for wu in container_wus: - self.report.report_workunit(wu) - yield wu - - def create_container_hierarchy( - self, table_data: TableData, dataset_urn: str - ) -> Iterable[MetadataWorkUnit]: - logger.debug(f"Creating containers for {dataset_urn}") - base_full_path = table_data.table_path - parent_key = None - if table_data.is_s3: - bucket_name = get_bucket_name(table_data.table_path) - bucket_key = self.gen_bucket_key(bucket_name) - yield from self.create_emit_containers( - container_key=bucket_key, - name=bucket_name, - sub_types=["S3 bucket"], - parent_container_key=None, - ) - parent_key = bucket_key - base_full_path = get_bucket_relative_path(table_data.table_path) - - parent_folder_path = ( - base_full_path[: base_full_path.rfind("/")] - if base_full_path.rfind("/") != -1 - else "" - ) - - # Dataset is in the root folder - if not parent_folder_path and parent_key is None: - logger.warning( - f"Failed to associate Dataset ({dataset_urn}) with container" - ) - return - - for folder in parent_folder_path.split("/"): - abs_path = folder - if parent_key: - prefix: str = "" - if isinstance(parent_key, S3BucketKey): - prefix = parent_key.bucket_name - elif isinstance(parent_key, FolderKey): - prefix = parent_key.folder_abs_path - abs_path = prefix + "/" + folder - folder_key = self.gen_folder_key(abs_path) - yield from self.create_emit_containers( - container_key=folder_key, - name=folder, - sub_types=["Folder"], - parent_container_key=parent_key, - ) - parent_key = folder_key - - assert parent_key is not None - yield from add_dataset_to_container(parent_key, dataset_urn) - def get_fields(self, table_data: TableData, path_spec: PathSpec) -> List: if table_data.is_s3: if self.source_config.aws_config is None: @@ -650,7 +567,15 @@ def ingest_table( if table_data.full_path == table_data.table_path else None ) - s3_tags = self.get_s3_tags(bucket, key_prefix, dataset_urn) + s3_tags = get_s3_tags( + bucket, + key_prefix, + dataset_urn, + self.source_config.aws_config, + self.ctx, + self.source_config.use_s3_bucket_tags, + self.source_config.use_s3_object_tags, + ) if s3_tags is not None: dataset_snapshot.aspects.append(s3_tags) @@ -659,88 +584,16 @@ def ingest_table( self.report.report_workunit(wu) yield wu - yield from self.create_container_hierarchy(table_data, dataset_urn) + container_wus = self.container_WU_creator.create_container_hierarchy( + table_data.table_path, table_data.is_s3, dataset_urn + ) + for wu in container_wus: + self.report.report_workunit(wu) + yield wu if self.source_config.profiling.enabled: yield from self.get_table_profile(table_data, dataset_urn) - def gen_bucket_key(self, name): - return S3BucketKey( - platform="s3", - instance=self.source_config.env - if self.source_config.platform_instance is None - else self.source_config.platform_instance, - bucket_name=name, - ) - - def get_s3_tags( - self, bucket_name: str, key_name: Optional[str], dataset_urn: str - ) -> Optional[GlobalTagsClass]: - if self.source_config.aws_config is None: - raise ValueError("aws_config not set. Cannot browse s3") - new_tags = GlobalTagsClass(tags=[]) - tags_to_add = [] - if self.source_config.use_s3_bucket_tags: - s3 = self.source_config.aws_config.get_s3_resource() - bucket = s3.Bucket(bucket_name) - try: - tags_to_add.extend( - [ - make_tag_urn(f"""{tag["Key"]}:{tag["Value"]}""") - for tag in bucket.Tagging().tag_set - ] - ) - except s3.meta.client.exceptions.ClientError: - logger.warn(f"No tags found for bucket={bucket_name}") - - if self.source_config.use_s3_object_tags and key_name is not None: - s3_client = self.source_config.aws_config.get_s3_client() - object_tagging = s3_client.get_object_tagging( - Bucket=bucket_name, Key=key_name - ) - tag_set = object_tagging["TagSet"] - if tag_set: - tags_to_add.extend( - [ - make_tag_urn(f"""{tag["Key"]}:{tag["Value"]}""") - for tag in tag_set - ] - ) - else: - # Unlike bucket tags, if an object does not have tags, it will just return an empty array - # as opposed to an exception. - logger.warn(f"No tags found for bucket={bucket_name} key={key_name}") - if len(tags_to_add) == 0: - return None - if self.ctx.graph is not None: - logger.debug("Connected to DatahubApi, grabbing current tags to maintain.") - current_tags: Optional[GlobalTagsClass] = self.ctx.graph.get_aspect_v2( - entity_urn=dataset_urn, - aspect="globalTags", - aspect_type=GlobalTagsClass, - ) - if current_tags: - tags_to_add.extend( - [current_tag.tag for current_tag in current_tags.tags] - ) - else: - logger.warn("Could not connect to DatahubApi. No current tags to maintain") - # Remove duplicate tags - tags_to_add = list(set(tags_to_add)) - new_tags = GlobalTagsClass( - tags=[TagAssociationClass(tag_to_add) for tag_to_add in tags_to_add] - ) - return new_tags - - def gen_folder_key(self, abs_path): - return FolderKey( - platform=self.source_config.platform, - instance=self.source_config.env - if self.source_config.platform_instance is None - else self.source_config.platform_instance, - folder_abs_path=abs_path, - ) - def get_prefix(self, relative_path: str) -> str: index = re.search(r"[\*|\{]", relative_path) if index: @@ -779,25 +632,14 @@ def resolve_templated_folders(self, bucket_name: str, prefix: str) -> Iterable[s yield prefix return - folders: Iterable[str] = self.list_folders(bucket_name, folder_split[0]) + folders: Iterable[str] = list_folders( + bucket_name, folder_split[0], self.source_config.aws_config + ) for folder in folders: yield from self.resolve_templated_folders( bucket_name, f"{folder}{folder_split[1]}" ) - def list_folders(self, bucket_name: str, prefix: str) -> Iterable[str]: - assert self.source_config.aws_config - s3_client = self.source_config.aws_config.get_s3_client() - paginator = s3_client.get_paginator("list_objects_v2") - for page in paginator.paginate( - Bucket=bucket_name, Prefix=prefix, Delimiter="/" - ): - for o in page.get("CommonPrefixes", []): - folder: str = str(o.get("Prefix")) - if folder.endswith("/"): - folder = folder[:-1] - yield f"{folder}" - def s3_browser(self, path_spec: PathSpec) -> Iterable[Tuple[str, datetime, int]]: if self.source_config.aws_config is None: raise ValueError("aws_config not set. Cannot browse s3") @@ -833,7 +675,9 @@ def s3_browser(self, path_spec: PathSpec) -> Iterable[Tuple[str, datetime, int]] for folder in self.resolve_templated_folders( bucket_name, get_bucket_relative_path(include[:table_index]) ): - for f in self.list_folders(bucket_name, f"{folder}"): + for f in list_folders( + bucket_name, f"{folder}", self.source_config.aws_config + ): logger.info(f"Processing folder: {f}") for obj in ( @@ -871,7 +715,11 @@ def local_browser(self, path_spec: PathSpec) -> Iterable[Tuple[str, datetime, in ), os.path.getsize(full_path) def get_workunits(self) -> Iterable[MetadataWorkUnit]: - self.processed_containers = [] + self.container_WU_creator = ContainerWUCreator( + self.source_config.platform, + self.source_config.platform_instance, + self.source_config.env, + ) with PerfTimer() as timer: assert self.source_config.path_specs for path_spec in self.source_config.path_specs: diff --git a/metadata-ingestion/tests/integration/delta_lake/golden_files/local/golden_mces_allow_table.json b/metadata-ingestion/tests/integration/delta_lake/golden_files/local/golden_mces_allow_table.json new file mode 100644 index 00000000000000..bdefaaac03e9ae --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/golden_files/local/golden_mces_allow_table.json @@ -0,0 +1,1114 @@ +[ +{ + "auditHeader": null, + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:delta-lake,my-platform.tests/integration/delta_lake/test_data/delta_tables/my_table_no_name,UAT)", + "aspects": [ + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": { + "number_of_files": "5", + "partition_columns": "['foo', 'bar']", + "table_creation_time": "1655831648843", + "id": "de711767-c7b9-4c33-99d7-510978dc1fa5", + "version": "4", + "location": "tests/integration/delta_lake/test_data/delta_tables", + "operation": "WRITE", + "operationParameters": "{}", + "readVersion": "3", + "isolationLevel": "Serializable", + "isBlindAppend": "True", + "operationMetrics": "{}", + "engineInfo": "local Delta-Standalone/0.4.0", + "version_creation_time": "1655831649788" + }, + "externalUrl": null, + "name": "my_table_no_name", + "qualifiedName": null, + "description": null, + "uri": null, + "tags": [] + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "my_table_no_name", + "platform": "urn:li:dataPlatform:delta-lake", + "version": 4, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown", + "impersonator": null + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown", + "impersonator": null + }, + "deleted": null, + "dataset": null, + "cluster": null, + "hash": "", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.OtherSchema": { + "rawSchema": "" + } + }, + "fields": [ + { + "fieldPath": "bar", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "integer", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": true, + "jsonProps": null + }, + { + "fieldPath": "foo", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "integer", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": true, + "jsonProps": null + }, + { + "fieldPath": "zip", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "string", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": false, + "jsonProps": null + } + ], + "primaryKeys": null, + "foreignKeysSpecs": null, + "foreignKeys": null + } + } + ] + } + }, + "proposedDelta": null, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:189046201d696e7810132cfa64dad337", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "containerProperties", + "aspect": { + "value": "{\"customProperties\": {\"platform\": \"delta-lake\", \"instance\": \"my-platform\", \"folder_abs_path\": \"tests\"}, \"name\": \"tests\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:189046201d696e7810132cfa64dad337", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "value": "{\"platform\": \"urn:li:dataPlatform:delta-lake\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:189046201d696e7810132cfa64dad337", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "value": "{\"typeNames\": [\"Folder\"]}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:acf0f3806f475a7397ee745329ef2967", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "containerProperties", + "aspect": { + "value": "{\"customProperties\": {\"platform\": \"delta-lake\", \"instance\": \"my-platform\", \"folder_abs_path\": \"tests/integration\"}, \"name\": \"integration\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:acf0f3806f475a7397ee745329ef2967", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "value": "{\"platform\": \"urn:li:dataPlatform:delta-lake\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:acf0f3806f475a7397ee745329ef2967", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "value": "{\"typeNames\": [\"Folder\"]}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:acf0f3806f475a7397ee745329ef2967", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "value": "{\"container\": \"urn:li:container:189046201d696e7810132cfa64dad337\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:1876d057d0ee364677b85427342e2c82", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "containerProperties", + "aspect": { + "value": "{\"customProperties\": {\"platform\": \"delta-lake\", \"instance\": \"my-platform\", \"folder_abs_path\": \"tests/integration/delta_lake\"}, \"name\": \"delta_lake\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:1876d057d0ee364677b85427342e2c82", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "value": "{\"platform\": \"urn:li:dataPlatform:delta-lake\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:1876d057d0ee364677b85427342e2c82", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "value": "{\"typeNames\": [\"Folder\"]}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:1876d057d0ee364677b85427342e2c82", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "value": "{\"container\": \"urn:li:container:acf0f3806f475a7397ee745329ef2967\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:7888b6dab77b7e77709699c9a1b81aa4", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "containerProperties", + "aspect": { + "value": "{\"customProperties\": {\"platform\": \"delta-lake\", \"instance\": \"my-platform\", \"folder_abs_path\": \"tests/integration/delta_lake/test_data\"}, \"name\": \"test_data\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:7888b6dab77b7e77709699c9a1b81aa4", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "value": "{\"platform\": \"urn:li:dataPlatform:delta-lake\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:7888b6dab77b7e77709699c9a1b81aa4", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "value": "{\"typeNames\": [\"Folder\"]}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:7888b6dab77b7e77709699c9a1b81aa4", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "value": "{\"container\": \"urn:li:container:1876d057d0ee364677b85427342e2c82\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:a282913be26fceff334523c2be119df1", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "containerProperties", + "aspect": { + "value": "{\"customProperties\": {\"platform\": \"delta-lake\", \"instance\": \"my-platform\", \"folder_abs_path\": \"tests/integration/delta_lake/test_data/delta_tables\"}, \"name\": \"delta_tables\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:a282913be26fceff334523c2be119df1", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "value": "{\"platform\": \"urn:li:dataPlatform:delta-lake\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:a282913be26fceff334523c2be119df1", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "value": "{\"typeNames\": [\"Folder\"]}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:a282913be26fceff334523c2be119df1", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "value": "{\"container\": \"urn:li:container:7888b6dab77b7e77709699c9a1b81aa4\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:delta-lake,my-platform.tests/integration/delta_lake/test_data/delta_tables/my_table_no_name,UAT)", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "value": "{\"container\": \"urn:li:container:a282913be26fceff334523c2be119df1\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:delta-lake,my-platform.tests/integration/delta_lake/test_data/delta_tables/sales,UAT)", + "aspects": [ + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": { + "number_of_files": "3", + "partition_columns": "[]", + "table_creation_time": "1655664813952", + "id": "eca9d2a0-4ce6-4ace-a732-75fda0157fb8", + "version": "0", + "location": "tests/integration/delta_lake/test_data/delta_tables", + "operation": "CONVERT", + "operationParameters": "{}", + "isolationLevel": "Serializable", + "isBlindAppend": "True", + "operationMetrics": "{}", + "engineInfo": "local Delta-Standalone/0.4.0", + "version_creation_time": "1655664815399" + }, + "externalUrl": null, + "name": "my_table", + "qualifiedName": null, + "description": "my table description", + "uri": null, + "tags": [] + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "my_table", + "platform": "urn:li:dataPlatform:delta-lake", + "version": 0, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown", + "impersonator": null + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown", + "impersonator": null + }, + "deleted": null, + "dataset": null, + "cluster": null, + "hash": "", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.OtherSchema": { + "rawSchema": "" + } + }, + "fields": [ + { + "fieldPath": "customer", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "string", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": false, + "jsonProps": null + }, + { + "fieldPath": "day", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "integer", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": false, + "jsonProps": null + }, + { + "fieldPath": "month", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "integer", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": false, + "jsonProps": null + }, + { + "fieldPath": "sale_id", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "string", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": false, + "jsonProps": null + }, + { + "fieldPath": "total_cost", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NullType": {} + } + }, + "nativeDataType": "float", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": false, + "jsonProps": null + }, + { + "fieldPath": "year", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "integer", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": false, + "jsonProps": null + } + ], + "primaryKeys": null, + "foreignKeysSpecs": null, + "foreignKeys": null + } + } + ] + } + }, + "proposedDelta": null, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:delta-lake,my-platform.tests/integration/delta_lake/test_data/delta_tables/sales,UAT)", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "value": "{\"container\": \"urn:li:container:a282913be26fceff334523c2be119df1\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:delta-lake,my-platform.tests/integration/delta_lake/test_data/delta_tables/my_table_basic,UAT)", + "aspects": [ + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": { + "number_of_files": "5", + "partition_columns": "['foo', 'bar']", + "table_creation_time": "1655831476360", + "id": "628d06df-ecb0-4314-a97e-75d8872db7c3", + "version": "4", + "location": "tests/integration/delta_lake/test_data/delta_tables", + "operation": "WRITE", + "operationParameters": "{}", + "readVersion": "3", + "isolationLevel": "Serializable", + "isBlindAppend": "True", + "operationMetrics": "{}", + "engineInfo": "local Delta-Standalone/0.4.0", + "version_creation_time": "1655831477768" + }, + "externalUrl": null, + "name": "test-table-basic", + "qualifiedName": null, + "description": "test delta table basic with table name", + "uri": null, + "tags": [] + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "test-table-basic", + "platform": "urn:li:dataPlatform:delta-lake", + "version": 4, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown", + "impersonator": null + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown", + "impersonator": null + }, + "deleted": null, + "dataset": null, + "cluster": null, + "hash": "", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.OtherSchema": { + "rawSchema": "" + } + }, + "fields": [ + { + "fieldPath": "bar", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "integer", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": true, + "jsonProps": null + }, + { + "fieldPath": "foo", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "integer", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": true, + "jsonProps": null + }, + { + "fieldPath": "zip", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "string", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": false, + "jsonProps": null + } + ], + "primaryKeys": null, + "foreignKeysSpecs": null, + "foreignKeys": null + } + } + ] + } + }, + "proposedDelta": null, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:delta-lake,my-platform.tests/integration/delta_lake/test_data/delta_tables/my_table_basic,UAT)", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "value": "{\"container\": \"urn:li:container:a282913be26fceff334523c2be119df1\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:delta-lake,my-platform.tests/integration/delta_lake/test_data/delta_tables/level1/my_table_inner,UAT)", + "aspects": [ + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": { + "number_of_files": "5", + "partition_columns": "['foo', 'bar']", + "table_creation_time": "1655831864836", + "id": "3775a0fd-4f58-4dea-b71a-e3fedb10f5b4", + "version": "4", + "location": "tests/integration/delta_lake/test_data/delta_tables", + "operation": "WRITE", + "operationParameters": "{}", + "readVersion": "3", + "isolationLevel": "Serializable", + "isBlindAppend": "True", + "operationMetrics": "{}", + "engineInfo": "local Delta-Standalone/0.4.0", + "version_creation_time": "1655831866541" + }, + "externalUrl": null, + "name": "test-table-inner", + "qualifiedName": null, + "description": "test delta table basic with table name at inner location", + "uri": null, + "tags": [] + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "test-table-inner", + "platform": "urn:li:dataPlatform:delta-lake", + "version": 4, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown", + "impersonator": null + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown", + "impersonator": null + }, + "deleted": null, + "dataset": null, + "cluster": null, + "hash": "", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.OtherSchema": { + "rawSchema": "" + } + }, + "fields": [ + { + "fieldPath": "bar", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "integer", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": true, + "jsonProps": null + }, + { + "fieldPath": "foo", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "integer", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": true, + "jsonProps": null + }, + { + "fieldPath": "zip", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "string", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": false, + "jsonProps": null + } + ], + "primaryKeys": null, + "foreignKeysSpecs": null, + "foreignKeys": null + } + } + ] + } + }, + "proposedDelta": null, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:3df8f6b0f3a70d42cf70612a2fe5e5ef", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "containerProperties", + "aspect": { + "value": "{\"customProperties\": {\"platform\": \"delta-lake\", \"instance\": \"my-platform\", \"folder_abs_path\": \"tests/integration/delta_lake/test_data/delta_tables/level1\"}, \"name\": \"level1\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:3df8f6b0f3a70d42cf70612a2fe5e5ef", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "value": "{\"platform\": \"urn:li:dataPlatform:delta-lake\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:3df8f6b0f3a70d42cf70612a2fe5e5ef", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "value": "{\"typeNames\": [\"Folder\"]}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:3df8f6b0f3a70d42cf70612a2fe5e5ef", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "value": "{\"container\": \"urn:li:container:a282913be26fceff334523c2be119df1\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:delta-lake,my-platform.tests/integration/delta_lake/test_data/delta_tables/level1/my_table_inner,UAT)", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "value": "{\"container\": \"urn:li:container:3df8f6b0f3a70d42cf70612a2fe5e5ef\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "allow_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +} +] \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/delta_lake/golden_files/local/golden_mces_inner_table.json b/metadata-ingestion/tests/integration/delta_lake/golden_files/local/golden_mces_inner_table.json new file mode 100644 index 00000000000000..385e5efe01d4f8 --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/golden_files/local/golden_mces_inner_table.json @@ -0,0 +1,1114 @@ +[ +{ + "auditHeader": null, + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:delta-lake,tests/integration/delta_lake/test_data/delta_tables/my_table_no_name,UAT)", + "aspects": [ + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": { + "number_of_files": "5", + "partition_columns": "['foo', 'bar']", + "table_creation_time": "1655831648843", + "id": "de711767-c7b9-4c33-99d7-510978dc1fa5", + "version": "4", + "location": "tests/integration/delta_lake/test_data/delta_tables", + "operation": "WRITE", + "operationParameters": "{}", + "readVersion": "3", + "isolationLevel": "Serializable", + "isBlindAppend": "True", + "operationMetrics": "{}", + "engineInfo": "local Delta-Standalone/0.4.0", + "version_creation_time": "1655831649788" + }, + "externalUrl": null, + "name": "my_table_no_name", + "qualifiedName": null, + "description": null, + "uri": null, + "tags": [] + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "my_table_no_name", + "platform": "urn:li:dataPlatform:delta-lake", + "version": 4, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown", + "impersonator": null + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown", + "impersonator": null + }, + "deleted": null, + "dataset": null, + "cluster": null, + "hash": "", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.OtherSchema": { + "rawSchema": "" + } + }, + "fields": [ + { + "fieldPath": "bar", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "integer", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": true, + "jsonProps": null + }, + { + "fieldPath": "foo", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "integer", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": true, + "jsonProps": null + }, + { + "fieldPath": "zip", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "string", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": false, + "jsonProps": null + } + ], + "primaryKeys": null, + "foreignKeysSpecs": null, + "foreignKeys": null + } + } + ] + } + }, + "proposedDelta": null, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:bdfaaacd66870755e65612e0b88dd4bf", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "containerProperties", + "aspect": { + "value": "{\"customProperties\": {\"platform\": \"delta-lake\", \"instance\": \"UAT\", \"folder_abs_path\": \"tests\"}, \"name\": \"tests\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:bdfaaacd66870755e65612e0b88dd4bf", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "value": "{\"platform\": \"urn:li:dataPlatform:delta-lake\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:bdfaaacd66870755e65612e0b88dd4bf", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "value": "{\"typeNames\": [\"Folder\"]}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:974a39dc631803eddedc699cc9bb9759", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "containerProperties", + "aspect": { + "value": "{\"customProperties\": {\"platform\": \"delta-lake\", \"instance\": \"UAT\", \"folder_abs_path\": \"tests/integration\"}, \"name\": \"integration\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:974a39dc631803eddedc699cc9bb9759", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "value": "{\"platform\": \"urn:li:dataPlatform:delta-lake\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:974a39dc631803eddedc699cc9bb9759", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "value": "{\"typeNames\": [\"Folder\"]}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:974a39dc631803eddedc699cc9bb9759", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "value": "{\"container\": \"urn:li:container:bdfaaacd66870755e65612e0b88dd4bf\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:dae543a1ed7ecfea4079a971dc7805a6", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "containerProperties", + "aspect": { + "value": "{\"customProperties\": {\"platform\": \"delta-lake\", \"instance\": \"UAT\", \"folder_abs_path\": \"tests/integration/delta_lake\"}, \"name\": \"delta_lake\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:dae543a1ed7ecfea4079a971dc7805a6", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "value": "{\"platform\": \"urn:li:dataPlatform:delta-lake\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:dae543a1ed7ecfea4079a971dc7805a6", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "value": "{\"typeNames\": [\"Folder\"]}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:dae543a1ed7ecfea4079a971dc7805a6", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "value": "{\"container\": \"urn:li:container:974a39dc631803eddedc699cc9bb9759\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:ee050cda8eca59687021c24cbc0bb8a4", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "containerProperties", + "aspect": { + "value": "{\"customProperties\": {\"platform\": \"delta-lake\", \"instance\": \"UAT\", \"folder_abs_path\": \"tests/integration/delta_lake/test_data\"}, \"name\": \"test_data\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:ee050cda8eca59687021c24cbc0bb8a4", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "value": "{\"platform\": \"urn:li:dataPlatform:delta-lake\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:ee050cda8eca59687021c24cbc0bb8a4", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "value": "{\"typeNames\": [\"Folder\"]}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:ee050cda8eca59687021c24cbc0bb8a4", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "value": "{\"container\": \"urn:li:container:dae543a1ed7ecfea4079a971dc7805a6\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:ad4b596846e8e010114b1ec82b324fab", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "containerProperties", + "aspect": { + "value": "{\"customProperties\": {\"platform\": \"delta-lake\", \"instance\": \"UAT\", \"folder_abs_path\": \"tests/integration/delta_lake/test_data/delta_tables\"}, \"name\": \"delta_tables\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:ad4b596846e8e010114b1ec82b324fab", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "value": "{\"platform\": \"urn:li:dataPlatform:delta-lake\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:ad4b596846e8e010114b1ec82b324fab", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "value": "{\"typeNames\": [\"Folder\"]}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:ad4b596846e8e010114b1ec82b324fab", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "value": "{\"container\": \"urn:li:container:ee050cda8eca59687021c24cbc0bb8a4\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:delta-lake,tests/integration/delta_lake/test_data/delta_tables/my_table_no_name,UAT)", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "value": "{\"container\": \"urn:li:container:ad4b596846e8e010114b1ec82b324fab\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:delta-lake,tests/integration/delta_lake/test_data/delta_tables/sales,UAT)", + "aspects": [ + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": { + "number_of_files": "3", + "partition_columns": "[]", + "table_creation_time": "1655664813952", + "id": "eca9d2a0-4ce6-4ace-a732-75fda0157fb8", + "version": "0", + "location": "tests/integration/delta_lake/test_data/delta_tables", + "operation": "CONVERT", + "operationParameters": "{}", + "isolationLevel": "Serializable", + "isBlindAppend": "True", + "operationMetrics": "{}", + "engineInfo": "local Delta-Standalone/0.4.0", + "version_creation_time": "1655664815399" + }, + "externalUrl": null, + "name": "my_table", + "qualifiedName": null, + "description": "my table description", + "uri": null, + "tags": [] + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "my_table", + "platform": "urn:li:dataPlatform:delta-lake", + "version": 0, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown", + "impersonator": null + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown", + "impersonator": null + }, + "deleted": null, + "dataset": null, + "cluster": null, + "hash": "", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.OtherSchema": { + "rawSchema": "" + } + }, + "fields": [ + { + "fieldPath": "customer", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "string", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": false, + "jsonProps": null + }, + { + "fieldPath": "day", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "integer", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": false, + "jsonProps": null + }, + { + "fieldPath": "month", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "integer", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": false, + "jsonProps": null + }, + { + "fieldPath": "sale_id", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "string", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": false, + "jsonProps": null + }, + { + "fieldPath": "total_cost", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NullType": {} + } + }, + "nativeDataType": "float", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": false, + "jsonProps": null + }, + { + "fieldPath": "year", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "integer", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": false, + "jsonProps": null + } + ], + "primaryKeys": null, + "foreignKeysSpecs": null, + "foreignKeys": null + } + } + ] + } + }, + "proposedDelta": null, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:delta-lake,tests/integration/delta_lake/test_data/delta_tables/sales,UAT)", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "value": "{\"container\": \"urn:li:container:ad4b596846e8e010114b1ec82b324fab\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:delta-lake,tests/integration/delta_lake/test_data/delta_tables/my_table_basic,UAT)", + "aspects": [ + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": { + "number_of_files": "5", + "partition_columns": "['foo', 'bar']", + "table_creation_time": "1655831476360", + "id": "628d06df-ecb0-4314-a97e-75d8872db7c3", + "version": "4", + "location": "tests/integration/delta_lake/test_data/delta_tables", + "operation": "WRITE", + "operationParameters": "{}", + "readVersion": "3", + "isolationLevel": "Serializable", + "isBlindAppend": "True", + "operationMetrics": "{}", + "engineInfo": "local Delta-Standalone/0.4.0", + "version_creation_time": "1655831477768" + }, + "externalUrl": null, + "name": "test-table-basic", + "qualifiedName": null, + "description": "test delta table basic with table name", + "uri": null, + "tags": [] + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "test-table-basic", + "platform": "urn:li:dataPlatform:delta-lake", + "version": 4, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown", + "impersonator": null + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown", + "impersonator": null + }, + "deleted": null, + "dataset": null, + "cluster": null, + "hash": "", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.OtherSchema": { + "rawSchema": "" + } + }, + "fields": [ + { + "fieldPath": "bar", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "integer", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": true, + "jsonProps": null + }, + { + "fieldPath": "foo", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "integer", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": true, + "jsonProps": null + }, + { + "fieldPath": "zip", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "string", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": false, + "jsonProps": null + } + ], + "primaryKeys": null, + "foreignKeysSpecs": null, + "foreignKeys": null + } + } + ] + } + }, + "proposedDelta": null, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:delta-lake,tests/integration/delta_lake/test_data/delta_tables/my_table_basic,UAT)", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "value": "{\"container\": \"urn:li:container:ad4b596846e8e010114b1ec82b324fab\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:delta-lake,tests/integration/delta_lake/test_data/delta_tables/level1/my_table_inner,UAT)", + "aspects": [ + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": { + "number_of_files": "5", + "partition_columns": "['foo', 'bar']", + "table_creation_time": "1655831864836", + "id": "3775a0fd-4f58-4dea-b71a-e3fedb10f5b4", + "version": "4", + "location": "tests/integration/delta_lake/test_data/delta_tables", + "operation": "WRITE", + "operationParameters": "{}", + "readVersion": "3", + "isolationLevel": "Serializable", + "isBlindAppend": "True", + "operationMetrics": "{}", + "engineInfo": "local Delta-Standalone/0.4.0", + "version_creation_time": "1655831866541" + }, + "externalUrl": null, + "name": "test-table-inner", + "qualifiedName": null, + "description": "test delta table basic with table name at inner location", + "uri": null, + "tags": [] + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "test-table-inner", + "platform": "urn:li:dataPlatform:delta-lake", + "version": 4, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown", + "impersonator": null + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown", + "impersonator": null + }, + "deleted": null, + "dataset": null, + "cluster": null, + "hash": "", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.OtherSchema": { + "rawSchema": "" + } + }, + "fields": [ + { + "fieldPath": "bar", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "integer", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": true, + "jsonProps": null + }, + { + "fieldPath": "foo", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "integer", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": true, + "jsonProps": null + }, + { + "fieldPath": "zip", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "string", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": false, + "jsonProps": null + } + ], + "primaryKeys": null, + "foreignKeysSpecs": null, + "foreignKeys": null + } + } + ] + } + }, + "proposedDelta": null, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:6bb6dc6de93177210067d00b45b481bb", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "containerProperties", + "aspect": { + "value": "{\"customProperties\": {\"platform\": \"delta-lake\", \"instance\": \"UAT\", \"folder_abs_path\": \"tests/integration/delta_lake/test_data/delta_tables/level1\"}, \"name\": \"level1\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:6bb6dc6de93177210067d00b45b481bb", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "value": "{\"platform\": \"urn:li:dataPlatform:delta-lake\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:6bb6dc6de93177210067d00b45b481bb", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "value": "{\"typeNames\": [\"Folder\"]}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:6bb6dc6de93177210067d00b45b481bb", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "value": "{\"container\": \"urn:li:container:ad4b596846e8e010114b1ec82b324fab\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:delta-lake,tests/integration/delta_lake/test_data/delta_tables/level1/my_table_inner,UAT)", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "value": "{\"container\": \"urn:li:container:6bb6dc6de93177210067d00b45b481bb\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "inner_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +} +] \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/delta_lake/golden_files/local/golden_mces_relative_path.json b/metadata-ingestion/tests/integration/delta_lake/golden_files/local/golden_mces_relative_path.json new file mode 100644 index 00000000000000..7ff5db60855c30 --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/golden_files/local/golden_mces_relative_path.json @@ -0,0 +1,213 @@ +[ +{ + "auditHeader": null, + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:delta-lake,delta_tables/my_table_basic,UAT)", + "aspects": [ + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": { + "number_of_files": "5", + "partition_columns": "['foo', 'bar']", + "table_creation_time": "1655831476360", + "id": "628d06df-ecb0-4314-a97e-75d8872db7c3", + "version": "4", + "location": "tests/integration/delta_lake/test_data/delta_tables/my_table_basic/", + "operation": "WRITE", + "operationParameters": "{}", + "readVersion": "3", + "isolationLevel": "Serializable", + "isBlindAppend": "True", + "operationMetrics": "{}", + "engineInfo": "local Delta-Standalone/0.4.0", + "version_creation_time": "1655831477768" + }, + "externalUrl": null, + "name": "test-table-basic", + "qualifiedName": null, + "description": "test delta table basic with table name", + "uri": null, + "tags": [] + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "test-table-basic", + "platform": "urn:li:dataPlatform:delta-lake", + "version": 4, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown", + "impersonator": null + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown", + "impersonator": null + }, + "deleted": null, + "dataset": null, + "cluster": null, + "hash": "", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.OtherSchema": { + "rawSchema": "" + } + }, + "fields": [ + { + "fieldPath": "bar", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "integer", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": true, + "jsonProps": null + }, + { + "fieldPath": "foo", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "integer", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": true, + "jsonProps": null + }, + { + "fieldPath": "zip", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "string", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": false, + "jsonProps": null + } + ], + "primaryKeys": null, + "foreignKeysSpecs": null, + "foreignKeys": null + } + } + ] + } + }, + "proposedDelta": null, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "relative_path.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:85267d161e1a2ffa647cec6c1188549f", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "containerProperties", + "aspect": { + "value": "{\"customProperties\": {\"platform\": \"delta-lake\", \"instance\": \"UAT\", \"folder_abs_path\": \"delta_tables\"}, \"name\": \"delta_tables\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "relative_path.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:85267d161e1a2ffa647cec6c1188549f", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "value": "{\"platform\": \"urn:li:dataPlatform:delta-lake\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "relative_path.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:85267d161e1a2ffa647cec6c1188549f", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "value": "{\"typeNames\": [\"Folder\"]}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "relative_path.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:delta-lake,delta_tables/my_table_basic,UAT)", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "value": "{\"container\": \"urn:li:container:85267d161e1a2ffa647cec6c1188549f\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "relative_path.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +} +] \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/delta_lake/golden_files/local/golden_mces_single_table.json b/metadata-ingestion/tests/integration/delta_lake/golden_files/local/golden_mces_single_table.json new file mode 100644 index 00000000000000..3bde0bc9da303a --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/golden_files/local/golden_mces_single_table.json @@ -0,0 +1,517 @@ +[ +{ + "auditHeader": null, + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:delta-lake,tests/integration/delta_lake/test_data/delta_tables/my_table_basic,UAT)", + "aspects": [ + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": { + "number_of_files": "5", + "partition_columns": "['foo', 'bar']", + "table_creation_time": "1655831476360", + "id": "628d06df-ecb0-4314-a97e-75d8872db7c3", + "version": "4", + "location": "tests/integration/delta_lake/test_data/delta_tables/my_table_basic", + "operation": "WRITE", + "operationParameters": "{}", + "readVersion": "3", + "isolationLevel": "Serializable", + "isBlindAppend": "True", + "operationMetrics": "{}", + "engineInfo": "local Delta-Standalone/0.4.0", + "version_creation_time": "1655831477768" + }, + "externalUrl": null, + "name": "test-table-basic", + "qualifiedName": null, + "description": "test delta table basic with table name", + "uri": null, + "tags": [] + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "test-table-basic", + "platform": "urn:li:dataPlatform:delta-lake", + "version": 4, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown", + "impersonator": null + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown", + "impersonator": null + }, + "deleted": null, + "dataset": null, + "cluster": null, + "hash": "", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.OtherSchema": { + "rawSchema": "" + } + }, + "fields": [ + { + "fieldPath": "bar", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "integer", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": true, + "jsonProps": null + }, + { + "fieldPath": "foo", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "integer", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": true, + "jsonProps": null + }, + { + "fieldPath": "zip", + "jsonPath": null, + "nullable": true, + "description": "{}", + "created": null, + "lastModified": null, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "string", + "recursive": false, + "globalTags": null, + "glossaryTerms": null, + "isPartOfKey": false, + "isPartitioningKey": false, + "jsonProps": null + } + ], + "primaryKeys": null, + "foreignKeysSpecs": null, + "foreignKeys": null + } + } + ] + } + }, + "proposedDelta": null, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "single_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:bdfaaacd66870755e65612e0b88dd4bf", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "containerProperties", + "aspect": { + "value": "{\"customProperties\": {\"platform\": \"delta-lake\", \"instance\": \"UAT\", \"folder_abs_path\": \"tests\"}, \"name\": \"tests\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "single_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:bdfaaacd66870755e65612e0b88dd4bf", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "value": "{\"platform\": \"urn:li:dataPlatform:delta-lake\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "single_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:bdfaaacd66870755e65612e0b88dd4bf", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "value": "{\"typeNames\": [\"Folder\"]}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "single_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:974a39dc631803eddedc699cc9bb9759", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "containerProperties", + "aspect": { + "value": "{\"customProperties\": {\"platform\": \"delta-lake\", \"instance\": \"UAT\", \"folder_abs_path\": \"tests/integration\"}, \"name\": \"integration\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "single_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:974a39dc631803eddedc699cc9bb9759", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "value": "{\"platform\": \"urn:li:dataPlatform:delta-lake\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "single_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:974a39dc631803eddedc699cc9bb9759", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "value": "{\"typeNames\": [\"Folder\"]}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "single_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:974a39dc631803eddedc699cc9bb9759", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "value": "{\"container\": \"urn:li:container:bdfaaacd66870755e65612e0b88dd4bf\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "single_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:dae543a1ed7ecfea4079a971dc7805a6", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "containerProperties", + "aspect": { + "value": "{\"customProperties\": {\"platform\": \"delta-lake\", \"instance\": \"UAT\", \"folder_abs_path\": \"tests/integration/delta_lake\"}, \"name\": \"delta_lake\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "single_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:dae543a1ed7ecfea4079a971dc7805a6", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "value": "{\"platform\": \"urn:li:dataPlatform:delta-lake\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "single_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:dae543a1ed7ecfea4079a971dc7805a6", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "value": "{\"typeNames\": [\"Folder\"]}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "single_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:dae543a1ed7ecfea4079a971dc7805a6", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "value": "{\"container\": \"urn:li:container:974a39dc631803eddedc699cc9bb9759\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "single_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:ee050cda8eca59687021c24cbc0bb8a4", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "containerProperties", + "aspect": { + "value": "{\"customProperties\": {\"platform\": \"delta-lake\", \"instance\": \"UAT\", \"folder_abs_path\": \"tests/integration/delta_lake/test_data\"}, \"name\": \"test_data\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "single_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:ee050cda8eca59687021c24cbc0bb8a4", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "value": "{\"platform\": \"urn:li:dataPlatform:delta-lake\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "single_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:ee050cda8eca59687021c24cbc0bb8a4", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "value": "{\"typeNames\": [\"Folder\"]}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "single_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:ee050cda8eca59687021c24cbc0bb8a4", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "value": "{\"container\": \"urn:li:container:dae543a1ed7ecfea4079a971dc7805a6\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "single_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:ad4b596846e8e010114b1ec82b324fab", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "containerProperties", + "aspect": { + "value": "{\"customProperties\": {\"platform\": \"delta-lake\", \"instance\": \"UAT\", \"folder_abs_path\": \"tests/integration/delta_lake/test_data/delta_tables\"}, \"name\": \"delta_tables\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "single_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:ad4b596846e8e010114b1ec82b324fab", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "value": "{\"platform\": \"urn:li:dataPlatform:delta-lake\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "single_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:ad4b596846e8e010114b1ec82b324fab", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "value": "{\"typeNames\": [\"Folder\"]}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "single_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "container", + "entityUrn": "urn:li:container:ad4b596846e8e010114b1ec82b324fab", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "value": "{\"container\": \"urn:li:container:ee050cda8eca59687021c24cbc0bb8a4\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "single_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +}, +{ + "auditHeader": null, + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:delta-lake,tests/integration/delta_lake/test_data/delta_tables/my_table_basic,UAT)", + "entityKeyAspect": null, + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "value": "{\"container\": \"urn:li:container:ad4b596846e8e010114b1ec82b324fab\"}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1615443388097, + "runId": "single_table.json", + "registryName": null, + "registryVersion": null, + "properties": null + } +} +] \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/delta_lake/sources/local/allow_table.json b/metadata-ingestion/tests/integration/delta_lake/sources/local/allow_table.json new file mode 100644 index 00000000000000..1001eaadb3b9b5 --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/sources/local/allow_table.json @@ -0,0 +1,11 @@ +{ + "type": "delta-lake", + "config": { + "env": "UAT", + "base_path": "tests/integration/delta_lake/test_data/delta_tables", + "platform_instance": "my-platform", + "table_pattern": { + "allow": ["s*"] + } + } +} \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/delta_lake/sources/local/inner_table.json b/metadata-ingestion/tests/integration/delta_lake/sources/local/inner_table.json new file mode 100644 index 00000000000000..8997605527b4c2 --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/sources/local/inner_table.json @@ -0,0 +1,7 @@ +{ + "type": "delta-lake", + "config": { + "env": "UAT", + "base_path": "tests/integration/delta_lake/test_data/delta_tables" + } +} \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/delta_lake/sources/local/relative_path.json b/metadata-ingestion/tests/integration/delta_lake/sources/local/relative_path.json new file mode 100644 index 00000000000000..a1734aea8d3190 --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/sources/local/relative_path.json @@ -0,0 +1,8 @@ +{ + "type": "delta-lake", + "config": { + "env": "UAT", + "base_path": "tests/integration/delta_lake/test_data/", + "relative_path":"delta_tables/my_table_basic/" + } +} diff --git a/metadata-ingestion/tests/integration/delta_lake/sources/local/single_table.json b/metadata-ingestion/tests/integration/delta_lake/sources/local/single_table.json new file mode 100644 index 00000000000000..a12a7e71515a59 --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/sources/local/single_table.json @@ -0,0 +1,7 @@ +{ + "type": "delta-lake", + "config": { + "env": "UAT", + "base_path": "tests/integration/delta_lake/test_data/delta_tables/my_table_basic" + } +} \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/delta_lake/test-table/_delta_log/00000000000000000000.json b/metadata-ingestion/tests/integration/delta_lake/test-table/_delta_log/00000000000000000000.json new file mode 100644 index 00000000000000..c344027562aa8a --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/test-table/_delta_log/00000000000000000000.json @@ -0,0 +1,4 @@ +{"commitInfo":{"timestamp":1655724801453,"operation":"WRITE","operationParameters":{},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"local Delta-Standalone/0.4.0"}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"468553a9-3b52-4e50-b855-e64d2d070cb8","name":"test-table","description":"test delta table","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"foo\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"bar\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"zip\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["foo","bar"],"configuration":{},"createdTime":1655724801057}} +{"add":{"path":"0","partitionValues":{"bar":"0","foo":"0"},"size":100,"modificationTime":1655724801375,"dataChange":true,"tags":{"someTagKey":"someTagVal"}}} diff --git a/metadata-ingestion/tests/integration/delta_lake/test-table/_delta_log/00000000000000000001.json b/metadata-ingestion/tests/integration/delta_lake/test-table/_delta_log/00000000000000000001.json new file mode 100644 index 00000000000000..181b31f71db52d --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/test-table/_delta_log/00000000000000000001.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1655724802209,"operation":"WRITE","operationParameters":{},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"local Delta-Standalone/0.4.0"}} +{"add":{"path":"1","partitionValues":{"bar":"1","foo":"1"},"size":100,"modificationTime":1655724802208,"dataChange":true,"tags":{"someTagKey":"someTagVal"}}} diff --git a/metadata-ingestion/tests/integration/delta_lake/test-table/_delta_log/00000000000000000002.json b/metadata-ingestion/tests/integration/delta_lake/test-table/_delta_log/00000000000000000002.json new file mode 100644 index 00000000000000..007ed0efdd831b --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/test-table/_delta_log/00000000000000000002.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1655724802227,"operation":"WRITE","operationParameters":{},"readVersion":1,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"local Delta-Standalone/0.4.0"}} +{"add":{"path":"2","partitionValues":{"bar":"0","foo":"2"},"size":100,"modificationTime":1655724802226,"dataChange":true,"tags":{"someTagKey":"someTagVal"}}} diff --git a/metadata-ingestion/tests/integration/delta_lake/test-table/_delta_log/00000000000000000003.json b/metadata-ingestion/tests/integration/delta_lake/test-table/_delta_log/00000000000000000003.json new file mode 100644 index 00000000000000..8ed5a1c9ec6b15 --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/test-table/_delta_log/00000000000000000003.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1655724802244,"operation":"WRITE","operationParameters":{},"readVersion":2,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"local Delta-Standalone/0.4.0"}} +{"add":{"path":"3","partitionValues":{"bar":"1","foo":"0"},"size":100,"modificationTime":1655724802243,"dataChange":true,"tags":{"someTagKey":"someTagVal"}}} diff --git a/metadata-ingestion/tests/integration/delta_lake/test-table/_delta_log/00000000000000000004.json b/metadata-ingestion/tests/integration/delta_lake/test-table/_delta_log/00000000000000000004.json new file mode 100644 index 00000000000000..23423b22f9fdcf --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/test-table/_delta_log/00000000000000000004.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1655724802264,"operation":"WRITE","operationParameters":{},"readVersion":3,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"local Delta-Standalone/0.4.0"}} +{"add":{"path":"4","partitionValues":{"bar":"0","foo":"1"},"size":100,"modificationTime":1655724802263,"dataChange":true,"tags":{"someTagKey":"someTagVal"}}} diff --git a/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/level1/my_table_inner/_delta_log/00000000000000000000.json b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/level1/my_table_inner/_delta_log/00000000000000000000.json new file mode 100644 index 00000000000000..f93204d086bdc8 --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/level1/my_table_inner/_delta_log/00000000000000000000.json @@ -0,0 +1,4 @@ +{"commitInfo":{"timestamp":1655831865396,"operation":"WRITE","operationParameters":{},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"local Delta-Standalone/0.4.0"}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"3775a0fd-4f58-4dea-b71a-e3fedb10f5b4","name":"test-table-inner","description":"test delta table basic with table name at inner location","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"foo\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"bar\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"zip\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["foo","bar"],"configuration":{},"createdTime":1655831864836}} +{"add":{"path":"0","partitionValues":{"bar":"0","foo":"0"},"size":100,"modificationTime":1655831865289,"dataChange":true,"tags":{"someTagKey":"someTagVal"}}} diff --git a/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/level1/my_table_inner/_delta_log/00000000000000000001.json b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/level1/my_table_inner/_delta_log/00000000000000000001.json new file mode 100644 index 00000000000000..80df25b214793e --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/level1/my_table_inner/_delta_log/00000000000000000001.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1655831866337,"operation":"WRITE","operationParameters":{},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"local Delta-Standalone/0.4.0"}} +{"add":{"path":"1","partitionValues":{"bar":"1","foo":"1"},"size":100,"modificationTime":1655831866336,"dataChange":true,"tags":{"someTagKey":"someTagVal"}}} diff --git a/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/level1/my_table_inner/_delta_log/00000000000000000002.json b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/level1/my_table_inner/_delta_log/00000000000000000002.json new file mode 100644 index 00000000000000..936328a405e294 --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/level1/my_table_inner/_delta_log/00000000000000000002.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1655831866398,"operation":"WRITE","operationParameters":{},"readVersion":1,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"local Delta-Standalone/0.4.0"}} +{"add":{"path":"2","partitionValues":{"bar":"0","foo":"2"},"size":100,"modificationTime":1655831866397,"dataChange":true,"tags":{"someTagKey":"someTagVal"}}} diff --git a/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/level1/my_table_inner/_delta_log/00000000000000000003.json b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/level1/my_table_inner/_delta_log/00000000000000000003.json new file mode 100644 index 00000000000000..ea66495b76f64f --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/level1/my_table_inner/_delta_log/00000000000000000003.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1655831866447,"operation":"WRITE","operationParameters":{},"readVersion":2,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"local Delta-Standalone/0.4.0"}} +{"add":{"path":"3","partitionValues":{"bar":"1","foo":"0"},"size":100,"modificationTime":1655831866446,"dataChange":true,"tags":{"someTagKey":"someTagVal"}}} diff --git a/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/level1/my_table_inner/_delta_log/00000000000000000004.json b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/level1/my_table_inner/_delta_log/00000000000000000004.json new file mode 100644 index 00000000000000..72b451d92fac70 --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/level1/my_table_inner/_delta_log/00000000000000000004.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1655831866541,"operation":"WRITE","operationParameters":{},"readVersion":3,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"local Delta-Standalone/0.4.0"}} +{"add":{"path":"4","partitionValues":{"bar":"0","foo":"1"},"size":100,"modificationTime":1655831866541,"dataChange":true,"tags":{"someTagKey":"someTagVal"}}} diff --git a/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_basic/_delta_log/00000000000000000000.json b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_basic/_delta_log/00000000000000000000.json new file mode 100644 index 00000000000000..e913b115556d51 --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_basic/_delta_log/00000000000000000000.json @@ -0,0 +1,4 @@ +{"commitInfo":{"timestamp":1655831476907,"operation":"WRITE","operationParameters":{},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"local Delta-Standalone/0.4.0"}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"628d06df-ecb0-4314-a97e-75d8872db7c3","name":"test-table-basic","description":"test delta table basic with table name","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"foo\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"bar\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"zip\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["foo","bar"],"configuration":{},"createdTime":1655831476360}} +{"add":{"path":"0","partitionValues":{"bar":"0","foo":"0"},"size":100,"modificationTime":1655831476810,"dataChange":true,"tags":{"someTagKey":"someTagVal"}}} diff --git a/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_basic/_delta_log/00000000000000000001.json b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_basic/_delta_log/00000000000000000001.json new file mode 100644 index 00000000000000..adf1ac30f5ec23 --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_basic/_delta_log/00000000000000000001.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1655831477701,"operation":"WRITE","operationParameters":{},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"local Delta-Standalone/0.4.0"}} +{"add":{"path":"1","partitionValues":{"bar":"1","foo":"1"},"size":100,"modificationTime":1655831477701,"dataChange":true,"tags":{"someTagKey":"someTagVal"}}} diff --git a/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_basic/_delta_log/00000000000000000002.json b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_basic/_delta_log/00000000000000000002.json new file mode 100644 index 00000000000000..9c2c7935fd05ab --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_basic/_delta_log/00000000000000000002.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1655831477726,"operation":"WRITE","operationParameters":{},"readVersion":1,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"local Delta-Standalone/0.4.0"}} +{"add":{"path":"2","partitionValues":{"bar":"0","foo":"2"},"size":100,"modificationTime":1655831477725,"dataChange":true,"tags":{"someTagKey":"someTagVal"}}} diff --git a/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_basic/_delta_log/00000000000000000003.json b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_basic/_delta_log/00000000000000000003.json new file mode 100644 index 00000000000000..b3f11d22d01ea2 --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_basic/_delta_log/00000000000000000003.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1655831477745,"operation":"WRITE","operationParameters":{},"readVersion":2,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"local Delta-Standalone/0.4.0"}} +{"add":{"path":"3","partitionValues":{"bar":"1","foo":"0"},"size":100,"modificationTime":1655831477745,"dataChange":true,"tags":{"someTagKey":"someTagVal"}}} diff --git a/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_basic/_delta_log/00000000000000000004.json b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_basic/_delta_log/00000000000000000004.json new file mode 100644 index 00000000000000..f49ef6eb938d57 --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_basic/_delta_log/00000000000000000004.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1655831477768,"operation":"WRITE","operationParameters":{},"readVersion":3,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"local Delta-Standalone/0.4.0"}} +{"add":{"path":"4","partitionValues":{"bar":"0","foo":"1"},"size":100,"modificationTime":1655831477768,"dataChange":true,"tags":{"someTagKey":"someTagVal"}}} diff --git a/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_no_name/_delta_log/00000000000000000000.json b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_no_name/_delta_log/00000000000000000000.json new file mode 100644 index 00000000000000..16aaa1eb8f773b --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_no_name/_delta_log/00000000000000000000.json @@ -0,0 +1,4 @@ +{"commitInfo":{"timestamp":1655831649166,"operation":"WRITE","operationParameters":{},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"local Delta-Standalone/0.4.0"}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"de711767-c7b9-4c33-99d7-510978dc1fa5","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"foo\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"bar\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"zip\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["foo","bar"],"configuration":{},"createdTime":1655831648843}} +{"add":{"path":"0","partitionValues":{"bar":"0","foo":"0"},"size":100,"modificationTime":1655831649141,"dataChange":true,"tags":{"someTagKey":"someTagVal"}}} diff --git a/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_no_name/_delta_log/00000000000000000001.json b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_no_name/_delta_log/00000000000000000001.json new file mode 100644 index 00000000000000..023dec3c38175b --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_no_name/_delta_log/00000000000000000001.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1655831649715,"operation":"WRITE","operationParameters":{},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"local Delta-Standalone/0.4.0"}} +{"add":{"path":"1","partitionValues":{"bar":"1","foo":"1"},"size":100,"modificationTime":1655831649715,"dataChange":true,"tags":{"someTagKey":"someTagVal"}}} diff --git a/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_no_name/_delta_log/00000000000000000002.json b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_no_name/_delta_log/00000000000000000002.json new file mode 100644 index 00000000000000..ef50652bfd336c --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_no_name/_delta_log/00000000000000000002.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1655831649731,"operation":"WRITE","operationParameters":{},"readVersion":1,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"local Delta-Standalone/0.4.0"}} +{"add":{"path":"2","partitionValues":{"bar":"0","foo":"2"},"size":100,"modificationTime":1655831649731,"dataChange":true,"tags":{"someTagKey":"someTagVal"}}} diff --git a/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_no_name/_delta_log/00000000000000000003.json b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_no_name/_delta_log/00000000000000000003.json new file mode 100644 index 00000000000000..abaefe3d71b136 --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_no_name/_delta_log/00000000000000000003.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1655831649754,"operation":"WRITE","operationParameters":{},"readVersion":2,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"local Delta-Standalone/0.4.0"}} +{"add":{"path":"3","partitionValues":{"bar":"1","foo":"0"},"size":100,"modificationTime":1655831649754,"dataChange":true,"tags":{"someTagKey":"someTagVal"}}} diff --git a/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_no_name/_delta_log/00000000000000000004.json b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_no_name/_delta_log/00000000000000000004.json new file mode 100644 index 00000000000000..238ede0b54d7d6 --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/my_table_no_name/_delta_log/00000000000000000004.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1655831649788,"operation":"WRITE","operationParameters":{},"readVersion":3,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"local Delta-Standalone/0.4.0"}} +{"add":{"path":"4","partitionValues":{"bar":"0","foo":"1"},"size":100,"modificationTime":1655831649787,"dataChange":true,"tags":{"someTagKey":"someTagVal"}}} diff --git a/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/._SUCCESS.crc b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/._SUCCESS.crc new file mode 100644 index 00000000000000..3b7b044936a890 Binary files /dev/null and b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/._SUCCESS.crc differ diff --git a/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/.part-00000-64c688b3-46cc-44c9-86a4-d1f07a3570c1-c000.snappy.parquet.crc b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/.part-00000-64c688b3-46cc-44c9-86a4-d1f07a3570c1-c000.snappy.parquet.crc new file mode 100644 index 00000000000000..193fbb5caeffe5 Binary files /dev/null and b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/.part-00000-64c688b3-46cc-44c9-86a4-d1f07a3570c1-c000.snappy.parquet.crc differ diff --git a/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/.part-00000-71b51e51-8746-425f-8e20-41dc771a1b47-c000.snappy.parquet.crc b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/.part-00000-71b51e51-8746-425f-8e20-41dc771a1b47-c000.snappy.parquet.crc new file mode 100644 index 00000000000000..833d4a9e30bef4 Binary files /dev/null and b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/.part-00000-71b51e51-8746-425f-8e20-41dc771a1b47-c000.snappy.parquet.crc differ diff --git a/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/.part-00000-c4f4550a-83d5-4c35-bdb6-9f8bd1a9d154-c000.snappy.parquet.crc b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/.part-00000-c4f4550a-83d5-4c35-bdb6-9f8bd1a9d154-c000.snappy.parquet.crc new file mode 100644 index 00000000000000..bf059b2f012c53 Binary files /dev/null and b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/.part-00000-c4f4550a-83d5-4c35-bdb6-9f8bd1a9d154-c000.snappy.parquet.crc differ diff --git a/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/_SUCCESS b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/_SUCCESS new file mode 100644 index 00000000000000..e69de29bb2d1d6 diff --git a/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/_delta_log/00000000000000000000.json b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/_delta_log/00000000000000000000.json new file mode 100644 index 00000000000000..c85c293b2fd9fe --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/_delta_log/00000000000000000000.json @@ -0,0 +1,6 @@ +{"commitInfo":{"timestamp":1655664815399,"operation":"CONVERT","operationParameters":{},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{},"engineInfo":"local Delta-Standalone/0.4.0"}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"eca9d2a0-4ce6-4ace-a732-75fda0157fb8","name":"my_table","description":"my table description","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"year\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"month\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"day\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"sale_id\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"customer\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"total_cost\",\"type\":\"float\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1655664813952}} +{"add":{"path":"part-00000-c4f4550a-83d5-4c35-bdb6-9f8bd1a9d154-c000.snappy.parquet","partitionValues":{},"size":318922,"modificationTime":1654667918316,"dataChange":true}} +{"add":{"path":"part-00000-64c688b3-46cc-44c9-86a4-d1f07a3570c1-c000.snappy.parquet","partitionValues":{},"size":319296,"modificationTime":1654667918332,"dataChange":true}} +{"add":{"path":"part-00000-71b51e51-8746-425f-8e20-41dc771a1b47-c000.snappy.parquet","partitionValues":{},"size":318646,"modificationTime":1654667918336,"dataChange":true}} diff --git a/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/part-00000-64c688b3-46cc-44c9-86a4-d1f07a3570c1-c000.snappy.parquet b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/part-00000-64c688b3-46cc-44c9-86a4-d1f07a3570c1-c000.snappy.parquet new file mode 100644 index 00000000000000..67353937c0663d Binary files /dev/null and b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/part-00000-64c688b3-46cc-44c9-86a4-d1f07a3570c1-c000.snappy.parquet differ diff --git a/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/part-00000-71b51e51-8746-425f-8e20-41dc771a1b47-c000.snappy.parquet b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/part-00000-71b51e51-8746-425f-8e20-41dc771a1b47-c000.snappy.parquet new file mode 100644 index 00000000000000..0219ea07b4cbdc Binary files /dev/null and b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/part-00000-71b51e51-8746-425f-8e20-41dc771a1b47-c000.snappy.parquet differ diff --git a/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/part-00000-c4f4550a-83d5-4c35-bdb6-9f8bd1a9d154-c000.snappy.parquet b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/part-00000-c4f4550a-83d5-4c35-bdb6-9f8bd1a9d154-c000.snappy.parquet new file mode 100644 index 00000000000000..802bc008a59c00 Binary files /dev/null and b/metadata-ingestion/tests/integration/delta_lake/test_data/delta_tables/sales/part-00000-c4f4550a-83d5-4c35-bdb6-9f8bd1a9d154-c000.snappy.parquet differ diff --git a/metadata-ingestion/tests/integration/delta_lake/test_local_delta_lake.py b/metadata-ingestion/tests/integration/delta_lake/test_local_delta_lake.py new file mode 100644 index 00000000000000..63c18ab8bc75ce --- /dev/null +++ b/metadata-ingestion/tests/integration/delta_lake/test_local_delta_lake.py @@ -0,0 +1,73 @@ +import json +import logging +import os + +import pytest + +from datahub.ingestion.run.pipeline import Pipeline +from tests.test_helpers import mce_helpers + +FROZEN_TIME = "2020-04-14 07:00:00" + +SOURCE_FILES_PATH = "./tests/integration/delta_lake/sources/local" +source_files = os.listdir(SOURCE_FILES_PATH) + + +@pytest.mark.parametrize("source_file", source_files) +def test_delta_lake(pytestconfig, source_file, tmp_path, mock_time): + test_resources_dir = pytestconfig.rootpath / "tests/integration/delta_lake" + + f = open(os.path.join(SOURCE_FILES_PATH, source_file)) + source = json.load(f) + + config_dict = {} + config_dict["source"] = source + config_dict["sink"] = { + "type": "file", + "config": { + "filename": f"{tmp_path}/{source_file}", + }, + # "type": "datahub-rest", + # "config": {"server": "http://localhost:8080"} + } + + config_dict["run_id"] = source_file + + pipeline = Pipeline.create(config_dict) + pipeline.run() + pipeline.raise_from_status() + + print(f"tmp pth: {tmp_path}") + print(f"source file : {source_file}") + print(f"testresource dir: {test_resources_dir}") + # Verify the output. + mce_helpers.check_golden_file( + pytestconfig, + output_path=f"{tmp_path}/{source_file}", + golden_path=f"{test_resources_dir}/golden_files/local/golden_mces_{source_file}", + ) + + +def test_data_lake_incorrect_config_raises_error(tmp_path, mock_time): + config_dict = {} + config_dict["sink"] = { + "type": "file", + "config": { + "filename": f"{tmp_path}/mces.json", + }, + # "type": "datahub-rest", + # "config": {"server": "http://localhost:8080"} + } + + # Case 1 : named variable in table name is not present in include + source = { + "type": "delta-lake", + "config": {"base_path": "invalid/path"}, + } + config_dict["source"] = source + with pytest.raises(Exception) as e_info: + pipeline = Pipeline.create(config_dict) + pipeline.run() + pipeline.raise_from_status() + + logging.debug(e_info) diff --git a/metadata-service/war/src/main/resources/boot/data_platforms.json b/metadata-service/war/src/main/resources/boot/data_platforms.json index 22dd867f23d7a3..79280bcc2a8ad0 100644 --- a/metadata-service/war/src/main/resources/boot/data_platforms.json +++ b/metadata-service/war/src/main/resources/boot/data_platforms.json @@ -494,5 +494,15 @@ "displayName": "N/A", "type": "OTHERS" } + }, + { + "urn": "urn:li:dataPlatform:delta-lake", + "aspect": { + "datasetNameDelimiter": ".", + "name": "delta-lake", + "displayName": "Delta Lake", + "type": "OTHERS", + "logoUrl": "/assets/platforms/deltalakelogo.png" + } } ]