diff --git a/metadata-ingestion/setup.py b/metadata-ingestion/setup.py index 867d315c14688..b4eb5be18110d 100644 --- a/metadata-ingestion/setup.py +++ b/metadata-ingestion/setup.py @@ -124,7 +124,7 @@ def get_long_description(): aws_common = { # AWS Python SDK - "boto3", + "boto3==1.24.59", # Deal with a version incompatibility between botocore (used by boto3) and urllib3. # See https://github.com/boto/botocore/pull/2563. "botocore!=1.23.0", @@ -199,7 +199,8 @@ def get_long_description(): iceberg_common = { # Iceberg Python SDK - "acryl-iceberg-legacy==0.0.4", + "fsspec[abfs]==2023.1.0", + "pyiceberg[adlfs,s3fs]==0.3.0", "azure-identity==1.10.0", } diff --git a/metadata-ingestion/src/datahub/ingestion/source/azure/azure_common.py b/metadata-ingestion/src/datahub/ingestion/source/azure/azure_common.py index 1a48725330df9..93a90cbd4b440 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/azure/azure_common.py +++ b/metadata-ingestion/src/datahub/ingestion/source/azure/azure_common.py @@ -1,7 +1,5 @@ -from typing import Dict, Optional, Union +from typing import Dict, Optional -from azure.identity import ClientSecretCredential -from azure.storage.filedatalake import DataLakeServiceClient, FileSystemClient from pydantic import Field, root_validator from datahub.configuration import ConfigModel @@ -47,29 +45,7 @@ class AdlsSourceConfig(ConfigModel): ) def get_abfss_url(self, folder_path: str = "") -> str: - if not folder_path.startswith("/"): - folder_path = f"/{folder_path}" - return f"abfss://{self.container_name}@{self.account_name}.dfs.core.windows.net{folder_path}" - - def get_filesystem_client(self) -> FileSystemClient: - return self.get_service_client().get_file_system_client(self.container_name) - - def get_service_client(self) -> DataLakeServiceClient: - return DataLakeServiceClient( - account_url=f"https://{self.account_name}.dfs.core.windows.net", - credential=self.get_credentials(), - ) - - def get_credentials( - self, - ) -> Union[Optional[str], ClientSecretCredential]: - if self.client_id and self.client_secret and self.tenant_id: - return ClientSecretCredential( - tenant_id=self.tenant_id, - client_id=self.client_id, - client_secret=self.client_secret, - ) - return self.sas_token if self.sas_token is not None else self.account_key + return f"abfss://{self.container_name}@{self.account_name}.dfs.core.windows.net{strltrim(folder_path, self.container_name)}" @root_validator() def _check_credential_values(cls, values: Dict) -> Dict: @@ -86,3 +62,7 @@ def _check_credential_values(cls, values: Dict) -> Dict: raise ConfigurationError( "credentials missing, requires one combination of account_key or sas_token or (client_id and client_secret and tenant_id)" ) + + +def strltrim(to_trim: str, prefix: str) -> str: + return to_trim[len(prefix) :] if to_trim.startswith(prefix) else to_trim diff --git a/metadata-ingestion/src/datahub/ingestion/source/iceberg/iceberg.py b/metadata-ingestion/src/datahub/ingestion/source/iceberg/iceberg.py index b45f20b87376e..53110008d0a3e 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/iceberg/iceberg.py +++ b/metadata-ingestion/src/datahub/ingestion/source/iceberg/iceberg.py @@ -1,14 +1,31 @@ import json import logging import uuid -from typing import Any, Dict, Iterable, List, Optional, Tuple - -from iceberg.api import types as IcebergTypes -from iceberg.api.table import Table -from iceberg.api.types.types import NestedField -from iceberg.core.base_table import BaseTable -from iceberg.core.filesystem.filesystem_tables import FilesystemTables -from iceberg.exceptions import NoSuchTableException +from typing import Any, Dict, Iterable, List, Optional + +from pyiceberg.exceptions import NoSuchIcebergTableError +from pyiceberg.schema import Schema, SchemaVisitorPerPrimitiveType, visit +from pyiceberg.table import Table +from pyiceberg.types import ( + BinaryType, + BooleanType, + DateType, + DecimalType, + DoubleType, + FixedType, + FloatType, + IntegerType, + ListType, + LongType, + MapType, + NestedField, + StringType, + StructType, + TimestampType, + TimestamptzType, + TimeType, + UUIDType, +) from datahub.emitter.mce_builder import ( make_data_platform_urn, @@ -65,13 +82,13 @@ LOGGER = logging.getLogger(__name__) _all_atomic_types = { - IcebergTypes.BooleanType: "boolean", - IcebergTypes.IntegerType: "int", - IcebergTypes.LongType: "long", - IcebergTypes.FloatType: "float", - IcebergTypes.DoubleType: "double", - IcebergTypes.BinaryType: "bytes", - IcebergTypes.StringType: "string", + BooleanType: "boolean", + IntegerType: "int", + LongType: "long", + FloatType: "float", + DoubleType: "double", + BinaryType: "bytes", + StringType: "string", } @@ -100,13 +117,13 @@ class IcebergSource(StatefulIngestionSourceBase): The DataHub Iceberg source plugin extracts metadata from [Iceberg tables](https://iceberg.apache.org/spec/) stored in a distributed or local file system. Typically, Iceberg tables are stored in a distributed file system like S3 or Azure Data Lake Storage (ADLS) and registered in a catalog. There are various catalog implementations like Filesystem-based, RDBMS-based or even REST-based catalogs. This Iceberg source plugin relies on the - [Iceberg python_legacy library](https://github.com/apache/iceberg/tree/master/python_legacy) and its support for catalogs is limited at the moment. - A new version of the [Iceberg Python library](https://github.com/apache/iceberg/tree/master/python) is currently in development and should fix this. + [pyiceberg library](https://py.iceberg.apache.org/) and its support for catalogs is limited at the moment. + A new version of pyiceberg is currently in development and should fix this. Because of this limitation, this source plugin **will only ingest HadoopCatalog-based tables that have a `version-hint.text` metadata file**. Ingestion of tables happens in 2 steps: 1. Discover Iceberg tables stored in file system. - 2. Load discovered tables using Iceberg python_legacy library + 2. Load discovered tables using pyiceberg. The current implementation of the Iceberg source plugin will only discover tables stored in a local file system or in ADLS. Support for S3 could be added fairly easily. @@ -117,7 +134,6 @@ def __init__(self, config: IcebergSourceConfig, ctx: PipelineContext) -> None: self.platform: str = "iceberg" self.report: IcebergSourceReport = IcebergSourceReport() self.config: IcebergSourceConfig = config - self.iceberg_client: FilesystemTables = config.filesystem_tables self.stale_entity_removal_handler = StaleEntityRemovalHandler( source=self, @@ -139,27 +155,58 @@ def get_workunits(self) -> Iterable[MetadataWorkUnit]: ) def get_workunits_internal(self) -> Iterable[MetadataWorkUnit]: - for dataset_path, dataset_name in self.config.get_paths(): # Tuple[str, str] - try: + catalog = self.config.get_catalog() + datasets = [] + if catalog: + for namespace in catalog.list_namespaces(): + for tableName in catalog.list_tables(namespace): + dataset_name = ".".join(tableName) + if not self.config.table_pattern.allowed(dataset_name): + # Dataset name is rejected by pattern, report as dropped. + self.report.report_dropped(dataset_name) + continue + else: + datasets.append((tableName, dataset_name)) + else: + # Will be obsolete once HadoopCatalog is supported by PyIceberg, or we migrate to REST catalog + for ( + dataset_path, + dataset_name, + ) in self.config.get_paths(): # Tuple[str, str] if not self.config.table_pattern.allowed(dataset_name): - # Path contained a valid Iceberg table, but is rejected by pattern. + # Dataset name is rejected by pattern, report as dropped. self.report.report_dropped(dataset_name) continue + else: + datasets.append((dataset_path, dataset_name)) - # Try to load an Iceberg table. Might not contain one, this will be caught by NoSuchTableException. - table: Table = self.iceberg_client.load(dataset_path) + for dataset_path, dataset_name in datasets: + try: + # Try to load an Iceberg table. Might not contain one, this will be caught by NoSuchIcebergTableError. + if isinstance(dataset_path, str): + table = self.config.load_table(dataset_name, dataset_path) + else: + table = catalog.load_table(dataset_path) yield from self._create_iceberg_workunit(dataset_name, table) - except NoSuchTableException: + dataset_urn: str = make_dataset_urn_with_platform_instance( + self.platform, + dataset_name, + self.config.platform_instance, + self.config.env, + ) + self.stale_entity_removal_handler.add_entity_to_state( + type="table", urn=dataset_urn + ) + except NoSuchIcebergTableError: # Path did not contain a valid Iceberg table. Silently ignore this. + # Once we move to catalogs, this won't be needed. LOGGER.debug( f"Path {dataset_path} does not contain table {dataset_name}" ) pass except Exception as e: - # This is a custom CCCS modification, when we complete CLDN-1784 we - # will be able to change this back to emit an exception instead of a warning - self.report.report_warning("general", f"Failed to create workunit: {e}") - LOGGER.warning( + self.report.report_failure("general", f"Failed to create workunit: {e}") + LOGGER.exception( f"Exception while processing table {dataset_path}, skipping it.", ) @@ -178,26 +225,21 @@ def _create_iceberg_workunit( aspects=[Status(removed=False)], ) - custom_properties: Dict = dict(table.properties()) - custom_properties["location"] = table.location() - try: - if isinstance(table, BaseTable) and table.current_snapshot(): - custom_properties["snapshot-id"] = str( - table.current_snapshot().snapshot_id - ) - custom_properties[ - "manifest-list" - ] = table.current_snapshot().manifest_location - except KeyError: - # The above API is not well implemented, and can throw KeyError when there is no data. - pass + # Dataset properties aspect. + custom_properties = table.metadata.properties.copy() + custom_properties["location"] = table.metadata.location + custom_properties["format-version"] = str(table.metadata.format_version) + if table.current_snapshot(): + custom_properties["snapshot-id"] = str(table.current_snapshot().snapshot_id) + custom_properties["manifest-list"] = table.current_snapshot().manifest_list dataset_properties = DatasetPropertiesClass( tags=[], - description=table.properties().get("comment", None), + description=table.metadata.properties.get("comment", None), customProperties=custom_properties, ) dataset_snapshot.aspects.append(dataset_properties) + # Dataset ownership aspect. dataset_ownership = self._get_ownership_aspect(table) if dataset_ownership: dataset_snapshot.aspects.append(dataset_ownership) @@ -221,8 +263,10 @@ def _create_iceberg_workunit( def _get_ownership_aspect(self, table: Table) -> Optional[OwnershipClass]: owners = [] if self.config.user_ownership_property: - if self.config.user_ownership_property in table.properties(): - user_owner = table.properties()[self.config.user_ownership_property] + if self.config.user_ownership_property in table.metadata.properties: + user_owner = table.metadata.properties[ + self.config.user_ownership_property + ] owners.append( OwnerClass( owner=make_user_urn(user_owner), @@ -231,8 +275,10 @@ def _get_ownership_aspect(self, table: Table) -> Optional[OwnershipClass]: ) ) if self.config.group_ownership_property: - if self.config.group_ownership_property in table.properties(): - group_owner = table.properties()[self.config.group_ownership_property] + if self.config.group_ownership_property in table.metadata.properties: + group_owner = table.metadata.properties[ + self.config.group_ownership_property + ] owners.append( OwnerClass( owner=make_group_urn(group_owner), @@ -240,9 +286,7 @@ def _get_ownership_aspect(self, table: Table) -> Optional[OwnershipClass]: source=None, ) ) - if owners: - return OwnershipClass(owners=owners) - return None + return OwnershipClass(owners=owners) if owners else None def _get_dataplatform_instance_aspect( self, dataset_urn: str @@ -267,58 +311,26 @@ def _get_dataplatform_instance_aspect( def _create_schema_metadata( self, dataset_name: str, table: Table ) -> SchemaMetadata: - schema_fields: List[SchemaField] = self._get_schema_fields( - table.schema().columns() - ) + schema_fields = self._get_schema_fields_for_schema(table.schema()) schema_metadata = SchemaMetadata( schemaName=dataset_name, platform=make_data_platform_urn(self.platform), version=0, hash="", - platformSchema=OtherSchema(rawSchema=repr(table.schema())), + platformSchema=OtherSchema(rawSchema=str(table.schema())), fields=schema_fields, ) return schema_metadata - def _get_schema_fields(self, columns: Tuple) -> List[SchemaField]: - canonical_schema: List[SchemaField] = [] - for column in columns: - fields = self._get_schema_fields_for_column(column) - canonical_schema.extend(fields) - return canonical_schema - - def _get_schema_fields_for_column( + def _get_schema_fields_for_schema( self, - column: NestedField, + schema: Schema, ) -> List[SchemaField]: - field_type: IcebergTypes.Type = column.type - if field_type.is_primitive_type() or field_type.is_nested_type(): - avro_schema: Dict = self._get_avro_schema_from_data_type(column) - schema_fields: List[SchemaField] = schema_util.avro_schema_to_mce_fields( - json.dumps(avro_schema), default_nullable=column.is_optional - ) - return schema_fields - - raise ValueError(f"Invalid Iceberg field type: {field_type}") - - def _get_avro_schema_from_data_type(self, column: NestedField) -> Dict[str, Any]: - """ - See Iceberg documentation for Avro mapping: - https://iceberg.apache.org/#spec/#appendix-a-format-specific-requirements - """ - # The record structure represents the dataset level. - # The inner fields represent the complex field (struct/array/map/union). - return { - "type": "record", - "name": "__struct_", - "fields": [ - { - "name": column.name, - "type": _parse_datatype(column.type, column.is_optional), - "doc": column.doc, - } - ], - } + avro_schema = visit(schema, ToAvroSchemaIcebergVisitor()) + schema_fields = schema_util.avro_schema_to_mce_fields( + json.dumps(avro_schema), default_nullable=False + ) + return schema_fields def get_platform_instance_id(self) -> str: assert self.config.platform_instance is not None @@ -328,134 +340,146 @@ def get_report(self) -> SourceReport: return self.report -def _parse_datatype(type: IcebergTypes.Type, nullable: bool = False) -> Dict[str, Any]: - # Check for complex types: struct, list, map - if type.is_list_type(): - list_type: IcebergTypes.ListType = type +class ToAvroSchemaIcebergVisitor(SchemaVisitorPerPrimitiveType[Dict[str, Any]]): + """Implementation of a visitor to build an Avro schema as a dictionary from an Iceberg schema.""" + + @staticmethod + def _gen_name(prefix: str) -> str: + return f"{prefix}{str(uuid.uuid4()).replace('-', '')}" + + def schema(self, schema: Schema, struct_result: Dict[str, Any]) -> Dict[str, Any]: + return struct_result + + def struct( + self, struct: StructType, field_results: List[Dict[str, Any]] + ) -> Dict[str, Any]: + nullable = True return { - "type": "array", - "items": _parse_datatype(list_type.element_type), - "native_data_type": str(type), + "type": "record", + "name": self._gen_name("__struct_"), + "fields": field_results, + "native_data_type": str(struct), "_nullable": nullable, } - elif type.is_map_type(): + + def field(self, field: NestedField, field_result: Dict[str, Any]) -> Dict[str, Any]: + field_result["_nullable"] = not field.required + return { + "name": field.name, + "type": field_result, + "doc": field.doc, + } + + def list( + self, list_type: ListType, element_result: Dict[str, Any] + ) -> Dict[str, Any]: + return { + "type": "array", + "items": element_result, + "native_data_type": str(list_type), + "_nullable": not list_type.element_required, + } + + def map( + self, + map_type: MapType, + key_result: Dict[str, Any], + value_result: Dict[str, Any], + ) -> Dict[str, Any]: # The Iceberg Map type will be handled differently. The idea is to translate the map # similar to the Map.Entry struct of Java i.e. as an array of map_entry struct, where # the map_entry struct has a key field and a value field. The key and value type can # be complex or primitive types. - map_type: IcebergTypes.MapType = type - map_entry: Dict[str, Any] = { + key_result["_nullable"] = False + value_result["_nullable"] = not map_type.value_required + map_entry = { "type": "record", - "name": _gen_name("__map_entry_"), + "name": self._gen_name("__map_entry_"), "fields": [ { "name": "key", - "type": _parse_datatype(map_type.key_type(), False), + "type": key_result, }, { "name": "value", - "type": _parse_datatype(map_type.value_type(), True), + "type": value_result, }, ], } return { "type": "array", "items": map_entry, - "native_data_type": str(type), - "_nullable": nullable, + "native_data_type": str(map_type), } - elif type.is_struct_type(): - structType: IcebergTypes.StructType = type - return _parse_struct_fields(structType.fields, nullable) - else: - # Primitive types - return _parse_basic_datatype(type, nullable) - - -def _parse_struct_fields(parts: Tuple[NestedField], nullable: bool) -> Dict[str, Any]: - fields = [] - for nested_field in parts: # type: NestedField - field_name = nested_field.name - field_type = _parse_datatype(nested_field.type, nested_field.is_optional) - fields.append({"name": field_name, "type": field_type, "doc": nested_field.doc}) - return { - "type": "record", - "name": _gen_name("__struct_"), - "fields": fields, - "native_data_type": "struct<{}>".format(parts), - "_nullable": nullable, - } - - -def _parse_basic_datatype( - type: IcebergTypes.PrimitiveType, nullable: bool -) -> Dict[str, Any]: - """ - See https://iceberg.apache.org/#spec/#avro - """ - # Check for an atomic types. - for iceberg_type in _all_atomic_types.keys(): - if isinstance(type, iceberg_type): - return { - "type": _all_atomic_types[iceberg_type], - "native_data_type": repr(type), - "_nullable": nullable, - } - - # Fixed is a special case where it is not an atomic type and not a logical type. - if isinstance(type, IcebergTypes.FixedType): - fixed_type: IcebergTypes.FixedType = type + + def visit_fixed(self, fixed_type: FixedType) -> Dict[str, Any]: return { "type": "fixed", - "name": _gen_name("__fixed_"), - "size": fixed_type.length, - "native_data_type": repr(fixed_type), - "_nullable": nullable, + "name": self._gen_name("__fixed_"), + "size": len(fixed_type), + "native_data_type": str(fixed_type), } - # Not an atomic type, so check for a logical type. - if isinstance(type, IcebergTypes.DecimalType): + def visit_decimal(self, decimal_type: DecimalType) -> Dict[str, Any]: # Also of interest: https://avro.apache.org/docs/current/spec.html#Decimal - decimal_type: IcebergTypes.DecimalType = type return { # "type": "bytes", # when using bytes, avro drops _nullable attribute and others. See unit test. "type": "fixed", # to fix avro bug ^ resolved by using a fixed type - "name": _gen_name( + "name": self._gen_name( "__fixed_" ), # to fix avro bug ^ resolved by using a fixed type "size": 1, # to fix avro bug ^ resolved by using a fixed type "logicalType": "decimal", "precision": decimal_type.precision, "scale": decimal_type.scale, - "native_data_type": repr(decimal_type), - "_nullable": nullable, + "native_data_type": str(decimal_type), } - elif isinstance(type, IcebergTypes.UUIDType): - uuid_type: IcebergTypes.UUIDType = type + + def visit_boolean(self, boolean_type: BooleanType) -> Dict[str, Any]: return { - "type": "string", - "logicalType": "uuid", - "native_data_type": repr(uuid_type), - "_nullable": nullable, + "type": "boolean", + "native_data_type": str(boolean_type), } - elif isinstance(type, IcebergTypes.DateType): - date_type: IcebergTypes.DateType = type + + def visit_integer(self, integer_type: IntegerType) -> Dict[str, Any]: + return { + "type": "int", + "native_data_type": str(integer_type), + } + + def visit_long(self, long_type: LongType) -> Dict[str, Any]: + return { + "type": "long", + "native_data_type": str(long_type), + } + + def visit_float(self, float_type: FloatType) -> Dict[str, Any]: + return { + "type": "float", + "native_data_type": str(float_type), + } + + def visit_double(self, double_type: DoubleType) -> Dict[str, Any]: + return { + "type": "double", + "native_data_type": str(double_type), + } + + def visit_date(self, date_type: DateType) -> Dict[str, Any]: return { "type": "int", "logicalType": "date", - "native_data_type": repr(date_type), - "_nullable": nullable, + "native_data_type": str(date_type), } - elif isinstance(type, IcebergTypes.TimeType): - time_type: IcebergTypes.TimeType = type + + def visit_time(self, time_type: TimeType) -> Dict[str, Any]: return { "type": "long", "logicalType": "time-micros", - "native_data_type": repr(time_type), - "_nullable": nullable, + "native_data_type": str(time_type), } - elif isinstance(type, IcebergTypes.TimestampType): - timestamp_type: IcebergTypes.TimestampType = type + + def visit_timestamp(self, timestamp_type: TimestampType) -> Dict[str, Any]: # Avro supports 2 types of timestamp: # - Timestamp: independent of a particular timezone or calendar (TZ information is lost) # - Local Timestamp: represents a timestamp in a local timezone, regardless of what specific time zone is considered local @@ -468,12 +492,40 @@ def _parse_basic_datatype( # "logicalType": "timestamp-micros" # if timestamp_type.adjust_to_utc # else "local-timestamp-micros", - "native_data_type": repr(timestamp_type), - "_nullable": nullable, + "native_data_type": str(timestamp_type), } - return {"type": "null", "native_data_type": repr(type)} + def visit_timestampz(self, timestamptz_type: TimestamptzType) -> Dict[str, Any]: + # Avro supports 2 types of timestamp: + # - Timestamp: independent of a particular timezone or calendar (TZ information is lost) + # - Local Timestamp: represents a timestamp in a local timezone, regardless of what specific time zone is considered local + # utcAdjustment: bool = True + return { + "type": "long", + "logicalType": "timestamp-micros", + # Commented out since Avro's Python implementation (1.11.0) does not support local-timestamp-micros, even though it exists in the spec. + # See bug report: https://issues.apache.org/jira/browse/AVRO-3476 and PR https://github.com/apache/avro/pull/1634 + # "logicalType": "timestamp-micros" + # if timestamp_type.adjust_to_utc + # else "local-timestamp-micros", + "native_data_type": str(timestamptz_type), + } + def visit_string(self, string_type: StringType) -> Dict[str, Any]: + return { + "type": "string", + "native_data_type": str(string_type), + } + + def visit_uuid(self, uuid_type: UUIDType) -> Dict[str, Any]: + return { + "type": "string", + "logicalType": "uuid", + "native_data_type": str(uuid_type), + } -def _gen_name(prefix: str) -> str: - return f"{prefix}{str(uuid.uuid4()).replace('-', '')}" + def visit_binary(self, binary_type: BinaryType) -> Dict[str, Any]: + return { + "type": "bytes", + "native_data_type": str(binary_type), + } diff --git a/metadata-ingestion/src/datahub/ingestion/source/iceberg/iceberg_common.py b/metadata-ingestion/src/datahub/ingestion/source/iceberg/iceberg_common.py index 5ca5bcce11d4f..54b616e5e6abf 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/iceberg/iceberg_common.py +++ b/metadata-ingestion/src/datahub/ingestion/source/iceberg/iceberg_common.py @@ -1,12 +1,14 @@ -import os from dataclasses import dataclass, field -from typing import Dict, Iterable, List, Optional, Tuple +from typing import Callable, Dict, Iterable, List, Optional, Tuple import pydantic -from azure.storage.filedatalake import FileSystemClient, PathProperties -from iceberg.core.filesystem.abfss_filesystem import AbfssFileSystem -from iceberg.core.filesystem.filesystem_tables import FilesystemTables +from fsspec import AbstractFileSystem, filesystem from pydantic import Field, root_validator +from pyiceberg.catalog import Catalog, load_catalog +from pyiceberg.exceptions import NoSuchIcebergTableError +from pyiceberg.io import FileIO, load_file_io +from pyiceberg.serializers import FromInputFile +from pyiceberg.table import Table from datahub.configuration.common import ( AllowDenyPattern, @@ -22,6 +24,7 @@ StatefulIngestionConfigBase, ) + class IcebergProfilingConfig(ConfigModel): enabled: bool = Field( default=False, @@ -49,6 +52,27 @@ class IcebergProfilingConfig(ConfigModel): # include_field_sample_values: bool = True +class IcebergSourceStatefulIngestionConfig(StatefulStaleMetadataRemovalConfig): + """Iceberg custom stateful ingestion config definition(overrides _entity_types of StatefulStaleMetadataRemovalConfig).""" + + _entity_types: List[str] = pydantic.Field(default=["table"]) + + +class IcebergCatalogConfig(ConfigModel): + """ + Iceberg catalog config. + + https://py.iceberg.apache.org/configuration/ + """ + + name: str = Field( + description="Name of catalog", + ) + conf: Dict[str, str] = Field( + description="Catalog specific configuration. See [PyIceberg documentation](https://py.iceberg.apache.org/configuration/) for details.", + ) + + class IcebergSourceConfig(StatefulIngestionConfigBase): # Override the stateful_ingestion config param with the Iceberg custom stateful ingestion config in the IcebergSourceConfig stateful_ingestion: Optional[StatefulStaleMetadataRemovalConfig] = pydantic.Field( @@ -62,6 +86,10 @@ class IcebergSourceConfig(StatefulIngestionConfigBase): default=None, description="Local path to crawl for Iceberg tables. This is one filesystem type supported by this source and **only one can be configured**.", ) + catalog: Optional[IcebergCatalogConfig] = Field( + default=None, + description="Catalog configuration where to find Iceberg tables.", + ) max_path_depth: int = Field( default=2, description="Maximum folder depth to crawl for Iceberg tables. Folders deeper than this value will be silently ignored.", @@ -97,71 +125,91 @@ def validate_platform_instance(cls: "IcebergSourceConfig", values: Dict) -> Dict def _ensure_one_filesystem_is_configured( cls: "IcebergSourceConfig", values: Dict ) -> Dict: - if values.get("adls") and values.get("localfs"): + count = sum( + [ + 1 + for x in [ + values.get("catalog"), + values.get("adls"), + values.get("localfs"), + ] + if x is not None + ] + ) + if count == 0: raise ConfigurationError( - "Only one filesystem can be configured: adls or localfs" + "One filesystem (catalog or adls or localfs) needs to be configured." ) - elif not values.get("adls") and not values.get("localfs"): + elif count > 1: raise ConfigurationError( - "One filesystem (adls or localfs) needs to be configured." + "Only one filesystem can be configured: catalog or adls or localfs" ) return values - @property - def adls_filesystem_client(self) -> FileSystemClient: - """Azure Filesystem client if configured. + def load_table(self, table_name: str, table_location: str) -> Table: + """Now that Iceberg catalog support has been added to this source, this method can be removed when we migrate away from HadoopCatalog. + + Args: + table_name (str): Name of the Iceberg table + table_location (str): Location of Iceberg table Raises: - ConfigurationError: If ADLS is not configured. + NoSuchIcebergTableError: If an Iceberg table could not be loaded from the specified location Returns: - FileSystemClient: Azure Filesystem client instance to access storage account files and folders. + Table: An Iceberg table instance """ - if self.adls: # TODO Use local imports for abfss - AbfssFileSystem.get_instance().set_conf(self.adls.dict()) - return self.adls.get_filesystem_client() - raise ConfigurationError("No ADLS filesystem client configured") - - @property - def filesystem_tables(self) -> FilesystemTables: - """Iceberg FilesystemTables abstraction to access tables on a filesystem. - Currently supporting ADLS (Azure Storage Account) and local filesystem. + table_location = ( + self.adls.get_abfss_url(table_location) if self.adls else table_location + ) + io = load_file_io( + properties={**vars(self.adls)} if self.adls else {}, + location=table_location, + ) + try: + table_version = self._read_version_hint(table_location, io) + metadata_location = ( + f"{table_location}/metadata/v{table_version}.metadata.json" + ) + metadata_file = io.new_input(metadata_location) + metadata = FromInputFile.table_metadata(metadata_file) + return Table( + identifier=table_name, + metadata=metadata, + metadata_location=metadata_location, + io=io, + ) + except FileNotFoundError as e: + raise NoSuchIcebergTableError() from e - Raises: - ConfigurationError: If no filesystem was configured. + # Temporary until we migrate away from HadoopCatalog (or pyiceberg implements https://github.com/apache/iceberg/issues/6430). + def _read_version_hint(self, location: str, io: FileIO) -> int: + version_hint_file = io.new_input(f"{location}/metadata/version-hint.text") - Returns: - FilesystemTables: An Iceberg FilesystemTables abstraction instance to access tables on a filesystem - """ - if self.adls: - return FilesystemTables(self.adls.dict()) - elif self.localfs: - return FilesystemTables() - raise ConfigurationError("No filesystem client configured") + if not version_hint_file.exists(): + raise FileNotFoundError() + else: + with version_hint_file.open() as f: + return int(f.read()) - def _get_adls_paths(self, root_path: str, depth: int) -> Iterable[Tuple[str, str]]: - if self.adls and depth < self.max_path_depth: - sub_paths = self.adls_filesystem_client.get_paths( - path=root_path, recursive=False - ) - sub_path: PathProperties - for sub_path in sub_paths: - if sub_path.is_directory: + def _get_paths( + self, + fs: AbstractFileSystem, + root_path: str, + path: str, + depth: int, + fix_path: Callable[[str], str] = lambda path: path, + ) -> Iterable[Tuple[str, str]]: + if depth < self.max_path_depth: + for sub_path in fs.ls(path, detail=True): + if sub_path["type"] == "directory": dataset_name = ".".join( - sub_path.name[len(self.adls.base_path) + 1 :].split("/") + s for s in strltrim(sub_path["name"], root_path).split("/") if s + ) + yield fix_path(sub_path["name"]), dataset_name + yield from self._get_paths( + fs, root_path, sub_path["name"], depth + 1 ) - yield self.adls.get_abfss_url(sub_path.name), dataset_name - yield from self._get_adls_paths(sub_path.name, depth + 1) - - def _get_localfs_paths( - self, root_path: str, depth: int - ) -> Iterable[Tuple[str, str]]: - if self.localfs and depth < self.max_path_depth: - for f in os.scandir(root_path): - if f.is_dir(): - dataset_name = ".".join(f.path[len(self.localfs) + 1 :].split("/")) - yield f.path, dataset_name - yield from self._get_localfs_paths(f.path, depth + 1) def get_paths(self) -> Iterable[Tuple[str, str]]: """Generates a sequence of data paths and dataset names. @@ -174,12 +222,32 @@ def get_paths(self) -> Iterable[Tuple[str, str]]: and the second item is the associated dataset name. """ if self.adls: - yield from self._get_adls_paths(self.adls.base_path, 0) + yield from self._get_paths( + filesystem("abfs", **vars(self.adls)), + f"{self.adls.container_name}/{self.adls.base_path}", + f"{self.adls.container_name}/{self.adls.base_path}", + 0, + self.adls.get_abfss_url, + ) elif self.localfs: - yield from self._get_localfs_paths(self.localfs, 0) + yield from self._get_paths( + filesystem("file"), self.localfs, self.localfs, 0 + ) else: raise ConfigurationError("No filesystem client configured") + def get_catalog(self) -> Catalog: + """Returns the Iceberg catalog instance as configured by the `catalog` dictionary. + + Returns: + Catalog: Iceberg catalog instance, `None` is not configured. + """ + return ( + load_catalog(name=self.catalog.name, **self.catalog.conf) + if self.catalog + else None + ) + @dataclass class IcebergSourceReport(StaleEntityRemovalSourceReport): @@ -195,3 +263,7 @@ def report_dropped(self, ent_name: str) -> None: def report_entity_profiled(self, name: str) -> None: self.entities_profiled += 1 + + +def strltrim(to_trim: str, prefix: str) -> str: + return to_trim[len(prefix) :] if to_trim.startswith(prefix) else to_trim diff --git a/metadata-ingestion/src/datahub/ingestion/source/iceberg/iceberg_profiler.py b/metadata-ingestion/src/datahub/ingestion/source/iceberg/iceberg_profiler.py index 766477e3a1cd8..9697901cb65c3 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/iceberg/iceberg_profiler.py +++ b/metadata-ingestion/src/datahub/ingestion/source/iceberg/iceberg_profiler.py @@ -1,17 +1,21 @@ from datetime import datetime, timedelta from typing import Any, Callable, Dict, Iterable, Union, cast -from iceberg.api import types as IcebergTypes -from iceberg.api.data_file import DataFile -from iceberg.api.manifest_file import ManifestFile -from iceberg.api.schema import Schema -from iceberg.api.snapshot import Snapshot -from iceberg.api.table import Table -from iceberg.api.types import Conversions, NestedField, Type, TypeID -from iceberg.core.base_table import BaseTable -from iceberg.core.filesystem import FileSystemInputFile -from iceberg.core.manifest_reader import ManifestReader -from iceberg.exceptions.exceptions import FileSystemNotFound +from pyiceberg.conversions import from_bytes +from pyiceberg.schema import Schema +from pyiceberg.table import Table +from pyiceberg.types import ( + DateType, + DecimalType, + DoubleType, + FloatType, + IcebergType, + IntegerType, + LongType, + TimestampType, + TimestamptzType, + TimeType, +) from datahub.emitter.mce_builder import get_sys_time from datahub.emitter.mcp import MetadataChangeProposalWrapper @@ -54,11 +58,14 @@ def _aggregate_bounds( manifest_values: Dict[int, Any], ) -> None: for field_id, value_encoded in manifest_values.items(): # type: int, Any - field: NestedField = schema.find_field(field_id) - # Bounds in manifests can reference historical field IDs that are not part of the current schema. - # We simply not profile those since we only care about the current snapshot. - if field and IcebergProfiler._is_numeric_type(field.type): - value_decoded = Conversions.from_byte_buffer(field.type, value_encoded) + try: + field = schema.find_field(field_id) + except ValueError: + # Bounds in manifests can reference historical field IDs that are not part of the current schema. + # We simply not profile those since we only care about the current snapshot. + continue + if IcebergProfiler._is_numeric_type(field.field_type): + value_decoded = from_bytes(field.field_type, value_encoded) if value_decoded: agg_value = aggregated_values.get(field_id) aggregated_values[field_id] = ( @@ -97,12 +104,23 @@ def profile_table( Yields: Iterator[Iterable[MetadataWorkUnit]]: Workunits related to datasetProfile. """ - if not table.snapshots() or not isinstance(table, BaseTable): + current_snapshot = table.current_snapshot() + if not current_snapshot: # Table has no data, cannot profile, or we can't get current_snapshot. return - row_count: int = int(table.current_snapshot().summary["total-records"]) - column_count: int = len(table.schema()._id_to_name) + row_count = ( + int(current_snapshot.summary.additional_properties["total-records"]) + if current_snapshot.summary + else 0 + ) + column_count = len( + [ + id + for id in table.schema().field_ids + if table.schema().find_field(id).field_type.is_primitive + ] + ) dataset_profile = DatasetProfileClass( timestampMillis=get_sys_time(), rowCount=row_count, @@ -110,47 +128,36 @@ def profile_table( ) dataset_profile.fieldProfiles = [] - field_paths: Dict[int, str] = table.schema()._id_to_name - current_snapshot: Snapshot = table.current_snapshot() - total_count: int = 0 + total_count = 0 null_counts: Dict[int, int] = {} min_bounds: Dict[int, Any] = {} max_bounds: Dict[int, Any] = {} - manifest: ManifestFile - try: - for manifest in current_snapshot.manifests: - manifest_input_file = FileSystemInputFile.from_location( - manifest.manifest_path, table.ops.conf - ) - manifest_reader = ManifestReader.read(manifest_input_file) - data_file: DataFile - for data_file in manifest_reader.iterator(): - if self.config.include_field_null_count: - null_counts = self._aggregate_counts( - null_counts, data_file.null_value_counts() - ) - if self.config.include_field_min_value: - self._aggregate_bounds( - table.schema(), - min, - min_bounds, - data_file.lower_bounds(), - ) - if self.config.include_field_max_value: - self._aggregate_bounds( - table.schema(), - max, - max_bounds, - data_file.upper_bounds(), - ) - total_count += data_file.record_count() - # TODO Work on error handling to provide better feedback. Iceberg exceptions are weak... - except FileSystemNotFound as e: - raise Exception("Error loading table manifests") from e + for manifest in current_snapshot.manifests(table.io): + for manifest_entry in manifest.fetch_manifest_entry(table.io): + data_file = manifest_entry.data_file + if self.config.include_field_null_count: + null_counts = self._aggregate_counts( + null_counts, data_file.null_value_counts + ) + if self.config.include_field_min_value: + self._aggregate_bounds( + table.schema(), + min, + min_bounds, + data_file.lower_bounds, + ) + if self.config.include_field_max_value: + self._aggregate_bounds( + table.schema(), + max, + max_bounds, + data_file.upper_bounds, + ) + total_count += data_file.record_count if row_count: # Iterating through fieldPaths introduces unwanted stats for list element fields... - for field_id, field_path in field_paths.items(): - field: NestedField = table.schema().find_field(field_id) + for field_path, field_id in table.schema()._name_to_id.items(): + field = table.schema().find_field(field_id) column_profile = DatasetFieldProfileClass(fieldPath=field_path) if self.config.include_field_null_count: column_profile.nullCount = cast(int, null_counts.get(field_id, 0)) @@ -161,7 +168,7 @@ def profile_table( if self.config.include_field_min_value: column_profile.min = ( self._renderValue( - dataset_name, field.type, min_bounds.get(field_id) + dataset_name, field.field_type, min_bounds.get(field_id) ) if field_id in min_bounds else None @@ -169,7 +176,7 @@ def profile_table( if self.config.include_field_max_value: column_profile.max = ( self._renderValue( - dataset_name, field.type, max_bounds.get(field_id) + dataset_name, field.field_type, max_bounds.get(field_id) ) if field_id in max_bounds else None @@ -185,21 +192,21 @@ def profile_table( self.report.report_entity_profiled(dataset_name) yield wu - # The following will eventually be done by the Iceberg API (in the new Python refactored API). def _renderValue( - self, dataset_name: str, value_type: Type, value: Any + self, dataset_name: str, value_type: IcebergType, value: Any ) -> Union[str, None]: try: - if value_type.type_id == TypeID.TIMESTAMP: - if value_type.adjust_to_utc: - # TODO Deal with utc when required - microsecond_unix_ts = value - else: - microsecond_unix_ts = value + if isinstance(value_type, (TimestampType, TimestamptzType)): + # if value_type.adjust_to_utc: + # # TODO Deal with utc when required + # microsecond_unix_ts = value + # else: + # microsecond_unix_ts = value + microsecond_unix_ts = value return datetime.fromtimestamp(microsecond_unix_ts / 1000000.0).strftime( "%Y-%m-%d %H:%M:%S" ) - elif value_type.type_id == TypeID.DATE: + elif isinstance(value_type, DateType): return (datetime(1970, 1, 1, 0, 0) + timedelta(value - 1)).strftime( "%Y-%m-%d" ) @@ -212,17 +219,18 @@ def _renderValue( return None @staticmethod - def _is_numeric_type(type: Type) -> bool: + def _is_numeric_type(type: IcebergType) -> bool: return isinstance( type, ( - IcebergTypes.DateType, - IcebergTypes.DecimalType, - IcebergTypes.DoubleType, - IcebergTypes.FloatType, - IcebergTypes.IntegerType, - IcebergTypes.LongType, - IcebergTypes.TimestampType, - IcebergTypes.TimeType, + DateType, + DecimalType, + DoubleType, + FloatType, + IntegerType, + LongType, + TimestampType, + TimestamptzType, + TimeType, ), ) diff --git a/metadata-ingestion/tests/integration/iceberg/.gitignore b/metadata-ingestion/tests/integration/iceberg/.gitignore new file mode 100644 index 0000000000000..a7dfcf56788b4 --- /dev/null +++ b/metadata-ingestion/tests/integration/iceberg/.gitignore @@ -0,0 +1,3 @@ +# Folders created by Iceberg's docker-compose +notebooks/ +warehouse/ \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/iceberg/docker-compose.yml b/metadata-ingestion/tests/integration/iceberg/docker-compose.yml new file mode 100644 index 0000000000000..166dd495ab212 --- /dev/null +++ b/metadata-ingestion/tests/integration/iceberg/docker-compose.yml @@ -0,0 +1,63 @@ +version: "3" + +services: + spark-iceberg: + image: tabulario/spark-iceberg + container_name: spark-iceberg + depends_on: + - rest + - minio + volumes: + - ./warehouse:/home/iceberg/warehouse + - ./notebooks:/home/iceberg/notebooks/notebooks + - ./setup:/home/iceberg/setup + environment: + - AWS_ACCESS_KEY_ID=admin + - AWS_SECRET_ACCESS_KEY=password + - AWS_REGION=us-east-1 + ports: + - 8888:8888 + - 8080:8080 + - 10000:10000 + - 10001:10001 + links: + - rest:rest + - minio:minio + rest: + image: tabulario/iceberg-rest:0.2.0 + ports: + - 8181:8181 + environment: + - AWS_ACCESS_KEY_ID=admin + - AWS_SECRET_ACCESS_KEY=password + - AWS_REGION=us-east-1 + - CATALOG_WAREHOUSE=s3a://warehouse/wh/ + - CATALOG_IO__IMPL=org.apache.iceberg.aws.s3.S3FileIO + - CATALOG_S3_ENDPOINT=http://minio:9000 + minio: + image: minio/minio + container_name: minio + environment: + - MINIO_ROOT_USER=admin + - MINIO_ROOT_PASSWORD=password + ports: + - 9001:9001 + - 9000:9000 + command: ["server", "/data", "--console-address", ":9001"] + mc: + depends_on: + - minio + image: minio/mc + container_name: mc + environment: + - AWS_ACCESS_KEY_ID=admin + - AWS_SECRET_ACCESS_KEY=password + - AWS_REGION=us-east-1 + entrypoint: > + /bin/sh -c " + until (/usr/bin/mc config host add minio http://minio:9000 admin password) do echo '...waiting...' && sleep 1; done; + /usr/bin/mc rm -r --force minio/warehouse; + /usr/bin/mc mb minio/warehouse; + /usr/bin/mc policy set public minio/warehouse; + exit 0; + " diff --git a/metadata-ingestion/tests/integration/iceberg/iceberg_deleted_table_mces_golden.json b/metadata-ingestion/tests/integration/iceberg/iceberg_deleted_table_mces_golden.json new file mode 100644 index 0000000000000..487a6f0c0a83d --- /dev/null +++ b/metadata-ingestion/tests/integration/iceberg/iceberg_deleted_table_mces_golden.json @@ -0,0 +1,16 @@ +[ + { + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:iceberg,test_platform_instance.nyc.taxis,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "value": "{\"removed\": true}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1586847600000, + "runId": "iceberg-2020_04_14-07_00_00" + } + } +] \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/iceberg/iceberg_ingest_mces_golden.json b/metadata-ingestion/tests/integration/iceberg/iceberg_ingest_mces_golden.json new file mode 100644 index 0000000000000..e4bc75712ae62 --- /dev/null +++ b/metadata-ingestion/tests/integration/iceberg/iceberg_ingest_mces_golden.json @@ -0,0 +1,322 @@ +[ + { + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:iceberg,nyc.taxis,PROD)", + "aspects": [ + { + "com.linkedin.pegasus2avro.common.Status": { + "removed": false + } + }, + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": { + "owner": "root", + "created-at": "2023-01-09T16:10:37.115997100Z", + "write.format.default": "parquet", + "location": "s3a://warehouse/wh/nyc/taxis", + "format-version": "1", + "snapshot-id": "1023498847080782840", + "manifest-list": "s3a://warehouse/wh/nyc/taxis/metadata/snap-1023498847080782840-1-cd5fb158-a88b-4e94-92de-cbbccd2ac705.avro" + }, + "tags": [] + } + }, + { + "com.linkedin.pegasus2avro.common.Ownership": { + "owners": [ + { + "owner": "urn:li:corpuser:root", + "type": "TECHNICAL_OWNER" + }, + { + "owner": "urn:li:corpGroup:root", + "type": "TECHNICAL_OWNER" + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + } + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "nyc.taxis", + "platform": "urn:li:dataPlatform:iceberg", + "version": 0, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "hash": "", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.OtherSchema": { + "rawSchema": "table {\n 1: VendorID: optional long\n 2: tpep_pickup_datetime: optional timestamptz\n 3: tpep_dropoff_datetime: optional timestamptz\n 4: passenger_count: optional double\n 5: trip_distance: optional double\n 6: RatecodeID: optional double\n 7: store_and_fwd_flag: optional string\n 8: PULocationID: optional long\n 9: DOLocationID: optional long\n 10: payment_type: optional long\n 11: fare_amount: optional double\n 12: extra: optional double\n 13: mta_tax: optional double\n 14: tip_amount: optional double\n 15: tolls_amount: optional double\n 16: improvement_surcharge: optional double\n 17: total_amount: optional double\n 18: congestion_surcharge: optional double\n 19: airport_fee: optional double\n}" + } + }, + "fields": [ + { + "fieldPath": "[version=2.0].[type=struct].[type=long].VendorID", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "long", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"long\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=long].tpep_pickup_datetime", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.TimeType": {} + } + }, + "nativeDataType": "timestamptz", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"logicalType\": \"timestamp-micros\", \"native_data_type\": \"timestamptz\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=long].tpep_dropoff_datetime", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.TimeType": {} + } + }, + "nativeDataType": "timestamptz", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"logicalType\": \"timestamp-micros\", \"native_data_type\": \"timestamptz\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=double].passenger_count", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "double", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"double\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=double].trip_distance", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "double", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"double\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=double].RatecodeID", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "double", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"double\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=string].store_and_fwd_flag", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "string", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"string\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=long].PULocationID", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "long", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"long\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=long].DOLocationID", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "long", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"long\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=long].payment_type", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "long", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"long\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=double].fare_amount", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "double", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"double\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=double].extra", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "double", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"double\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=double].mta_tax", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "double", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"double\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=double].tip_amount", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "double", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"double\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=double].tolls_amount", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "double", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"double\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=double].improvement_surcharge", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "double", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"double\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=double].total_amount", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "double", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"double\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=double].congestion_surcharge", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "double", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"double\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=double].airport_fee", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "double", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"double\", \"_nullable\": true}" + } + ] + } + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1586847600000, + "runId": "iceberg-test" + } + } +] \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/iceberg/iceberg_profile_mces_golden.json b/metadata-ingestion/tests/integration/iceberg/iceberg_profile_mces_golden.json new file mode 100644 index 0000000000000..8a72e443470bc --- /dev/null +++ b/metadata-ingestion/tests/integration/iceberg/iceberg_profile_mces_golden.json @@ -0,0 +1,336 @@ +[ + { + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:iceberg,nyc.taxis,PROD)", + "aspects": [ + { + "com.linkedin.pegasus2avro.common.Status": { + "removed": false + } + }, + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": { + "owner": "root", + "created-at": "2023-01-13T11:56:49.169944100Z", + "write.format.default": "parquet", + "location": "s3a://warehouse/wh/nyc/taxis", + "format-version": "1", + "snapshot-id": "6917251075794782448", + "manifest-list": "s3a://warehouse/wh/nyc/taxis/metadata/snap-6917251075794782448-1-dc6938ef-2471-4b9c-aa18-4d7d7c200607.avro" + }, + "tags": [] + } + }, + { + "com.linkedin.pegasus2avro.common.Ownership": { + "owners": [ + { + "owner": "urn:li:corpuser:root", + "type": "TECHNICAL_OWNER" + }, + { + "owner": "urn:li:corpGroup:root", + "type": "TECHNICAL_OWNER" + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + } + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "nyc.taxis", + "platform": "urn:li:dataPlatform:iceberg", + "version": 0, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "hash": "", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.OtherSchema": { + "rawSchema": "table {\n 1: VendorID: optional long\n 2: tpep_pickup_datetime: optional timestamptz\n 3: tpep_dropoff_datetime: optional timestamptz\n 4: passenger_count: optional double\n 5: trip_distance: optional double\n 6: RatecodeID: optional double\n 7: store_and_fwd_flag: optional string\n 8: PULocationID: optional long\n 9: DOLocationID: optional long\n 10: payment_type: optional long\n 11: fare_amount: optional double\n 12: extra: optional double\n 13: mta_tax: optional double\n 14: tip_amount: optional double\n 15: tolls_amount: optional double\n 16: improvement_surcharge: optional double\n 17: total_amount: optional double\n 18: congestion_surcharge: optional double\n 19: airport_fee: optional double\n}" + } + }, + "fields": [ + { + "fieldPath": "[version=2.0].[type=struct].[type=long].VendorID", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "long", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"long\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=long].tpep_pickup_datetime", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.TimeType": {} + } + }, + "nativeDataType": "timestamptz", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"logicalType\": \"timestamp-micros\", \"native_data_type\": \"timestamptz\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=long].tpep_dropoff_datetime", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.TimeType": {} + } + }, + "nativeDataType": "timestamptz", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"logicalType\": \"timestamp-micros\", \"native_data_type\": \"timestamptz\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=double].passenger_count", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "double", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"double\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=double].trip_distance", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "double", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"double\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=double].RatecodeID", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "double", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"double\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=string].store_and_fwd_flag", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "string", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"string\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=long].PULocationID", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "long", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"long\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=long].DOLocationID", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "long", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"long\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=long].payment_type", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "long", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"long\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=double].fare_amount", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "double", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"double\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=double].extra", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "double", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"double\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=double].mta_tax", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "double", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"double\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=double].tip_amount", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "double", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"double\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=double].tolls_amount", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "double", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"double\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=double].improvement_surcharge", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "double", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"double\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=double].total_amount", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "double", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"double\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=double].congestion_surcharge", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "double", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"double\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=double].airport_fee", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "double", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"double\", \"_nullable\": true}" + } + ] + } + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1586847600000, + "runId": "iceberg-test" + } + }, + { + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:iceberg,nyc.taxis,PROD)", + "changeType": "UPSERT", + "aspectName": "datasetProfile", + "aspect": { + "value": "{\"timestampMillis\": 1586847600000, \"partitionSpec\": {\"type\": \"FULL_TABLE\", \"partition\": \"FULL_TABLE_SNAPSHOT\"}, \"rowCount\": 2171187, \"columnCount\": 19, \"fieldProfiles\": [{\"fieldPath\": \"VendorID\", \"nullCount\": 0, \"nullProportion\": 0.0, \"min\": \"1\", \"max\": \"6\"}, {\"fieldPath\": \"tpep_pickup_datetime\", \"nullCount\": 0, \"nullProportion\": 0.0, \"min\": \"2009-01-01 03:21:35\", \"max\": \"2021-07-19 05:22:03\"}, {\"fieldPath\": \"tpep_dropoff_datetime\", \"nullCount\": 0, \"nullProportion\": 0.0, \"min\": \"2009-01-01 03:51:24\", \"max\": \"2021-07-19 05:38:54\"}, {\"fieldPath\": \"passenger_count\", \"nullCount\": 128020, \"nullProportion\": 0.05896313859653728, \"max\": \"9.0\"}, {\"fieldPath\": \"trip_distance\", \"nullCount\": 0, \"nullProportion\": 0.0, \"max\": \"280567.84\"}, {\"fieldPath\": \"RatecodeID\", \"nullCount\": 128020, \"nullProportion\": 0.05896313859653728, \"min\": \"1.0\", \"max\": \"99.0\"}, {\"fieldPath\": \"store_and_fwd_flag\", \"nullCount\": 128020, \"nullProportion\": 0.05896313859653728}, {\"fieldPath\": \"PULocationID\", \"nullCount\": 0, \"nullProportion\": 0.0, \"min\": \"1\", \"max\": \"265\"}, {\"fieldPath\": \"DOLocationID\", \"nullCount\": 0, \"nullProportion\": 0.0, \"min\": \"1\", \"max\": \"265\"}, {\"fieldPath\": \"payment_type\", \"nullCount\": 0, \"nullProportion\": 0.0, \"max\": \"4\"}, {\"fieldPath\": \"fare_amount\", \"nullCount\": 0, \"nullProportion\": 0.0, \"min\": \"-415.0\", \"max\": \"395854.44\"}, {\"fieldPath\": \"extra\", \"nullCount\": 0, \"nullProportion\": 0.0, \"min\": \"-5.5\", \"max\": \"8.25\"}, {\"fieldPath\": \"mta_tax\", \"nullCount\": 0, \"nullProportion\": 0.0, \"min\": \"-0.5\", \"max\": \"3.3\"}, {\"fieldPath\": \"tip_amount\", \"nullCount\": 0, \"nullProportion\": 0.0, \"min\": \"-333.32\", \"max\": \"380.0\"}, {\"fieldPath\": \"tolls_amount\", \"nullCount\": 0, \"nullProportion\": 0.0, \"min\": \"-31.75\", \"max\": \"137.25\"}, {\"fieldPath\": \"improvement_surcharge\", \"nullCount\": 0, \"nullProportion\": 0.0, \"min\": \"-0.3\", \"max\": \"0.3\"}, {\"fieldPath\": \"total_amount\", \"nullCount\": 0, \"nullProportion\": 0.0, \"min\": \"-415.3\", \"max\": \"395854.74\"}, {\"fieldPath\": \"congestion_surcharge\", \"nullCount\": 128020, \"nullProportion\": 0.05896313859653728, \"min\": \"-2.5\", \"max\": \"2.5\"}, {\"fieldPath\": \"airport_fee\", \"nullCount\": 128096, \"nullProportion\": 0.058998142490720516, \"min\": \"-1.25\", \"max\": \"1.25\"}]}", + "contentType": "application/json" + }, + "systemMetadata": { + "lastObserved": 1586847600000, + "runId": "iceberg-test" + } + } +] \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/iceberg/iceberg_profile_to_file.yml b/metadata-ingestion/tests/integration/iceberg/iceberg_profile_to_file.yml new file mode 100644 index 0000000000000..e8fe15fa51d5d --- /dev/null +++ b/metadata-ingestion/tests/integration/iceberg/iceberg_profile_to_file.yml @@ -0,0 +1,24 @@ +run_id: iceberg-test + +source: + type: iceberg + config: + catalog: + name: default + conf: + uri: http://localhost:8181 + s3.access-key-id: admin + s3.secret-access-key: password + s3.region: us-east-1 + warehouse: s3a://warehouse/wh/ + py-io-impl: pyiceberg.io.pyarrow.PyArrowFileIO + s3.endpoint: http://localhost:9000 + user_ownership_property: owner + group_ownership_property: owner + profiling: + enabled: true + +sink: + type: file + config: + filename: "./iceberg_mces.json" diff --git a/metadata-ingestion/tests/integration/iceberg/iceberg_to_file.yml b/metadata-ingestion/tests/integration/iceberg/iceberg_to_file.yml new file mode 100644 index 0000000000000..600fc65668e42 --- /dev/null +++ b/metadata-ingestion/tests/integration/iceberg/iceberg_to_file.yml @@ -0,0 +1,21 @@ +run_id: iceberg-test + +source: + type: iceberg + config: + catalog: + name: default + conf: + uri: http://localhost:8181 + s3.access-key-id: admin + s3.secret-access-key: password + s3.region: us-east-1 + warehouse: s3a://warehouse/wh/ + s3.endpoint: http://localhost:9000 + user_ownership_property: owner + group_ownership_property: owner + +sink: + type: file + config: + filename: "./iceberg_mces.json" diff --git a/metadata-ingestion/tests/integration/iceberg/setup/create.py b/metadata-ingestion/tests/integration/iceberg/setup/create.py new file mode 100644 index 0000000000000..8f036ad1f6bd6 --- /dev/null +++ b/metadata-ingestion/tests/integration/iceberg/setup/create.py @@ -0,0 +1,6 @@ +from pyspark.sql import SparkSession + +spark = SparkSession.builder.getOrCreate() + +df = spark.read.parquet("/home/iceberg/data/yellow_tripdata_2021-04.parquet") +df.write.saveAsTable("nyc.taxis") diff --git a/metadata-ingestion/tests/integration/iceberg/setup/delete.py b/metadata-ingestion/tests/integration/iceberg/setup/delete.py new file mode 100644 index 0000000000000..b00306982f517 --- /dev/null +++ b/metadata-ingestion/tests/integration/iceberg/setup/delete.py @@ -0,0 +1,5 @@ +from pyspark.sql import SparkSession + +spark = SparkSession.builder.getOrCreate() + +spark.sql("DROP TABLE nyc.taxis PURGE") diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/ingest_test/iceberg_mces_golden.json b/metadata-ingestion/tests/integration/iceberg/test_data/ingest_test/iceberg_mces_golden.json deleted file mode 100644 index b106b91275835..0000000000000 --- a/metadata-ingestion/tests/integration/iceberg/test_data/ingest_test/iceberg_mces_golden.json +++ /dev/null @@ -1,131 +0,0 @@ -[ -{ - "proposedSnapshot": { - "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { - "urn": "urn:li:dataset:(urn:li:dataPlatform:iceberg,namespace.iceberg_test,PROD)", - "aspects": [ - { - "com.linkedin.pegasus2avro.common.Status": { - "removed": false - } - }, - { - "com.linkedin.pegasus2avro.dataset.DatasetProperties": { - "customProperties": { - "owner": "new_owner", - "provider": "ICEBERG", - "location": "/namespace/iceberg_test" - }, - "tags": [] - } - }, - { - "com.linkedin.pegasus2avro.common.Ownership": { - "owners": [ - { - "owner": "urn:li:corpuser:new_owner", - "type": "TECHNICAL_OWNER" - }, - { - "owner": "urn:li:corpGroup:new_owner", - "type": "TECHNICAL_OWNER" - } - ], - "lastModified": { - "time": 0, - "actor": "urn:li:corpuser:unknown" - } - } - }, - { - "com.linkedin.pegasus2avro.schema.SchemaMetadata": { - "schemaName": "namespace.iceberg_test", - "platform": "urn:li:dataPlatform:iceberg", - "version": 0, - "created": { - "time": 0, - "actor": "urn:li:corpuser:unknown" - }, - "lastModified": { - "time": 0, - "actor": "urn:li:corpuser:unknown" - }, - "hash": "", - "platformSchema": { - "com.linkedin.pegasus2avro.schema.OtherSchema": { - "rawSchema": "Schema(1: level: required string(level documentation),2: event_time: required timestamptz(event_time documentation),3: message: required string(message documentation),4: call_stack: optional list(call_stack documentation))" - } - }, - "fields": [ - { - "fieldPath": "[version=2.0].[type=struct].[type=string].level", - "nullable": false, - "description": "level documentation", - "type": { - "type": { - "com.linkedin.pegasus2avro.schema.StringType": {} - } - }, - "nativeDataType": "string", - "recursive": false, - "isPartOfKey": false, - "jsonProps": "{\"native_data_type\": \"string\", \"_nullable\": false}" - }, - { - "fieldPath": "[version=2.0].[type=struct].[type=long].event_time", - "nullable": false, - "description": "event_time documentation", - "type": { - "type": { - "com.linkedin.pegasus2avro.schema.TimeType": {} - } - }, - "nativeDataType": "timestamptz", - "recursive": false, - "isPartOfKey": false, - "jsonProps": "{\"logicalType\": \"timestamp-micros\", \"native_data_type\": \"timestamptz\", \"_nullable\": false}" - }, - { - "fieldPath": "[version=2.0].[type=struct].[type=string].message", - "nullable": false, - "description": "message documentation", - "type": { - "type": { - "com.linkedin.pegasus2avro.schema.StringType": {} - } - }, - "nativeDataType": "string", - "recursive": false, - "isPartOfKey": false, - "jsonProps": "{\"native_data_type\": \"string\", \"_nullable\": false}" - }, - { - "fieldPath": "[version=2.0].[type=struct].[type=array].[type=string].call_stack", - "nullable": true, - "description": "call_stack documentation", - "type": { - "type": { - "com.linkedin.pegasus2avro.schema.ArrayType": { - "nestedType": [ - "string" - ] - } - } - }, - "nativeDataType": "list", - "recursive": false, - "isPartOfKey": false, - "jsonProps": "{\"native_data_type\": \"list\", \"_nullable\": true}" - } - ] - } - } - ] - } - }, - "systemMetadata": { - "lastObserved": 1586847600000, - "runId": "iceberg-test" - } -} -] \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/ingest_test/namespace/iceberg_test/metadata/v1.metadata.json b/metadata-ingestion/tests/integration/iceberg/test_data/ingest_test/namespace/iceberg_test/metadata/v1.metadata.json deleted file mode 100644 index e4ac0b9685ddc..0000000000000 --- a/metadata-ingestion/tests/integration/iceberg/test_data/ingest_test/namespace/iceberg_test/metadata/v1.metadata.json +++ /dev/null @@ -1,105 +0,0 @@ -{ - "format-version" : 1, - "table-uuid" : "11bbe5de-5ef6-4074-80db-f041065f9862", - "location" : "/namespace/iceberg_test", - "last-updated-ms" : 1648729616724, - "last-column-id" : 5, - "schema" : { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "level", - "required" : true, - "type" : "string" - }, { - "id" : 2, - "name" : "event_time", - "required" : true, - "type" : "timestamptz" - }, { - "id" : 3, - "name" : "message", - "required" : true, - "type" : "string" - }, { - "id" : 4, - "name" : "call_stack", - "required" : false, - "type" : { - "type" : "list", - "element-id" : 5, - "element" : "string", - "element-required" : true - } - } ] - }, - "current-schema-id" : 0, - "schemas" : [ { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "level", - "required" : true, - "type" : "string" - }, { - "id" : 2, - "name" : "event_time", - "required" : true, - "type" : "timestamptz" - }, { - "id" : 3, - "name" : "message", - "required" : true, - "type" : "string" - }, { - "id" : 4, - "name" : "call_stack", - "required" : false, - "type" : { - "type" : "list", - "element-id" : 5, - "element" : "string", - "element-required" : true - } - } ] - } ], - "partition-spec" : [ { - "name" : "event_time_hour", - "transform" : "hour", - "source-id" : 2, - "field-id" : 1000 - }, { - "name" : "level", - "transform" : "identity", - "source-id" : 1, - "field-id" : 1001 - } ], - "default-spec-id" : 0, - "partition-specs" : [ { - "spec-id" : 0, - "fields" : [ { - "name" : "event_time_hour", - "transform" : "hour", - "source-id" : 2, - "field-id" : 1000 - }, { - "name" : "level", - "transform" : "identity", - "source-id" : 1, - "field-id" : 1001 - } ] - } ], - "last-partition-id" : 1001, - "default-sort-order-id" : 0, - "sort-orders" : [ { - "order-id" : 0, - "fields" : [ ] - } ], - "properties" : { }, - "current-snapshot-id" : -1, - "snapshots" : [ ], - "snapshot-log" : [ ], - "metadata-log" : [ ] -} \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/ingest_test/namespace/iceberg_test/metadata/v2.metadata.json b/metadata-ingestion/tests/integration/iceberg/test_data/ingest_test/namespace/iceberg_test/metadata/v2.metadata.json deleted file mode 100644 index 02221330b0665..0000000000000 --- a/metadata-ingestion/tests/integration/iceberg/test_data/ingest_test/namespace/iceberg_test/metadata/v2.metadata.json +++ /dev/null @@ -1,118 +0,0 @@ -{ - "format-version" : 1, - "table-uuid" : "16e6ecee-cd5d-470f-a7a6-a197944fa4db", - "location" : "/namespace/iceberg_test", - "last-updated-ms" : 1649086837695, - "last-column-id" : 5, - "schema" : { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "level", - "required" : true, - "type" : "string", - "doc" : "level documentation" - }, { - "id" : 2, - "name" : "event_time", - "required" : true, - "type" : "timestamptz", - "doc" : "event_time documentation" - }, { - "id" : 3, - "name" : "message", - "required" : true, - "type" : "string", - "doc" : "message documentation" - }, { - "id" : 4, - "name" : "call_stack", - "required" : false, - "type" : { - "type" : "list", - "element-id" : 5, - "element" : "string", - "element-required" : true - }, - "doc" : "call_stack documentation" - } ] - }, - "current-schema-id" : 0, - "schemas" : [ { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "level", - "required" : true, - "type" : "string", - "doc" : "level documentation" - }, { - "id" : 2, - "name" : "event_time", - "required" : true, - "type" : "timestamptz", - "doc" : "event_time documentation" - }, { - "id" : 3, - "name" : "message", - "required" : true, - "type" : "string", - "doc" : "message documentation" - }, { - "id" : 4, - "name" : "call_stack", - "required" : false, - "type" : { - "type" : "list", - "element-id" : 5, - "element" : "string", - "element-required" : true - }, - "doc" : "call_stack documentation" - } ] - } ], - "partition-spec" : [ { - "name" : "event_time_hour", - "transform" : "hour", - "source-id" : 2, - "field-id" : 1000 - }, { - "name" : "level", - "transform" : "identity", - "source-id" : 1, - "field-id" : 1001 - } ], - "default-spec-id" : 0, - "partition-specs" : [ { - "spec-id" : 0, - "fields" : [ { - "name" : "event_time_hour", - "transform" : "hour", - "source-id" : 2, - "field-id" : 1000 - }, { - "name" : "level", - "transform" : "identity", - "source-id" : 1, - "field-id" : 1001 - } ] - } ], - "last-partition-id" : 1001, - "default-sort-order-id" : 0, - "sort-orders" : [ { - "order-id" : 0, - "fields" : [ ] - } ], - "properties" : { - "owner" : "new_owner" - }, - "current-snapshot-id" : -1, - "snapshots" : [ ], - "snapshot-log" : [ ], - "metadata-log" : [ { - "timestamp-ms" : 1649086837511, - "metadata-file" : "/namespace/iceberg_test/metadata/v1.metadata.json" - } ] -} \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/ingest_test/namespace/iceberg_test/metadata/version-hint.text b/metadata-ingestion/tests/integration/iceberg/test_data/ingest_test/namespace/iceberg_test/metadata/version-hint.text deleted file mode 100644 index d8263ee986059..0000000000000 --- a/metadata-ingestion/tests/integration/iceberg/test_data/ingest_test/namespace/iceberg_test/metadata/version-hint.text +++ /dev/null @@ -1 +0,0 @@ -2 \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/data/00000-0-72133c37-bb5c-4ffd-8ead-08f33fa2675d-00001.parquet b/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/data/00000-0-72133c37-bb5c-4ffd-8ead-08f33fa2675d-00001.parquet deleted file mode 100644 index 48e75a030f1ca..0000000000000 Binary files a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/data/00000-0-72133c37-bb5c-4ffd-8ead-08f33fa2675d-00001.parquet and /dev/null differ diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/data/00000-3-c638dd0f-498a-4ce9-b525-8242758d18f8-00001.parquet b/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/data/00000-3-c638dd0f-498a-4ce9-b525-8242758d18f8-00001.parquet deleted file mode 100644 index c70b94612db64..0000000000000 Binary files a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/data/00000-3-c638dd0f-498a-4ce9-b525-8242758d18f8-00001.parquet and /dev/null differ diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/data/00001-1-5f69f6ed-191f-4a11-9953-09435ffce01d-00001.parquet b/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/data/00001-1-5f69f6ed-191f-4a11-9953-09435ffce01d-00001.parquet deleted file mode 100644 index 4c95fceed72e6..0000000000000 Binary files a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/data/00001-1-5f69f6ed-191f-4a11-9953-09435ffce01d-00001.parquet and /dev/null differ diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/data/00001-4-b21a5375-b547-40b9-89ca-caf4fcfe6685-00001.parquet b/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/data/00001-4-b21a5375-b547-40b9-89ca-caf4fcfe6685-00001.parquet deleted file mode 100644 index d33a3fd0d8a07..0000000000000 Binary files a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/data/00001-4-b21a5375-b547-40b9-89ca-caf4fcfe6685-00001.parquet and /dev/null differ diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/metadata/00000-331b9f67-e02b-44b1-8ec8-4dfa287c3bd5.metadata.json b/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/metadata/00000-331b9f67-e02b-44b1-8ec8-4dfa287c3bd5.metadata.json deleted file mode 100644 index b6ffcfdc55daf..0000000000000 --- a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/metadata/00000-331b9f67-e02b-44b1-8ec8-4dfa287c3bd5.metadata.json +++ /dev/null @@ -1,73 +0,0 @@ -{ - "format-version" : 1, - "table-uuid" : "e54626bf-c7ab-4f36-a3d0-3e13eec0824f", - "location" : "/home/iceberg/warehouse/datahub/integration/profiling", - "last-updated-ms" : 1651614148692, - "last-column-id" : 3, - "schema" : { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "field_int", - "required" : false, - "type" : "long", - "doc" : "An integer field" - }, { - "id" : 2, - "name" : "field_str", - "required" : false, - "type" : "string", - "doc" : "A string field" - }, { - "id" : 3, - "name" : "field_timestamp", - "required" : false, - "type" : "timestamptz", - "doc" : "A timestamp field" - } ] - }, - "current-schema-id" : 0, - "schemas" : [ { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "field_int", - "required" : false, - "type" : "long", - "doc" : "An integer field" - }, { - "id" : 2, - "name" : "field_str", - "required" : false, - "type" : "string", - "doc" : "A string field" - }, { - "id" : 3, - "name" : "field_timestamp", - "required" : false, - "type" : "timestamptz", - "doc" : "A timestamp field" - } ] - } ], - "partition-spec" : [ ], - "default-spec-id" : 0, - "partition-specs" : [ { - "spec-id" : 0, - "fields" : [ ] - } ], - "last-partition-id" : 999, - "default-sort-order-id" : 0, - "sort-orders" : [ { - "order-id" : 0, - "fields" : [ ] - } ], - "properties" : { - "owner" : "root" - }, - "current-snapshot-id" : -1, - "snapshots" : [ ], - "snapshot-log" : [ ], - "metadata-log" : [ ] -} \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/metadata/00001-fb50681e-5f25-4180-99e2-065ef0b9791b.metadata.json b/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/metadata/00001-fb50681e-5f25-4180-99e2-065ef0b9791b.metadata.json deleted file mode 100644 index da2afa6569f11..0000000000000 --- a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/metadata/00001-fb50681e-5f25-4180-99e2-065ef0b9791b.metadata.json +++ /dev/null @@ -1,98 +0,0 @@ -{ - "format-version" : 1, - "table-uuid" : "e54626bf-c7ab-4f36-a3d0-3e13eec0824f", - "location" : "/home/iceberg/warehouse/datahub/integration/profiling", - "last-updated-ms" : 1651614151056, - "last-column-id" : 3, - "schema" : { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "field_int", - "required" : false, - "type" : "long", - "doc" : "An integer field" - }, { - "id" : 2, - "name" : "field_str", - "required" : false, - "type" : "string", - "doc" : "A string field" - }, { - "id" : 3, - "name" : "field_timestamp", - "required" : false, - "type" : "timestamptz", - "doc" : "A timestamp field" - } ] - }, - "current-schema-id" : 0, - "schemas" : [ { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "field_int", - "required" : false, - "type" : "long", - "doc" : "An integer field" - }, { - "id" : 2, - "name" : "field_str", - "required" : false, - "type" : "string", - "doc" : "A string field" - }, { - "id" : 3, - "name" : "field_timestamp", - "required" : false, - "type" : "timestamptz", - "doc" : "A timestamp field" - } ] - } ], - "partition-spec" : [ ], - "default-spec-id" : 0, - "partition-specs" : [ { - "spec-id" : 0, - "fields" : [ ] - } ], - "last-partition-id" : 999, - "default-sort-order-id" : 0, - "sort-orders" : [ { - "order-id" : 0, - "fields" : [ ] - } ], - "properties" : { - "owner" : "root" - }, - "current-snapshot-id" : 4437197002876030991, - "snapshots" : [ { - "snapshot-id" : 4437197002876030991, - "timestamp-ms" : 1651614151056, - "summary" : { - "operation" : "append", - "spark.app.id" : "local-1651614127284", - "added-data-files" : "2", - "added-records" : "2", - "added-files-size" : "2114", - "changed-partition-count" : "1", - "total-records" : "2", - "total-files-size" : "2114", - "total-data-files" : "2", - "total-delete-files" : "0", - "total-position-deletes" : "0", - "total-equality-deletes" : "0" - }, - "manifest-list" : "/home/iceberg/warehouse/datahub/integration/profiling/metadata/snap-4437197002876030991-1-23acaffc-9bed-4d97-8ddd-0ea1ea15a2b8.avro", - "schema-id" : 0 - } ], - "snapshot-log" : [ { - "timestamp-ms" : 1651614151056, - "snapshot-id" : 4437197002876030991 - } ], - "metadata-log" : [ { - "timestamp-ms" : 1651614148692, - "metadata-file" : "/home/iceberg/warehouse/datahub/integration/profiling/metadata/00000-331b9f67-e02b-44b1-8ec8-4dfa287c3bd5.metadata.json" - } ] -} \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/metadata/00002-cc241948-4c12-46d0-9a75-ce3578ec03d4.metadata.json b/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/metadata/00002-cc241948-4c12-46d0-9a75-ce3578ec03d4.metadata.json deleted file mode 100644 index ab028a647de4c..0000000000000 --- a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/metadata/00002-cc241948-4c12-46d0-9a75-ce3578ec03d4.metadata.json +++ /dev/null @@ -1,124 +0,0 @@ -{ - "format-version" : 1, - "table-uuid" : "e54626bf-c7ab-4f36-a3d0-3e13eec0824f", - "location" : "/home/iceberg/warehouse/datahub/integration/profiling", - "last-updated-ms" : 1651614244732, - "last-column-id" : 3, - "schema" : { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "field_int", - "required" : false, - "type" : "long", - "doc" : "An integer field" - }, { - "id" : 2, - "name" : "field_str", - "required" : false, - "type" : "string", - "doc" : "A string field" - }, { - "id" : 3, - "name" : "field_timestamp", - "required" : false, - "type" : "timestamptz", - "doc" : "A timestamp field" - } ] - }, - "current-schema-id" : 0, - "schemas" : [ { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "field_int", - "required" : false, - "type" : "long", - "doc" : "An integer field" - }, { - "id" : 2, - "name" : "field_str", - "required" : false, - "type" : "string", - "doc" : "A string field" - }, { - "id" : 3, - "name" : "field_timestamp", - "required" : false, - "type" : "timestamptz", - "doc" : "A timestamp field" - } ] - } ], - "partition-spec" : [ ], - "default-spec-id" : 0, - "partition-specs" : [ { - "spec-id" : 0, - "fields" : [ ] - } ], - "last-partition-id" : 999, - "default-sort-order-id" : 0, - "sort-orders" : [ { - "order-id" : 0, - "fields" : [ ] - } ], - "properties" : { - "owner" : "root" - }, - "current-snapshot-id" : 4220723025353071767, - "snapshots" : [ { - "snapshot-id" : 4437197002876030991, - "timestamp-ms" : 1651614151056, - "summary" : { - "operation" : "append", - "spark.app.id" : "local-1651614127284", - "added-data-files" : "2", - "added-records" : "2", - "added-files-size" : "2114", - "changed-partition-count" : "1", - "total-records" : "2", - "total-files-size" : "2114", - "total-data-files" : "2", - "total-delete-files" : "0", - "total-position-deletes" : "0", - "total-equality-deletes" : "0" - }, - "manifest-list" : "/home/iceberg/warehouse/datahub/integration/profiling/metadata/snap-4437197002876030991-1-23acaffc-9bed-4d97-8ddd-0ea1ea15a2b8.avro", - "schema-id" : 0 - }, { - "snapshot-id" : 4220723025353071767, - "parent-snapshot-id" : 4437197002876030991, - "timestamp-ms" : 1651614244732, - "summary" : { - "operation" : "append", - "spark.app.id" : "local-1651614127284", - "added-data-files" : "2", - "added-records" : "2", - "added-files-size" : "2111", - "changed-partition-count" : "1", - "total-records" : "4", - "total-files-size" : "4225", - "total-data-files" : "4", - "total-delete-files" : "0", - "total-position-deletes" : "0", - "total-equality-deletes" : "0" - }, - "manifest-list" : "/home/iceberg/warehouse/datahub/integration/profiling/metadata/snap-4220723025353071767-1-ec0bd970-e5ef-4843-abcb-e96a35a8f14d.avro", - "schema-id" : 0 - } ], - "snapshot-log" : [ { - "timestamp-ms" : 1651614151056, - "snapshot-id" : 4437197002876030991 - }, { - "timestamp-ms" : 1651614244732, - "snapshot-id" : 4220723025353071767 - } ], - "metadata-log" : [ { - "timestamp-ms" : 1651614148692, - "metadata-file" : "/home/iceberg/warehouse/datahub/integration/profiling/metadata/00000-331b9f67-e02b-44b1-8ec8-4dfa287c3bd5.metadata.json" - }, { - "timestamp-ms" : 1651614151056, - "metadata-file" : "/home/iceberg/warehouse/datahub/integration/profiling/metadata/00001-fb50681e-5f25-4180-99e2-065ef0b9791b.metadata.json" - } ] -} \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/metadata/23acaffc-9bed-4d97-8ddd-0ea1ea15a2b8-m0.avro b/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/metadata/23acaffc-9bed-4d97-8ddd-0ea1ea15a2b8-m0.avro deleted file mode 100644 index 3019df4adae30..0000000000000 Binary files a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/metadata/23acaffc-9bed-4d97-8ddd-0ea1ea15a2b8-m0.avro and /dev/null differ diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/metadata/ec0bd970-e5ef-4843-abcb-e96a35a8f14d-m0.avro b/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/metadata/ec0bd970-e5ef-4843-abcb-e96a35a8f14d-m0.avro deleted file mode 100644 index 1b51cd60d136a..0000000000000 Binary files a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/metadata/ec0bd970-e5ef-4843-abcb-e96a35a8f14d-m0.avro and /dev/null differ diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/metadata/snap-4220723025353071767-1-ec0bd970-e5ef-4843-abcb-e96a35a8f14d.avro b/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/metadata/snap-4220723025353071767-1-ec0bd970-e5ef-4843-abcb-e96a35a8f14d.avro deleted file mode 100644 index 0dd50d23037e9..0000000000000 Binary files a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/metadata/snap-4220723025353071767-1-ec0bd970-e5ef-4843-abcb-e96a35a8f14d.avro and /dev/null differ diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/metadata/snap-4437197002876030991-1-23acaffc-9bed-4d97-8ddd-0ea1ea15a2b8.avro b/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/metadata/snap-4437197002876030991-1-23acaffc-9bed-4d97-8ddd-0ea1ea15a2b8.avro deleted file mode 100644 index 93f69f0ac1540..0000000000000 Binary files a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/metadata/snap-4437197002876030991-1-23acaffc-9bed-4d97-8ddd-0ea1ea15a2b8.avro and /dev/null differ diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/metadata/version-hint.text b/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/metadata/version-hint.text deleted file mode 100755 index d8263ee986059..0000000000000 --- a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/datahub/integration/profiling/metadata/version-hint.text +++ /dev/null @@ -1 +0,0 @@ -2 \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/iceberg_mces_golden.json b/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/iceberg_mces_golden.json deleted file mode 100644 index edfa8f80670cf..0000000000000 --- a/metadata-ingestion/tests/integration/iceberg/test_data/profiling_test/iceberg_mces_golden.json +++ /dev/null @@ -1,129 +0,0 @@ -[ -{ - "proposedSnapshot": { - "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { - "urn": "urn:li:dataset:(urn:li:dataPlatform:iceberg,datahub.integration.profiling,PROD)", - "aspects": [ - { - "com.linkedin.pegasus2avro.common.Status": { - "removed": false - } - }, - { - "com.linkedin.pegasus2avro.dataset.DatasetProperties": { - "customProperties": { - "owner": "root", - "provider": "ICEBERG", - "location": "/home/iceberg/warehouse/datahub/integration/profiling", - "snapshot-id": "4220723025353071767", - "manifest-list": "/home/iceberg/warehouse/datahub/integration/profiling/metadata/snap-4220723025353071767-1-ec0bd970-e5ef-4843-abcb-e96a35a8f14d.avro" - }, - "tags": [] - } - }, - { - "com.linkedin.pegasus2avro.common.Ownership": { - "owners": [ - { - "owner": "urn:li:corpuser:root", - "type": "TECHNICAL_OWNER" - }, - { - "owner": "urn:li:corpGroup:root", - "type": "TECHNICAL_OWNER" - } - ], - "lastModified": { - "time": 0, - "actor": "urn:li:corpuser:unknown" - } - } - }, - { - "com.linkedin.pegasus2avro.schema.SchemaMetadata": { - "schemaName": "datahub.integration.profiling", - "platform": "urn:li:dataPlatform:iceberg", - "version": 0, - "created": { - "time": 0, - "actor": "urn:li:corpuser:unknown" - }, - "lastModified": { - "time": 0, - "actor": "urn:li:corpuser:unknown" - }, - "hash": "", - "platformSchema": { - "com.linkedin.pegasus2avro.schema.OtherSchema": { - "rawSchema": "Schema(1: field_int: optional long(An integer field),2: field_str: optional string(A string field),3: field_timestamp: optional timestamptz(A timestamp field))" - } - }, - "fields": [ - { - "fieldPath": "[version=2.0].[type=struct].[type=long].field_int", - "nullable": true, - "description": "An integer field", - "type": { - "type": { - "com.linkedin.pegasus2avro.schema.NumberType": {} - } - }, - "nativeDataType": "long", - "recursive": false, - "isPartOfKey": false, - "jsonProps": "{\"native_data_type\": \"long\", \"_nullable\": true}" - }, - { - "fieldPath": "[version=2.0].[type=struct].[type=string].field_str", - "nullable": true, - "description": "A string field", - "type": { - "type": { - "com.linkedin.pegasus2avro.schema.StringType": {} - } - }, - "nativeDataType": "string", - "recursive": false, - "isPartOfKey": false, - "jsonProps": "{\"native_data_type\": \"string\", \"_nullable\": true}" - }, - { - "fieldPath": "[version=2.0].[type=struct].[type=long].field_timestamp", - "nullable": true, - "description": "A timestamp field", - "type": { - "type": { - "com.linkedin.pegasus2avro.schema.TimeType": {} - } - }, - "nativeDataType": "timestamptz", - "recursive": false, - "isPartOfKey": false, - "jsonProps": "{\"logicalType\": \"timestamp-micros\", \"native_data_type\": \"timestamptz\", \"_nullable\": true}" - } - ] - } - } - ] - } - }, - "systemMetadata": { - "lastObserved": 1586847600000, - "runId": "iceberg-test" - } -}, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:iceberg,datahub.integration.profiling,PROD)", - "changeType": "UPSERT", - "aspectName": "datasetProfile", - "aspect": { - "value": "{\"timestampMillis\": 1586847600000, \"partitionSpec\": {\"type\": \"FULL_TABLE\", \"partition\": \"FULL_TABLE_SNAPSHOT\"}, \"rowCount\": 4, \"columnCount\": 3, \"fieldProfiles\": [{\"fieldPath\": \"field_int\", \"nullCount\": 0, \"nullProportion\": 0.0, \"min\": \"1\", \"max\": \"4\"}, {\"fieldPath\": \"field_str\", \"nullCount\": 0, \"nullProportion\": 0.0}, {\"fieldPath\": \"field_timestamp\", \"nullCount\": 2, \"nullProportion\": 0.5, \"min\": \"2022-05-03 21:42:29\", \"max\": \"2022-05-03 21:44:04\"}]}", - "contentType": "application/json" - }, - "systemMetadata": { - "lastObserved": 1586847600000, - "runId": "iceberg-test" - } -} -] \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/iceberg_deleted_table_mces_golden.json b/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/iceberg_deleted_table_mces_golden.json deleted file mode 100644 index d376d8b645d66..0000000000000 --- a/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/iceberg_deleted_table_mces_golden.json +++ /dev/null @@ -1,159 +0,0 @@ -[ -{ - "proposedSnapshot": { - "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { - "urn": "urn:li:dataset:(urn:li:dataPlatform:iceberg,test_platform_instance.namespace.iceberg_test,PROD)", - "aspects": [ - { - "com.linkedin.pegasus2avro.common.Status": { - "removed": false - } - }, - { - "com.linkedin.pegasus2avro.dataset.DatasetProperties": { - "customProperties": { - "owner": "new_owner", - "provider": "ICEBERG", - "location": "/namespace/iceberg_test" - }, - "tags": [] - } - }, - { - "com.linkedin.pegasus2avro.common.Ownership": { - "owners": [ - { - "owner": "urn:li:corpuser:new_owner", - "type": "TECHNICAL_OWNER" - }, - { - "owner": "urn:li:corpGroup:new_owner", - "type": "TECHNICAL_OWNER" - } - ], - "lastModified": { - "time": 0, - "actor": "urn:li:corpuser:unknown" - } - } - }, - { - "com.linkedin.pegasus2avro.schema.SchemaMetadata": { - "schemaName": "namespace.iceberg_test", - "platform": "urn:li:dataPlatform:iceberg", - "version": 0, - "created": { - "time": 0, - "actor": "urn:li:corpuser:unknown" - }, - "lastModified": { - "time": 0, - "actor": "urn:li:corpuser:unknown" - }, - "hash": "", - "platformSchema": { - "com.linkedin.pegasus2avro.schema.OtherSchema": { - "rawSchema": "Schema(1: level: required string(level documentation),2: event_time: required timestamptz(event_time documentation),3: message: required string(message documentation),4: call_stack: optional list(call_stack documentation))" - } - }, - "fields": [ - { - "fieldPath": "[version=2.0].[type=struct].[type=string].level", - "nullable": false, - "description": "level documentation", - "type": { - "type": { - "com.linkedin.pegasus2avro.schema.StringType": {} - } - }, - "nativeDataType": "string", - "recursive": false, - "isPartOfKey": false, - "jsonProps": "{\"native_data_type\": \"string\", \"_nullable\": false}" - }, - { - "fieldPath": "[version=2.0].[type=struct].[type=long].event_time", - "nullable": false, - "description": "event_time documentation", - "type": { - "type": { - "com.linkedin.pegasus2avro.schema.TimeType": {} - } - }, - "nativeDataType": "timestamptz", - "recursive": false, - "isPartOfKey": false, - "jsonProps": "{\"logicalType\": \"timestamp-micros\", \"native_data_type\": \"timestamptz\", \"_nullable\": false}" - }, - { - "fieldPath": "[version=2.0].[type=struct].[type=string].message", - "nullable": false, - "description": "message documentation", - "type": { - "type": { - "com.linkedin.pegasus2avro.schema.StringType": {} - } - }, - "nativeDataType": "string", - "recursive": false, - "isPartOfKey": false, - "jsonProps": "{\"native_data_type\": \"string\", \"_nullable\": false}" - }, - { - "fieldPath": "[version=2.0].[type=struct].[type=array].[type=string].call_stack", - "nullable": true, - "description": "call_stack documentation", - "type": { - "type": { - "com.linkedin.pegasus2avro.schema.ArrayType": { - "nestedType": [ - "string" - ] - } - } - }, - "nativeDataType": "list", - "recursive": false, - "isPartOfKey": false, - "jsonProps": "{\"native_data_type\": \"list\", \"_nullable\": true}" - } - ] - } - } - ] - } - }, - "systemMetadata": { - "lastObserved": 1586847600000, - "runId": "iceberg-2020_04_14-07_00_00" - } -}, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:iceberg,test_platform_instance.namespace.iceberg_test,PROD)", - "changeType": "UPSERT", - "aspectName": "dataPlatformInstance", - "aspect": { - "value": "{\"platform\": \"urn:li:dataPlatform:iceberg\", \"instance\": \"urn:li:dataPlatformInstance:(urn:li:dataPlatform:iceberg,test_platform_instance)\"}", - "contentType": "application/json" - }, - "systemMetadata": { - "lastObserved": 1586847600000, - "runId": "iceberg-2020_04_14-07_00_00" - } -}, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:iceberg,test_platform_instance.namespace.iceberg_test_2,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "value": "{\"removed\": true}", - "contentType": "application/json" - }, - "systemMetadata": { - "lastObserved": 1586847600000, - "runId": "iceberg-2020_04_14-07_00_00" - } -} -] \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run1/namespace/iceberg_test/metadata/v1.metadata.json b/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run1/namespace/iceberg_test/metadata/v1.metadata.json deleted file mode 100644 index e4ac0b9685ddc..0000000000000 --- a/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run1/namespace/iceberg_test/metadata/v1.metadata.json +++ /dev/null @@ -1,105 +0,0 @@ -{ - "format-version" : 1, - "table-uuid" : "11bbe5de-5ef6-4074-80db-f041065f9862", - "location" : "/namespace/iceberg_test", - "last-updated-ms" : 1648729616724, - "last-column-id" : 5, - "schema" : { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "level", - "required" : true, - "type" : "string" - }, { - "id" : 2, - "name" : "event_time", - "required" : true, - "type" : "timestamptz" - }, { - "id" : 3, - "name" : "message", - "required" : true, - "type" : "string" - }, { - "id" : 4, - "name" : "call_stack", - "required" : false, - "type" : { - "type" : "list", - "element-id" : 5, - "element" : "string", - "element-required" : true - } - } ] - }, - "current-schema-id" : 0, - "schemas" : [ { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "level", - "required" : true, - "type" : "string" - }, { - "id" : 2, - "name" : "event_time", - "required" : true, - "type" : "timestamptz" - }, { - "id" : 3, - "name" : "message", - "required" : true, - "type" : "string" - }, { - "id" : 4, - "name" : "call_stack", - "required" : false, - "type" : { - "type" : "list", - "element-id" : 5, - "element" : "string", - "element-required" : true - } - } ] - } ], - "partition-spec" : [ { - "name" : "event_time_hour", - "transform" : "hour", - "source-id" : 2, - "field-id" : 1000 - }, { - "name" : "level", - "transform" : "identity", - "source-id" : 1, - "field-id" : 1001 - } ], - "default-spec-id" : 0, - "partition-specs" : [ { - "spec-id" : 0, - "fields" : [ { - "name" : "event_time_hour", - "transform" : "hour", - "source-id" : 2, - "field-id" : 1000 - }, { - "name" : "level", - "transform" : "identity", - "source-id" : 1, - "field-id" : 1001 - } ] - } ], - "last-partition-id" : 1001, - "default-sort-order-id" : 0, - "sort-orders" : [ { - "order-id" : 0, - "fields" : [ ] - } ], - "properties" : { }, - "current-snapshot-id" : -1, - "snapshots" : [ ], - "snapshot-log" : [ ], - "metadata-log" : [ ] -} \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run1/namespace/iceberg_test/metadata/v2.metadata.json b/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run1/namespace/iceberg_test/metadata/v2.metadata.json deleted file mode 100644 index 02221330b0665..0000000000000 --- a/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run1/namespace/iceberg_test/metadata/v2.metadata.json +++ /dev/null @@ -1,118 +0,0 @@ -{ - "format-version" : 1, - "table-uuid" : "16e6ecee-cd5d-470f-a7a6-a197944fa4db", - "location" : "/namespace/iceberg_test", - "last-updated-ms" : 1649086837695, - "last-column-id" : 5, - "schema" : { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "level", - "required" : true, - "type" : "string", - "doc" : "level documentation" - }, { - "id" : 2, - "name" : "event_time", - "required" : true, - "type" : "timestamptz", - "doc" : "event_time documentation" - }, { - "id" : 3, - "name" : "message", - "required" : true, - "type" : "string", - "doc" : "message documentation" - }, { - "id" : 4, - "name" : "call_stack", - "required" : false, - "type" : { - "type" : "list", - "element-id" : 5, - "element" : "string", - "element-required" : true - }, - "doc" : "call_stack documentation" - } ] - }, - "current-schema-id" : 0, - "schemas" : [ { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "level", - "required" : true, - "type" : "string", - "doc" : "level documentation" - }, { - "id" : 2, - "name" : "event_time", - "required" : true, - "type" : "timestamptz", - "doc" : "event_time documentation" - }, { - "id" : 3, - "name" : "message", - "required" : true, - "type" : "string", - "doc" : "message documentation" - }, { - "id" : 4, - "name" : "call_stack", - "required" : false, - "type" : { - "type" : "list", - "element-id" : 5, - "element" : "string", - "element-required" : true - }, - "doc" : "call_stack documentation" - } ] - } ], - "partition-spec" : [ { - "name" : "event_time_hour", - "transform" : "hour", - "source-id" : 2, - "field-id" : 1000 - }, { - "name" : "level", - "transform" : "identity", - "source-id" : 1, - "field-id" : 1001 - } ], - "default-spec-id" : 0, - "partition-specs" : [ { - "spec-id" : 0, - "fields" : [ { - "name" : "event_time_hour", - "transform" : "hour", - "source-id" : 2, - "field-id" : 1000 - }, { - "name" : "level", - "transform" : "identity", - "source-id" : 1, - "field-id" : 1001 - } ] - } ], - "last-partition-id" : 1001, - "default-sort-order-id" : 0, - "sort-orders" : [ { - "order-id" : 0, - "fields" : [ ] - } ], - "properties" : { - "owner" : "new_owner" - }, - "current-snapshot-id" : -1, - "snapshots" : [ ], - "snapshot-log" : [ ], - "metadata-log" : [ { - "timestamp-ms" : 1649086837511, - "metadata-file" : "/namespace/iceberg_test/metadata/v1.metadata.json" - } ] -} \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run1/namespace/iceberg_test/metadata/version-hint.text b/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run1/namespace/iceberg_test/metadata/version-hint.text deleted file mode 100644 index d8263ee986059..0000000000000 --- a/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run1/namespace/iceberg_test/metadata/version-hint.text +++ /dev/null @@ -1 +0,0 @@ -2 \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run1/namespace/iceberg_test_2/metadata/v1.metadata.json b/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run1/namespace/iceberg_test_2/metadata/v1.metadata.json deleted file mode 100644 index e4ac0b9685ddc..0000000000000 --- a/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run1/namespace/iceberg_test_2/metadata/v1.metadata.json +++ /dev/null @@ -1,105 +0,0 @@ -{ - "format-version" : 1, - "table-uuid" : "11bbe5de-5ef6-4074-80db-f041065f9862", - "location" : "/namespace/iceberg_test", - "last-updated-ms" : 1648729616724, - "last-column-id" : 5, - "schema" : { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "level", - "required" : true, - "type" : "string" - }, { - "id" : 2, - "name" : "event_time", - "required" : true, - "type" : "timestamptz" - }, { - "id" : 3, - "name" : "message", - "required" : true, - "type" : "string" - }, { - "id" : 4, - "name" : "call_stack", - "required" : false, - "type" : { - "type" : "list", - "element-id" : 5, - "element" : "string", - "element-required" : true - } - } ] - }, - "current-schema-id" : 0, - "schemas" : [ { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "level", - "required" : true, - "type" : "string" - }, { - "id" : 2, - "name" : "event_time", - "required" : true, - "type" : "timestamptz" - }, { - "id" : 3, - "name" : "message", - "required" : true, - "type" : "string" - }, { - "id" : 4, - "name" : "call_stack", - "required" : false, - "type" : { - "type" : "list", - "element-id" : 5, - "element" : "string", - "element-required" : true - } - } ] - } ], - "partition-spec" : [ { - "name" : "event_time_hour", - "transform" : "hour", - "source-id" : 2, - "field-id" : 1000 - }, { - "name" : "level", - "transform" : "identity", - "source-id" : 1, - "field-id" : 1001 - } ], - "default-spec-id" : 0, - "partition-specs" : [ { - "spec-id" : 0, - "fields" : [ { - "name" : "event_time_hour", - "transform" : "hour", - "source-id" : 2, - "field-id" : 1000 - }, { - "name" : "level", - "transform" : "identity", - "source-id" : 1, - "field-id" : 1001 - } ] - } ], - "last-partition-id" : 1001, - "default-sort-order-id" : 0, - "sort-orders" : [ { - "order-id" : 0, - "fields" : [ ] - } ], - "properties" : { }, - "current-snapshot-id" : -1, - "snapshots" : [ ], - "snapshot-log" : [ ], - "metadata-log" : [ ] -} \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run1/namespace/iceberg_test_2/metadata/v2.metadata.json b/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run1/namespace/iceberg_test_2/metadata/v2.metadata.json deleted file mode 100644 index 02221330b0665..0000000000000 --- a/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run1/namespace/iceberg_test_2/metadata/v2.metadata.json +++ /dev/null @@ -1,118 +0,0 @@ -{ - "format-version" : 1, - "table-uuid" : "16e6ecee-cd5d-470f-a7a6-a197944fa4db", - "location" : "/namespace/iceberg_test", - "last-updated-ms" : 1649086837695, - "last-column-id" : 5, - "schema" : { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "level", - "required" : true, - "type" : "string", - "doc" : "level documentation" - }, { - "id" : 2, - "name" : "event_time", - "required" : true, - "type" : "timestamptz", - "doc" : "event_time documentation" - }, { - "id" : 3, - "name" : "message", - "required" : true, - "type" : "string", - "doc" : "message documentation" - }, { - "id" : 4, - "name" : "call_stack", - "required" : false, - "type" : { - "type" : "list", - "element-id" : 5, - "element" : "string", - "element-required" : true - }, - "doc" : "call_stack documentation" - } ] - }, - "current-schema-id" : 0, - "schemas" : [ { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "level", - "required" : true, - "type" : "string", - "doc" : "level documentation" - }, { - "id" : 2, - "name" : "event_time", - "required" : true, - "type" : "timestamptz", - "doc" : "event_time documentation" - }, { - "id" : 3, - "name" : "message", - "required" : true, - "type" : "string", - "doc" : "message documentation" - }, { - "id" : 4, - "name" : "call_stack", - "required" : false, - "type" : { - "type" : "list", - "element-id" : 5, - "element" : "string", - "element-required" : true - }, - "doc" : "call_stack documentation" - } ] - } ], - "partition-spec" : [ { - "name" : "event_time_hour", - "transform" : "hour", - "source-id" : 2, - "field-id" : 1000 - }, { - "name" : "level", - "transform" : "identity", - "source-id" : 1, - "field-id" : 1001 - } ], - "default-spec-id" : 0, - "partition-specs" : [ { - "spec-id" : 0, - "fields" : [ { - "name" : "event_time_hour", - "transform" : "hour", - "source-id" : 2, - "field-id" : 1000 - }, { - "name" : "level", - "transform" : "identity", - "source-id" : 1, - "field-id" : 1001 - } ] - } ], - "last-partition-id" : 1001, - "default-sort-order-id" : 0, - "sort-orders" : [ { - "order-id" : 0, - "fields" : [ ] - } ], - "properties" : { - "owner" : "new_owner" - }, - "current-snapshot-id" : -1, - "snapshots" : [ ], - "snapshot-log" : [ ], - "metadata-log" : [ { - "timestamp-ms" : 1649086837511, - "metadata-file" : "/namespace/iceberg_test/metadata/v1.metadata.json" - } ] -} \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run1/namespace/iceberg_test_2/metadata/version-hint.text b/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run1/namespace/iceberg_test_2/metadata/version-hint.text deleted file mode 100644 index d8263ee986059..0000000000000 --- a/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run1/namespace/iceberg_test_2/metadata/version-hint.text +++ /dev/null @@ -1 +0,0 @@ -2 \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run2/namespace/iceberg_test/metadata/v1.metadata.json b/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run2/namespace/iceberg_test/metadata/v1.metadata.json deleted file mode 100644 index e4ac0b9685ddc..0000000000000 --- a/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run2/namespace/iceberg_test/metadata/v1.metadata.json +++ /dev/null @@ -1,105 +0,0 @@ -{ - "format-version" : 1, - "table-uuid" : "11bbe5de-5ef6-4074-80db-f041065f9862", - "location" : "/namespace/iceberg_test", - "last-updated-ms" : 1648729616724, - "last-column-id" : 5, - "schema" : { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "level", - "required" : true, - "type" : "string" - }, { - "id" : 2, - "name" : "event_time", - "required" : true, - "type" : "timestamptz" - }, { - "id" : 3, - "name" : "message", - "required" : true, - "type" : "string" - }, { - "id" : 4, - "name" : "call_stack", - "required" : false, - "type" : { - "type" : "list", - "element-id" : 5, - "element" : "string", - "element-required" : true - } - } ] - }, - "current-schema-id" : 0, - "schemas" : [ { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "level", - "required" : true, - "type" : "string" - }, { - "id" : 2, - "name" : "event_time", - "required" : true, - "type" : "timestamptz" - }, { - "id" : 3, - "name" : "message", - "required" : true, - "type" : "string" - }, { - "id" : 4, - "name" : "call_stack", - "required" : false, - "type" : { - "type" : "list", - "element-id" : 5, - "element" : "string", - "element-required" : true - } - } ] - } ], - "partition-spec" : [ { - "name" : "event_time_hour", - "transform" : "hour", - "source-id" : 2, - "field-id" : 1000 - }, { - "name" : "level", - "transform" : "identity", - "source-id" : 1, - "field-id" : 1001 - } ], - "default-spec-id" : 0, - "partition-specs" : [ { - "spec-id" : 0, - "fields" : [ { - "name" : "event_time_hour", - "transform" : "hour", - "source-id" : 2, - "field-id" : 1000 - }, { - "name" : "level", - "transform" : "identity", - "source-id" : 1, - "field-id" : 1001 - } ] - } ], - "last-partition-id" : 1001, - "default-sort-order-id" : 0, - "sort-orders" : [ { - "order-id" : 0, - "fields" : [ ] - } ], - "properties" : { }, - "current-snapshot-id" : -1, - "snapshots" : [ ], - "snapshot-log" : [ ], - "metadata-log" : [ ] -} \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run2/namespace/iceberg_test/metadata/v2.metadata.json b/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run2/namespace/iceberg_test/metadata/v2.metadata.json deleted file mode 100644 index 02221330b0665..0000000000000 --- a/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run2/namespace/iceberg_test/metadata/v2.metadata.json +++ /dev/null @@ -1,118 +0,0 @@ -{ - "format-version" : 1, - "table-uuid" : "16e6ecee-cd5d-470f-a7a6-a197944fa4db", - "location" : "/namespace/iceberg_test", - "last-updated-ms" : 1649086837695, - "last-column-id" : 5, - "schema" : { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "level", - "required" : true, - "type" : "string", - "doc" : "level documentation" - }, { - "id" : 2, - "name" : "event_time", - "required" : true, - "type" : "timestamptz", - "doc" : "event_time documentation" - }, { - "id" : 3, - "name" : "message", - "required" : true, - "type" : "string", - "doc" : "message documentation" - }, { - "id" : 4, - "name" : "call_stack", - "required" : false, - "type" : { - "type" : "list", - "element-id" : 5, - "element" : "string", - "element-required" : true - }, - "doc" : "call_stack documentation" - } ] - }, - "current-schema-id" : 0, - "schemas" : [ { - "type" : "struct", - "schema-id" : 0, - "fields" : [ { - "id" : 1, - "name" : "level", - "required" : true, - "type" : "string", - "doc" : "level documentation" - }, { - "id" : 2, - "name" : "event_time", - "required" : true, - "type" : "timestamptz", - "doc" : "event_time documentation" - }, { - "id" : 3, - "name" : "message", - "required" : true, - "type" : "string", - "doc" : "message documentation" - }, { - "id" : 4, - "name" : "call_stack", - "required" : false, - "type" : { - "type" : "list", - "element-id" : 5, - "element" : "string", - "element-required" : true - }, - "doc" : "call_stack documentation" - } ] - } ], - "partition-spec" : [ { - "name" : "event_time_hour", - "transform" : "hour", - "source-id" : 2, - "field-id" : 1000 - }, { - "name" : "level", - "transform" : "identity", - "source-id" : 1, - "field-id" : 1001 - } ], - "default-spec-id" : 0, - "partition-specs" : [ { - "spec-id" : 0, - "fields" : [ { - "name" : "event_time_hour", - "transform" : "hour", - "source-id" : 2, - "field-id" : 1000 - }, { - "name" : "level", - "transform" : "identity", - "source-id" : 1, - "field-id" : 1001 - } ] - } ], - "last-partition-id" : 1001, - "default-sort-order-id" : 0, - "sort-orders" : [ { - "order-id" : 0, - "fields" : [ ] - } ], - "properties" : { - "owner" : "new_owner" - }, - "current-snapshot-id" : -1, - "snapshots" : [ ], - "snapshot-log" : [ ], - "metadata-log" : [ { - "timestamp-ms" : 1649086837511, - "metadata-file" : "/namespace/iceberg_test/metadata/v1.metadata.json" - } ] -} \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run2/namespace/iceberg_test/metadata/version-hint.text b/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run2/namespace/iceberg_test/metadata/version-hint.text deleted file mode 100644 index d8263ee986059..0000000000000 --- a/metadata-ingestion/tests/integration/iceberg/test_data/stateful_test/run2/namespace/iceberg_test/metadata/version-hint.text +++ /dev/null @@ -1 +0,0 @@ -2 \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/iceberg/test_iceberg.py b/metadata-ingestion/tests/integration/iceberg/test_iceberg.py index 032ba93b1b4c3..2383744bd739e 100644 --- a/metadata-ingestion/tests/integration/iceberg/test_iceberg.py +++ b/metadata-ingestion/tests/integration/iceberg/test_iceberg.py @@ -1,17 +1,17 @@ -from pathlib import PosixPath -from typing import Any, Dict, Optional, Union, cast +import subprocess +from typing import Any, Dict, List, Optional, cast from unittest.mock import patch import pytest from freezegun import freeze_time -from iceberg.core.filesystem.file_status import FileStatus -from iceberg.core.filesystem.local_filesystem import LocalFileSystem from datahub.ingestion.run.pipeline import Pipeline from datahub.ingestion.source.iceberg.iceberg import IcebergSource from datahub.ingestion.source.state.checkpoint import Checkpoint from datahub.ingestion.source.state.entity_removal_state import GenericCheckpointState from tests.test_helpers import mce_helpers +from tests.test_helpers.click_helpers import run_datahub_cmd +from tests.test_helpers.docker_helpers import wait_for_port from tests.test_helpers.state_helpers import ( run_and_get_pipeline, validate_all_providers_have_committed_successfully, @@ -31,89 +31,85 @@ def get_current_checkpoint_from_pipeline( ) +def spark_submit(file_path: str) -> None: + docker = "docker" + command = f"{docker} exec spark-iceberg spark-submit {file_path}" + ret = subprocess.run( + command, shell=True, stdout=subprocess.PIPE, stderr=subprocess.PIPE + ) + assert ret.returncode == 0 + + @freeze_time(FROZEN_TIME) @pytest.mark.integration -def test_iceberg_ingest(pytestconfig, tmp_path, mock_time): +def test_iceberg_ingest(docker_compose_runner, pytestconfig, tmp_path, mock_time): test_resources_dir = pytestconfig.rootpath / "tests/integration/iceberg/" - # Run the metadata ingestion pipeline. - pipeline = Pipeline.create( - { - "run_id": "iceberg-test", - "source": { - "type": "iceberg", - "config": { - "localfs": str(test_resources_dir / "test_data/ingest_test"), - "user_ownership_property": "owner", - "group_ownership_property": "owner", - }, - }, - "sink": { - "type": "file", - "config": { - "filename": f"{tmp_path}/iceberg_mces.json", - }, - }, - } - ) - pipeline.run() - pipeline.raise_from_status() - - # Verify the output. - mce_helpers.check_golden_file( - pytestconfig, - output_path=tmp_path / "iceberg_mces.json", - golden_path=test_resources_dir - / "test_data/ingest_test/iceberg_mces_golden.json", - ) + with docker_compose_runner( + test_resources_dir / "docker-compose.yml", "iceberg" + ) as docker_services: + wait_for_port(docker_services, "spark-iceberg", 8888, timeout=120) + + # Run the create.py pyspark file to populate the table. + spark_submit("/home/iceberg/setup/create.py") + + # Run the metadata ingestion pipeline. + config_file = (test_resources_dir / "iceberg_to_file.yml").resolve() + run_datahub_cmd( + ["ingest", "--strict-warnings", "-c", f"{config_file}"], tmp_path=tmp_path + ) + # These paths change from one instance run of the clickhouse docker to the other, and the FROZEN_TIME does not apply to these. + ignore_paths: List[str] = [ + r"root\[\d+\]\['proposedSnapshot'\].+\['aspects'\].+\['customProperties'\]\['created-at'\]", + r"root\[\d+\]\['proposedSnapshot'\].+\['aspects'\].+\['customProperties'\]\['snapshot-id'\]", + r"root\[\d+\]\['proposedSnapshot'\].+\['aspects'\].+\['customProperties'\]\['manifest-list'\]", + ] + # Verify the output. + mce_helpers.check_golden_file( + pytestconfig, + ignore_paths=ignore_paths, + output_path=tmp_path / "iceberg_mces.json", + golden_path=test_resources_dir / "iceberg_ingest_mces_golden.json", + ) @freeze_time(FROZEN_TIME) @pytest.mark.integration -def test_iceberg_stateful_ingest(pytestconfig, tmp_path, mock_time, mock_datahub_graph): - test_resources_dir = ( - pytestconfig.rootpath / "tests/integration/iceberg/test_data/stateful_test" - ) +def test_iceberg_stateful_ingest( + docker_compose_runner, pytestconfig, tmp_path, mock_time, mock_datahub_graph +): + test_resources_dir = pytestconfig.rootpath / "tests/integration/iceberg" platform_instance = "test_platform_instance" - scd_before_deletion: Dict[str, Any] = { - "localfs": str(test_resources_dir / "run1"), - "user_ownership_property": "owner", - "group_ownership_property": "owner", - "platform_instance": f"{platform_instance}", - # enable stateful ingestion - "stateful_ingestion": { - "enabled": True, - "remove_stale_metadata": True, - "fail_safe_threshold": 100.0, - "state_provider": { - "type": "datahub", - "config": {"datahub_api": {"server": GMS_SERVER}}, - }, - }, - } - - scd_after_deletion: Dict[str, Any] = { - "localfs": str(test_resources_dir / "run2"), - "user_ownership_property": "owner", - "group_ownership_property": "owner", - "platform_instance": f"{platform_instance}", - # enable stateful ingestion - "stateful_ingestion": { - "enabled": True, - "remove_stale_metadata": True, - "fail_safe_threshold": 100.0, - "state_provider": { - "type": "datahub", - "config": {"datahub_api": {"server": GMS_SERVER}}, - }, - }, - } - pipeline_config_dict: Dict[str, Any] = { "source": { "type": "iceberg", - "config": scd_before_deletion, + "config": { + "catalog": { + "name": "default", + "conf": { + "uri": "http://localhost:8181", + "s3.access-key-id": "admin", + "s3.secret-access-key": "password", + "s3.region": "us-east-1", + "warehouse": "s3a://warehouse/wh/", + "s3.endpoint": "http://localhost:9000", + }, + }, + "user_ownership_property": "owner", + "group_ownership_property": "owner", + "platform_instance": f"{platform_instance}", + # enable stateful ingestion + "stateful_ingestion": { + "enabled": True, + "remove_stale_metadata": True, + "fail_safe_threshold": 100.0, + "state_provider": { + "type": "datahub", + "config": {"datahub_api": {"server": GMS_SERVER}}, + }, + }, + }, }, "sink": { # we are not really interested in the resulting events for this test @@ -122,10 +118,17 @@ def test_iceberg_stateful_ingest(pytestconfig, tmp_path, mock_time, mock_datahub "pipeline_name": "test_pipeline", } - with patch( + with docker_compose_runner( + test_resources_dir / "docker-compose.yml", "iceberg" + ) as docker_services, patch( "datahub.ingestion.source.state_provider.datahub_ingestion_checkpointing_provider.DataHubGraph", mock_datahub_graph, ) as mock_checkpoint: + wait_for_port(docker_services, "spark-iceberg", 8888, timeout=120) + + # Run the create.py pyspark file to populate the table. + spark_submit("/home/iceberg/setup/create.py") + # Both checkpoint and reporting will use the same mocked graph instance. mock_checkpoint.return_value = mock_datahub_graph @@ -136,13 +139,14 @@ def test_iceberg_stateful_ingest(pytestconfig, tmp_path, mock_time, mock_datahub assert checkpoint1 assert checkpoint1.state - # Set iceberg config where a table is deleted. - pipeline_config_dict["source"]["config"] = scd_after_deletion # Capture MCEs of second run to validate Status(removed=true) deleted_mces_path = f"{tmp_path}/iceberg_deleted_mces.json" pipeline_config_dict["sink"]["type"] = "file" pipeline_config_dict["sink"]["config"] = {"filename": deleted_mces_path} + # Run the delete.py pyspark file to delete the table. + spark_submit("/home/iceberg/setup/delete.py") + # Do the second run of the pipeline. pipeline_run2 = run_and_get_pipeline(pipeline_config_dict) checkpoint2 = get_current_checkpoint_from_pipeline(pipeline_run2) @@ -160,7 +164,7 @@ def test_iceberg_stateful_ingest(pytestconfig, tmp_path, mock_time, mock_datahub assert len(difference_urns) == 1 - urn1 = "urn:li:dataset:(urn:li:dataPlatform:iceberg,test_platform_instance.namespace.iceberg_test_2,PROD)" + urn1 = "urn:li:dataset:(urn:li:dataPlatform:iceberg,test_platform_instance.nyc.taxis,PROD)" assert urn1 in difference_urns @@ -182,117 +186,32 @@ def test_iceberg_stateful_ingest(pytestconfig, tmp_path, mock_time, mock_datahub @freeze_time(FROZEN_TIME) @pytest.mark.integration -def test_iceberg_profiling(pytestconfig, tmp_path, mock_time): - """ - This test is using a table created using https://github.com/tabular-io/docker-spark-iceberg. - Here are the DDL statements that you can execute with `spark-sql`: - ```SQL - CREATE TABLE datahub.integration.profiling ( - field_int bigint COMMENT 'An integer field', - field_str string COMMENT 'A string field', - field_timestamp timestamp COMMENT 'A timestamp field') - USING iceberg; - - INSERT INTO datahub.integration.profiling VALUES (1, 'row1', current_timestamp()), (2, 'row2', null); - INSERT INTO datahub.integration.profiling VALUES (3, 'row3', current_timestamp()), (4, 'row4', null); - ``` - - When importing the metadata files into this test, we need to create a `version-hint.text` with a value that - reflects the version of the table, and then change the code in `TestLocalFileSystem._replace_path()` accordingly. - """ - test_resources_dir = ( - pytestconfig.rootpath / "tests/integration/iceberg/test_data/profiling_test" - ) +def test_iceberg_profiling(docker_compose_runner, pytestconfig, tmp_path, mock_time): + test_resources_dir = pytestconfig.rootpath / "tests/integration/iceberg/" - # Run the metadata ingestion pipeline. - pipeline = Pipeline.create( - { - "run_id": "iceberg-test", - "source": { - "type": "iceberg", - "config": { - "localfs": str(test_resources_dir), - "user_ownership_property": "owner", - "group_ownership_property": "owner", - "max_path_depth": 3, - "profiling": { - "enabled": True, - }, - "table_pattern": {"allow": ["datahub.integration.profiling"]}, - }, - }, - "sink": { - "type": "file", - "config": { - "filename": f"{tmp_path}/iceberg_mces.json", - }, - }, - } - ) + with docker_compose_runner( + test_resources_dir / "docker-compose.yml", "iceberg" + ) as docker_services: + wait_for_port(docker_services, "spark-iceberg", 8888, timeout=120) - class TestLocalFileSystem(LocalFileSystem): - # This class acts as a wrapper on LocalFileSystem to intercept calls using a path location. - # The wrapper will normalize those paths to be usable by the test. - fs: LocalFileSystem - - @staticmethod - def _replace_path(path: Union[str, PosixPath]) -> str: - # When the Iceberg table was created, its warehouse folder was '/home/iceberg/warehouse'. Iceberg tables - # are not portable, so we need to replace the warehouse folder by the test location at runtime. - normalized_path: str = str(path).replace( - "/home/iceberg/warehouse", str(test_resources_dir) - ) - - # When the Iceberg table was created, a postgres catalog was used instead of a HadoopCatalog. The HadoopCatalog - # expects a file named 'v{}.metadata.json' where {} is the version number from 'version-hint.text'. Since - # 'v2.metadata.json' does not exist, we will redirect the call to '00002-02782173-8364-4caf-a3c4-9567c1d6608f.metadata.json'. - if normalized_path.endswith("v2.metadata.json"): - return normalized_path.replace( - "v2.metadata.json", - "00002-cc241948-4c12-46d0-9a75-ce3578ec03d4.metadata.json", - ) - return normalized_path - - def __init__(self, fs: LocalFileSystem) -> None: - self.fs = fs - - def open(self, path: str, mode: str = "rb") -> object: - return self.fs.open(TestLocalFileSystem._replace_path(path), mode) - - def delete(self, path: str) -> None: - self.fs.delete(TestLocalFileSystem._replace_path(path)) - - def stat(self, path: str) -> FileStatus: - return self.fs.stat(TestLocalFileSystem._replace_path(path)) - - @staticmethod - def fix_path(path: str) -> str: - return TestLocalFileSystem.fs.fix_path( - TestLocalFileSystem._replace_path(path) - ) - - def create(self, path: str, overwrite: bool = False) -> object: - return self.fs.create(TestLocalFileSystem._replace_path(path), overwrite) - - def rename(self, src: str, dest: str) -> bool: - return self.fs.rename( - TestLocalFileSystem._replace_path(src), - TestLocalFileSystem._replace_path(dest), - ) - - def exists(self, path: str) -> bool: - return self.fs.exists(TestLocalFileSystem._replace_path(path)) - - local_fs_wrapper: TestLocalFileSystem = TestLocalFileSystem( - LocalFileSystem.get_instance() - ) - with patch.object(LocalFileSystem, "get_instance", return_value=local_fs_wrapper): - pipeline.run() - pipeline.raise_from_status() - - # Verify the output. - mce_helpers.check_golden_file( - pytestconfig, - output_path=tmp_path / "iceberg_mces.json", - golden_path=test_resources_dir / "iceberg_mces_golden.json", - ) + # Run the create.py pyspark file to populate the table. + spark_submit("/home/iceberg/setup/create.py") + + # Run the metadata ingestion pipeline. + config_file = (test_resources_dir / "iceberg_profile_to_file.yml").resolve() + run_datahub_cmd( + ["ingest", "--strict-warnings", "-c", f"{config_file}"], tmp_path=tmp_path + ) + # These paths change from one instance run of the clickhouse docker to the other, and the FROZEN_TIME does not apply to these. + ignore_paths: List[str] = [ + r"root\[\d+\]\['proposedSnapshot'\].+\['aspects'\].+\['customProperties'\]\['created-at'\]", + r"root\[\d+\]\['proposedSnapshot'\].+\['aspects'\].+\['customProperties'\]\['snapshot-id'\]", + r"root\[\d+\]\['proposedSnapshot'\].+\['aspects'\].+\['customProperties'\]\['manifest-list'\]", + ] + # Verify the output. + mce_helpers.check_golden_file( + pytestconfig, + ignore_paths=ignore_paths, + output_path=tmp_path / "iceberg_mces.json", + golden_path=test_resources_dir / "iceberg_profile_mces_golden.json", + ) diff --git a/metadata-ingestion/tests/unit/test_iceberg.py b/metadata-ingestion/tests/unit/test_iceberg.py index f3ea071d76400..cd375ea7cbc7f 100644 --- a/metadata-ingestion/tests/unit/test_iceberg.py +++ b/metadata-ingestion/tests/unit/test_iceberg.py @@ -1,13 +1,34 @@ from typing import Any, Optional import pytest -from iceberg.api import types as IcebergTypes -from iceberg.api.types.types import NestedField +from pyiceberg.schema import Schema +from pyiceberg.types import ( + BinaryType, + BooleanType, + DateType, + DecimalType, + DoubleType, + FixedType, + FloatType, + IntegerType, + ListType, + LongType, + MapType, + NestedField, + PrimitiveType, + StringType, + StructType, + TimestampType, + TimestamptzType, + TimeType, + UUIDType, +) from datahub.configuration.common import ConfigurationError from datahub.ingestion.api.common import PipelineContext from datahub.ingestion.source.azure.azure_common import AdlsSourceConfig from datahub.ingestion.source.iceberg.iceberg import IcebergSource, IcebergSourceConfig +from datahub.ingestion.source.iceberg.iceberg_common import IcebergCatalogConfig from datahub.metadata.com.linkedin.pegasus2avro.schema import ArrayType, SchemaField from datahub.metadata.schema_classes import ( ArrayTypeClass, @@ -120,7 +141,7 @@ def test_config_for_tests(): def test_config_no_filesystem(): """ - Test when a SAS token is used as an ADLS credential. + Test when no filesystem is configured. """ with pytest.raises(ConfigurationError): IcebergSource( @@ -133,10 +154,26 @@ def test_config_multiple_filesystems(): """ Test when more than 1 filesystem is configured. """ + catalog = IcebergCatalogConfig(name="default", conf={}) + adls = AdlsSourceConfig( + account_name="test", account_key="test", container_name="test" + ) + with pytest.raises(ConfigurationError): + IcebergSource( + ctx=PipelineContext(run_id="iceberg-source-test"), + config=IcebergSourceConfig(catalog=catalog, adls=adls, localfs="/tmp"), + ) with pytest.raises(ConfigurationError): - adls: AdlsSourceConfig = AdlsSourceConfig( - account_name="test", container_name="test" + IcebergSource( + ctx=PipelineContext(run_id="iceberg-source-test"), + config=IcebergSourceConfig(catalog=catalog, adls=adls), ) + with pytest.raises(ConfigurationError): + IcebergSource( + ctx=PipelineContext(run_id="iceberg-source-test"), + config=IcebergSourceConfig(catalog=catalog, localfs="/tmp"), + ) + with pytest.raises(ConfigurationError): IcebergSource( ctx=PipelineContext(run_id="iceberg-source-test"), config=IcebergSourceConfig(adls=adls, localfs="/tmp"), @@ -146,226 +183,285 @@ def test_config_multiple_filesystems(): @pytest.mark.parametrize( "iceberg_type, expected_schema_field_type", [ - (IcebergTypes.BinaryType.get(), BytesTypeClass), - (IcebergTypes.BooleanType.get(), BooleanTypeClass), - (IcebergTypes.DateType.get(), DateTypeClass), + (BinaryType(), BytesTypeClass), + (BooleanType(), BooleanTypeClass), + (DateType(), DateTypeClass), ( - IcebergTypes.DecimalType.of(3, 2), + DecimalType(3, 2), NumberTypeClass, ), - (IcebergTypes.DoubleType.get(), NumberTypeClass), - (IcebergTypes.FixedType.of_length(4), FixedTypeClass), - (IcebergTypes.FloatType.get(), NumberTypeClass), - (IcebergTypes.IntegerType.get(), NumberTypeClass), - (IcebergTypes.LongType.get(), NumberTypeClass), - (IcebergTypes.StringType.get(), StringTypeClass), + (DoubleType(), NumberTypeClass), + (FixedType(4), FixedTypeClass), + (FloatType(), NumberTypeClass), + (IntegerType(), NumberTypeClass), + (LongType(), NumberTypeClass), + (StringType(), StringTypeClass), ( - IcebergTypes.TimestampType.with_timezone(), + TimestampType(), TimeTypeClass, ), ( - IcebergTypes.TimestampType.without_timezone(), + TimestamptzType(), TimeTypeClass, ), - (IcebergTypes.TimeType.get(), TimeTypeClass), + (TimeType(), TimeTypeClass), ( - IcebergTypes.UUIDType.get(), + UUIDType(), StringTypeClass, ), ], ) def test_iceberg_primitive_type_to_schema_field( - iceberg_type: IcebergTypes.PrimitiveType, expected_schema_field_type: Any + iceberg_type: PrimitiveType, expected_schema_field_type: Any ) -> None: """ Test converting a primitive typed Iceberg field to a SchemaField """ iceberg_source_instance = with_iceberg_source() for column in [ - NestedField.required( - 1, "required_field", iceberg_type, "required field documentation" + NestedField( + 1, "required_field", iceberg_type, True, "required field documentation" ), - NestedField.optional( - 1, "optional_field", iceberg_type, "optional field documentation" + NestedField( + 1, "optional_field", iceberg_type, False, "optional field documentation" ), ]: - schema_fields = iceberg_source_instance._get_schema_fields_for_column(column) + schema = Schema(column) + schema_fields = iceberg_source_instance._get_schema_fields_for_schema(schema) assert ( len(schema_fields) == 1 ), f"Expected 1 field, but got {len(schema_fields)}" assert_field( - schema_fields[0], column.doc, column.is_optional, expected_schema_field_type + schema_fields[0], column.doc, column.optional, expected_schema_field_type ) @pytest.mark.parametrize( "iceberg_type, expected_array_nested_type", [ - (IcebergTypes.BinaryType.get(), "bytes"), - (IcebergTypes.BooleanType.get(), "boolean"), - (IcebergTypes.DateType.get(), "date"), + (BinaryType(), "bytes"), + (BooleanType(), "boolean"), + (DateType(), "date"), ( - IcebergTypes.DecimalType.of(3, 2), + DecimalType(3, 2), "decimal", ), - (IcebergTypes.DoubleType.get(), "double"), - (IcebergTypes.FixedType.of_length(4), "fixed"), - (IcebergTypes.FloatType.get(), "float"), - (IcebergTypes.IntegerType.get(), "int"), - (IcebergTypes.LongType.get(), "long"), - (IcebergTypes.StringType.get(), "string"), + (DoubleType(), "double"), + (FixedType(4), "fixed"), + (FloatType(), "float"), + (IntegerType(), "int"), + (LongType(), "long"), + (StringType(), "string"), ( - IcebergTypes.TimestampType.with_timezone(), + TimestampType(), "timestamp-micros", ), ( - IcebergTypes.TimestampType.without_timezone(), + TimestamptzType(), "timestamp-micros", ), - (IcebergTypes.TimeType.get(), "time-micros"), + (TimeType(), "time-micros"), ( - IcebergTypes.UUIDType.get(), + UUIDType(), "uuid", ), ], ) def test_iceberg_list_to_schema_field( - iceberg_type: IcebergTypes.PrimitiveType, expected_array_nested_type: Any + iceberg_type: PrimitiveType, expected_array_nested_type: Any ) -> None: """ Test converting a list typed Iceberg field to an ArrayType SchemaField, including the list nested type. """ - list_column: NestedField = NestedField.required( - 1, - "listField", - IcebergTypes.ListType.of_required(2, iceberg_type), - "documentation", - ) - iceberg_source_instance = with_iceberg_source() - schema_fields = iceberg_source_instance._get_schema_fields_for_column(list_column) - assert len(schema_fields) == 1, f"Expected 1 field, but got {len(schema_fields)}" - assert_field( - schema_fields[0], list_column.doc, list_column.is_optional, ArrayTypeClass - ) - assert isinstance( - schema_fields[0].type.type, ArrayType - ), f"Field type {schema_fields[0].type.type} was expected to be {ArrayType}" - arrayType: ArrayType = schema_fields[0].type.type - assert arrayType.nestedType == [ - expected_array_nested_type - ], f"List Field nested type {arrayType.nestedType} was expected to be {expected_array_nested_type}" + for list_column in [ + NestedField( + 1, + "listField", + ListType(2, iceberg_type, True), + True, + "required field, required element documentation", + ), + NestedField( + 1, + "listField", + ListType(2, iceberg_type, False), + True, + "required field, optional element documentation", + ), + NestedField( + 1, + "listField", + ListType(2, iceberg_type, True), + False, + "optional field, required element documentation", + ), + NestedField( + 1, + "listField", + ListType(2, iceberg_type, False), + False, + "optional field, optional element documentation", + ), + ]: + iceberg_source_instance = with_iceberg_source() + schema = Schema(list_column) + schema_fields = iceberg_source_instance._get_schema_fields_for_schema(schema) + assert ( + len(schema_fields) == 1 + ), f"Expected 1 field, but got {len(schema_fields)}" + assert_field( + schema_fields[0], list_column.doc, list_column.optional, ArrayTypeClass + ) + assert isinstance( + schema_fields[0].type.type, ArrayType + ), f"Field type {schema_fields[0].type.type} was expected to be {ArrayType}" + arrayType: ArrayType = schema_fields[0].type.type + assert arrayType.nestedType == [ + expected_array_nested_type + ], f"List Field nested type {arrayType.nestedType} was expected to be {expected_array_nested_type}" @pytest.mark.parametrize( "iceberg_type, expected_map_type", [ - (IcebergTypes.BinaryType.get(), BytesTypeClass), - (IcebergTypes.BooleanType.get(), BooleanTypeClass), - (IcebergTypes.DateType.get(), DateTypeClass), + (BinaryType(), BytesTypeClass), + (BooleanType(), BooleanTypeClass), + (DateType(), DateTypeClass), ( - IcebergTypes.DecimalType.of(3, 2), + DecimalType(3, 2), NumberTypeClass, ), - (IcebergTypes.DoubleType.get(), NumberTypeClass), - (IcebergTypes.FixedType.of_length(4), FixedTypeClass), - (IcebergTypes.FloatType.get(), NumberTypeClass), - (IcebergTypes.IntegerType.get(), NumberTypeClass), - (IcebergTypes.LongType.get(), NumberTypeClass), - (IcebergTypes.StringType.get(), StringTypeClass), + (DoubleType(), NumberTypeClass), + (FixedType(4), FixedTypeClass), + (FloatType(), NumberTypeClass), + (IntegerType(), NumberTypeClass), + (LongType(), NumberTypeClass), + (StringType(), StringTypeClass), ( - IcebergTypes.TimestampType.with_timezone(), + TimestampType(), TimeTypeClass, ), ( - IcebergTypes.TimestampType.without_timezone(), + TimestamptzType(), TimeTypeClass, ), - (IcebergTypes.TimeType.get(), TimeTypeClass), + (TimeType(), TimeTypeClass), ( - IcebergTypes.UUIDType.get(), + UUIDType(), StringTypeClass, ), ], ) def test_iceberg_map_to_schema_field( - iceberg_type: IcebergTypes.PrimitiveType, expected_map_type: Any + iceberg_type: PrimitiveType, expected_map_type: Any ) -> None: """ Test converting a map typed Iceberg field to a MapType SchemaField, where the key is the same type as the value. """ - map_column: NestedField = NestedField.required( - 1, - "mapField", - IcebergTypes.MapType.of_required(11, 12, iceberg_type, iceberg_type), - "documentation", - ) - iceberg_source_instance = with_iceberg_source() - schema_fields = iceberg_source_instance._get_schema_fields_for_column(map_column) - # Converting an Iceberg Map type will be done by creating an array of struct(key, value) records. - # The first field will be the array. - assert len(schema_fields) == 3, f"Expected 3 fields, but got {len(schema_fields)}" - assert_field( - schema_fields[0], map_column.doc, map_column.is_optional, ArrayTypeClass - ) + for map_column in [ + NestedField( + 1, + "mapField", + MapType(11, iceberg_type, 12, iceberg_type, True), + True, + "required field, required value documentation", + ), + NestedField( + 1, + "mapField", + MapType(11, iceberg_type, 12, iceberg_type, False), + True, + "required field, optional value documentation", + ), + NestedField( + 1, + "mapField", + MapType(11, iceberg_type, 12, iceberg_type, True), + False, + "optional field, required value documentation", + ), + NestedField( + 1, + "mapField", + MapType(11, iceberg_type, 12, iceberg_type, False), + False, + "optional field, optional value documentation", + ), + ]: + iceberg_source_instance = with_iceberg_source() + schema = Schema(map_column) + schema_fields = iceberg_source_instance._get_schema_fields_for_schema(schema) + # Converting an Iceberg Map type will be done by creating an array of struct(key, value) records. + # The first field will be the array. + assert ( + len(schema_fields) == 3 + ), f"Expected 3 fields, but got {len(schema_fields)}" + assert_field( + schema_fields[0], map_column.doc, map_column.optional, ArrayTypeClass + ) - # The second field will be the key type - assert_field(schema_fields[1], None, False, expected_map_type) + # The second field will be the key type + assert_field(schema_fields[1], None, False, expected_map_type) - # The third field will be the value type - assert_field(schema_fields[2], None, True, expected_map_type) + # The third field will be the value type + assert_field( + schema_fields[2], + None, + not map_column.field_type.value_required, + expected_map_type, + ) @pytest.mark.parametrize( "iceberg_type, expected_schema_field_type", [ - (IcebergTypes.BinaryType.get(), BytesTypeClass), - (IcebergTypes.BooleanType.get(), BooleanTypeClass), - (IcebergTypes.DateType.get(), DateTypeClass), + (BinaryType(), BytesTypeClass), + (BooleanType(), BooleanTypeClass), + (DateType(), DateTypeClass), ( - IcebergTypes.DecimalType.of(3, 2), + DecimalType(3, 2), NumberTypeClass, ), - (IcebergTypes.DoubleType.get(), NumberTypeClass), - (IcebergTypes.FixedType.of_length(4), FixedTypeClass), - (IcebergTypes.FloatType.get(), NumberTypeClass), - (IcebergTypes.IntegerType.get(), NumberTypeClass), - (IcebergTypes.LongType.get(), NumberTypeClass), - (IcebergTypes.StringType.get(), StringTypeClass), + (DoubleType(), NumberTypeClass), + (FixedType(4), FixedTypeClass), + (FloatType(), NumberTypeClass), + (IntegerType(), NumberTypeClass), + (LongType(), NumberTypeClass), + (StringType(), StringTypeClass), ( - IcebergTypes.TimestampType.with_timezone(), + TimestampType(), TimeTypeClass, ), ( - IcebergTypes.TimestampType.without_timezone(), + TimestamptzType(), TimeTypeClass, ), - (IcebergTypes.TimeType.get(), TimeTypeClass), + (TimeType(), TimeTypeClass), ( - IcebergTypes.UUIDType.get(), + UUIDType(), StringTypeClass, ), ], ) def test_iceberg_struct_to_schema_field( - iceberg_type: IcebergTypes.PrimitiveType, expected_schema_field_type: Any + iceberg_type: PrimitiveType, expected_schema_field_type: Any ) -> None: """ Test converting a struct typed Iceberg field to a RecordType SchemaField. """ - field1: NestedField = NestedField.required( - 11, "field1", iceberg_type, "field documentation" - ) - struct_column: NestedField = NestedField.required( - 1, "structField", IcebergTypes.StructType.of([field1]), "struct documentation" + field1 = NestedField(11, "field1", iceberg_type, True, "field documentation") + struct_column = NestedField( + 1, "structField", StructType(field1), True, "struct documentation" ) iceberg_source_instance = with_iceberg_source() - schema_fields = iceberg_source_instance._get_schema_fields_for_column(struct_column) + schema = Schema(struct_column) + schema_fields = iceberg_source_instance._get_schema_fields_for_schema(schema) assert len(schema_fields) == 2, f"Expected 2 fields, but got {len(schema_fields)}" assert_field( - schema_fields[0], struct_column.doc, struct_column.is_optional, RecordTypeClass + schema_fields[0], struct_column.doc, struct_column.optional, RecordTypeClass ) assert_field( - schema_fields[1], field1.doc, field1.is_optional, expected_schema_field_type + schema_fields[1], field1.doc, field1.optional, expected_schema_field_type )