diff --git a/.github/workflows/test_destinations.yml b/.github/workflows/test_destinations.yml index cfd0a3bd56..84a8f95d71 100644 --- a/.github/workflows/test_destinations.yml +++ b/.github/workflows/test_destinations.yml @@ -77,8 +77,13 @@ jobs: # key: venv-${{ runner.os }}-${{ steps.setup-python.outputs.python-version }}-${{ hashFiles('**/poetry.lock') }}-redshift - name: Install dependencies - # if: steps.cached-poetry-dependencies.outputs.cache-hit != 'true' - run: poetry install --no-interaction -E redshift -E postgis -E postgres -E gs -E s3 -E az -E parquet -E duckdb -E cli -E filesystem --with sentry-sdk --with pipeline,ibis -E deltalake + run: poetry install --no-interaction -E redshift -E postgis -E postgres -E gs -E s3 -E az -E parquet -E duckdb -E cli -E filesystem --with sentry-sdk --with pipeline,ibis -E deltalake -E pyiceberg + + - name: enable certificates for azure and duckdb + run: sudo mkdir -p /etc/pki/tls/certs && sudo ln -s /etc/ssl/certs/ca-certificates.crt /etc/pki/tls/certs/ca-bundle.crt + + - name: Upgrade sqlalchemy + run: poetry run pip install sqlalchemy==2.0.18 # minimum version required by `pyiceberg` - name: create secrets.toml run: pwd && echo "$DLT_SECRETS_TOML" > tests/.dlt/secrets.toml diff --git a/.github/workflows/test_local_destinations.yml b/.github/workflows/test_local_destinations.yml index 6f44e5fd5a..706bae1b0c 100644 --- a/.github/workflows/test_local_destinations.yml +++ b/.github/workflows/test_local_destinations.yml @@ -95,7 +95,10 @@ jobs: key: venv-${{ runner.os }}-${{ steps.setup-python.outputs.python-version }}-${{ hashFiles('**/poetry.lock') }}-local-destinations - name: Install dependencies - run: poetry install --no-interaction -E postgres -E postgis -E duckdb -E parquet -E filesystem -E cli -E weaviate -E qdrant -E sftp --with sentry-sdk --with pipeline,ibis -E deltalake + run: poetry install --no-interaction -E postgres -E postgis -E duckdb -E parquet -E filesystem -E cli -E weaviate -E qdrant -E sftp --with sentry-sdk --with pipeline,ibis -E deltalake -E pyiceberg + + - name: Upgrade sqlalchemy + run: poetry run pip install sqlalchemy==2.0.18 # minimum version required by `pyiceberg` - name: Start SFTP server run: docker compose -f "tests/load/filesystem_sftp/docker-compose.yml" up -d diff --git a/Makefile b/Makefile index 2a7f6dac0a..0ca8a2e0c3 100644 --- a/Makefile +++ b/Makefile @@ -44,7 +44,7 @@ has-poetry: poetry --version dev: has-poetry - poetry install --all-extras --with docs,providers,pipeline,sources,sentry-sdk,airflow + poetry install --all-extras --with docs,providers,pipeline,sources,sentry-sdk lint: ./tools/check-package.sh diff --git a/dlt/cli/source_detection.py b/dlt/cli/source_detection.py index 7067f8b896..0769605d01 100644 --- a/dlt/cli/source_detection.py +++ b/dlt/cli/source_detection.py @@ -29,8 +29,7 @@ def find_call_arguments_to_replace( if not isinstance(dn_node, ast.Constant) or not isinstance(dn_node.value, str): raise CliCommandInnerException( "init", - f"The pipeline script {init_script_name} must pass the {t_arg_name} as" - f" string to '{arg_name}' function in line {dn_node.lineno}", + f"The pipeline script {init_script_name} must pass the {t_arg_name} as string to '{arg_name}' function in line {dn_node.lineno}", # type: ignore[attr-defined] ) else: transformed_nodes.append((dn_node, ast.Constant(value=t_value, kind=None))) diff --git a/dlt/common/configuration/specs/aws_credentials.py b/dlt/common/configuration/specs/aws_credentials.py index 5f69be6a33..a75cd85225 100644 --- a/dlt/common/configuration/specs/aws_credentials.py +++ b/dlt/common/configuration/specs/aws_credentials.py @@ -8,6 +8,7 @@ CredentialsWithDefault, configspec, ) +from dlt.common.configuration.specs.mixins import WithObjectStoreRsCredentials, WithPyicebergConfig from dlt.common.configuration.specs.exceptions import ( InvalidBoto3Session, ObjectStoreRsCredentialsException, @@ -16,7 +17,9 @@ @configspec -class AwsCredentialsWithoutDefaults(CredentialsConfiguration): +class AwsCredentialsWithoutDefaults( + CredentialsConfiguration, WithObjectStoreRsCredentials, WithPyicebergConfig +): # credentials without boto implementation aws_access_key_id: str = None aws_secret_access_key: TSecretStrValue = None @@ -77,6 +80,16 @@ def to_object_store_rs_credentials(self) -> Dict[str, str]: return creds + def to_pyiceberg_fileio_config(self) -> Dict[str, Any]: + return { + "s3.access-key-id": self.aws_access_key_id, + "s3.secret-access-key": self.aws_secret_access_key, + "s3.session-token": self.aws_session_token, + "s3.region": self.region_name, + "s3.endpoint": self.endpoint_url, + "s3.connect-timeout": 300, + } + @configspec class AwsCredentials(AwsCredentialsWithoutDefaults, CredentialsWithDefault): diff --git a/dlt/common/configuration/specs/azure_credentials.py b/dlt/common/configuration/specs/azure_credentials.py index cf6ec493de..aabd0b471a 100644 --- a/dlt/common/configuration/specs/azure_credentials.py +++ b/dlt/common/configuration/specs/azure_credentials.py @@ -8,6 +8,7 @@ CredentialsWithDefault, configspec, ) +from dlt.common.configuration.specs.mixins import WithObjectStoreRsCredentials, WithPyicebergConfig from dlt import version from dlt.common.utils import without_none @@ -15,7 +16,7 @@ @configspec -class AzureCredentialsBase(CredentialsConfiguration): +class AzureCredentialsBase(CredentialsConfiguration, WithObjectStoreRsCredentials): azure_storage_account_name: str = None azure_account_host: Optional[str] = None """Alternative host when accessing blob storage endpoint ie. my_account.dfs.core.windows.net""" @@ -32,7 +33,7 @@ def to_object_store_rs_credentials(self) -> Dict[str, str]: @configspec -class AzureCredentialsWithoutDefaults(AzureCredentialsBase): +class AzureCredentialsWithoutDefaults(AzureCredentialsBase, WithPyicebergConfig): """Credentials for Azure Blob Storage, compatible with adlfs""" azure_storage_account_key: Optional[TSecretStrValue] = None @@ -49,6 +50,13 @@ def to_adlfs_credentials(self) -> Dict[str, Any]: account_host=self.azure_account_host, ) + def to_pyiceberg_fileio_config(self) -> Dict[str, Any]: + return { + "adlfs.account-name": self.azure_storage_account_name, + "adlfs.account-key": self.azure_storage_account_key, + "adlfs.sas-token": self.azure_storage_sas_token, + } + def create_sas_token(self) -> None: try: from azure.storage.blob import generate_account_sas, ResourceTypes @@ -72,7 +80,7 @@ def on_partial(self) -> None: @configspec -class AzureServicePrincipalCredentialsWithoutDefaults(AzureCredentialsBase): +class AzureServicePrincipalCredentialsWithoutDefaults(AzureCredentialsBase, WithPyicebergConfig): azure_tenant_id: str = None azure_client_id: str = None azure_client_secret: TSecretStrValue = None @@ -86,6 +94,14 @@ def to_adlfs_credentials(self) -> Dict[str, Any]: client_secret=self.azure_client_secret, ) + def to_pyiceberg_fileio_config(self) -> Dict[str, Any]: + return { + "adlfs.account-name": self.azure_storage_account_name, + "adlfs.tenant-id": self.azure_tenant_id, + "adlfs.client-id": self.azure_client_id, + "adlfs.client-secret": self.azure_client_secret, + } + @configspec class AzureCredentials(AzureCredentialsWithoutDefaults, CredentialsWithDefault): diff --git a/dlt/common/configuration/specs/base_configuration.py b/dlt/common/configuration/specs/base_configuration.py index 8d913d0542..41d1d7a0ca 100644 --- a/dlt/common/configuration/specs/base_configuration.py +++ b/dlt/common/configuration/specs/base_configuration.py @@ -359,7 +359,7 @@ def _get_resolvable_dataclass_fields(cls) -> Iterator[TDtcField]: def get_resolvable_fields(cls) -> Dict[str, type]: """Returns a mapping of fields to their type hints. Dunders should not be resolved and are not returned""" return { - f.name: eval(f.type) if isinstance(f.type, str) else f.type # type: ignore[arg-type] + f.name: eval(f.type) if isinstance(f.type, str) else f.type for f in cls._get_resolvable_dataclass_fields() } diff --git a/dlt/common/configuration/specs/config_providers_context.py b/dlt/common/configuration/specs/config_providers_context.py index 5d1a5b7f26..a244ab571f 100644 --- a/dlt/common/configuration/specs/config_providers_context.py +++ b/dlt/common/configuration/specs/config_providers_context.py @@ -1,5 +1,4 @@ import contextlib -import dataclasses import io from typing import ClassVar, List @@ -8,10 +7,6 @@ ConfigProvider, ContextProvider, ) -from dlt.common.configuration.specs.base_configuration import ( - ContainerInjectableContext, - NotResolved, -) from dlt.common.configuration.specs import ( GcpServiceAccountCredentials, BaseConfiguration, @@ -137,7 +132,7 @@ def _airflow_providers() -> List[ConfigProvider]: # check if we are in task context and provide more info from airflow.operators.python import get_current_context # noqa - ti: TaskInstance = get_current_context()["ti"] # type: ignore + ti: TaskInstance = get_current_context()["ti"] # type: ignore[assignment,unused-ignore] # log outside of stderr/out redirect if secrets_toml_var is None: diff --git a/dlt/common/configuration/specs/exceptions.py b/dlt/common/configuration/specs/exceptions.py index 928e46a8a0..fe87ef24d7 100644 --- a/dlt/common/configuration/specs/exceptions.py +++ b/dlt/common/configuration/specs/exceptions.py @@ -72,3 +72,7 @@ def __init__(self, spec: Type[Any], native_value: Any): class ObjectStoreRsCredentialsException(ConfigurationException): pass + + +class UnsupportedAuthenticationMethodException(ConfigurationException): + pass diff --git a/dlt/common/configuration/specs/gcp_credentials.py b/dlt/common/configuration/specs/gcp_credentials.py index 60ab1d4b56..17519b032a 100644 --- a/dlt/common/configuration/specs/gcp_credentials.py +++ b/dlt/common/configuration/specs/gcp_credentials.py @@ -11,7 +11,9 @@ InvalidGoogleServicesJson, NativeValueError, OAuth2ScopesRequired, + UnsupportedAuthenticationMethodException, ) +from dlt.common.configuration.specs.mixins import WithObjectStoreRsCredentials, WithPyicebergConfig from dlt.common.exceptions import MissingDependencyException from dlt.common.typing import DictStrAny, TSecretStrValue, StrAny from dlt.common.configuration.specs.base_configuration import ( @@ -23,7 +25,7 @@ @configspec -class GcpCredentials(CredentialsConfiguration): +class GcpCredentials(CredentialsConfiguration, WithObjectStoreRsCredentials, WithPyicebergConfig): token_uri: Final[str] = dataclasses.field( default="https://oauth2.googleapis.com/token", init=False, repr=False, compare=False ) @@ -126,6 +128,12 @@ def to_native_credentials(self) -> Any: else: return ServiceAccountCredentials.from_service_account_info(self) + def to_pyiceberg_fileio_config(self) -> Dict[str, Any]: + raise UnsupportedAuthenticationMethodException( + "Service Account authentication not supported with `iceberg` table format. Use OAuth" + " authentication instead." + ) + def __str__(self) -> str: return f"{self.client_email}@{self.project_id}" @@ -176,11 +184,19 @@ def to_native_representation(self) -> str: return json.dumps(self._info_dict()) def to_object_store_rs_credentials(self) -> Dict[str, str]: - raise NotImplementedError( - "`object_store` Rust crate does not support OAuth for GCP credentials. Reference:" - " https://docs.rs/object_store/latest/object_store/gcp." + raise UnsupportedAuthenticationMethodException( + "OAuth authentication not supported with `delta` table format. Use Service Account or" + " Application Default Credentials authentication instead." ) + def to_pyiceberg_fileio_config(self) -> Dict[str, Any]: + self.auth() + return { + "gcs.project-id": self.project_id, + "gcs.oauth2.token": self.token, + "gcs.oauth2.token-expires-at": (pendulum.now().timestamp() + 60) * 1000, + } + def auth(self, scopes: Union[str, List[str]] = None, redirect_url: str = None) -> None: if not self.refresh_token: self.add_scopes(scopes) @@ -313,6 +329,12 @@ def to_native_credentials(self) -> Any: else: return super().to_native_credentials() + def to_pyiceberg_fileio_config(self) -> Dict[str, Any]: + raise UnsupportedAuthenticationMethodException( + "Application Default Credentials authentication not supported with `iceberg` table" + " format. Use OAuth authentication instead." + ) + @configspec class GcpServiceAccountCredentials( @@ -334,3 +356,9 @@ def parse_native_representation(self, native_value: Any) -> None: except NativeValueError: pass GcpOAuthCredentialsWithoutDefaults.parse_native_representation(self, native_value) + + def to_pyiceberg_fileio_config(self) -> Dict[str, Any]: + if self.has_default_credentials(): + return GcpDefaultCredentials.to_pyiceberg_fileio_config(self) + else: + return GcpOAuthCredentialsWithoutDefaults.to_pyiceberg_fileio_config(self) diff --git a/dlt/common/configuration/specs/mixins.py b/dlt/common/configuration/specs/mixins.py new file mode 100644 index 0000000000..2f843aee5b --- /dev/null +++ b/dlt/common/configuration/specs/mixins.py @@ -0,0 +1,24 @@ +from typing import Dict, Any +from abc import abstractmethod, ABC + + +class WithObjectStoreRsCredentials(ABC): + @abstractmethod + def to_object_store_rs_credentials(self) -> Dict[str, Any]: + """Returns credentials dictionary for object_store Rust crate. + + Can be used for libraries that build on top of the object_store crate, such as `deltalake`. + + https://docs.rs/object_store/latest/object_store/ + """ + pass + + +class WithPyicebergConfig(ABC): + @abstractmethod + def to_pyiceberg_fileio_config(self) -> Dict[str, Any]: + """Returns `pyiceberg` FileIO configuration dictionary. + + https://py.iceberg.apache.org/configuration/#fileio + """ + pass diff --git a/dlt/common/data_writers/buffered.py b/dlt/common/data_writers/buffered.py index e2b6c9a442..6ef431a4d0 100644 --- a/dlt/common/data_writers/buffered.py +++ b/dlt/common/data_writers/buffered.py @@ -242,7 +242,7 @@ def _flush_items(self, allow_empty_file: bool = False) -> None: if self.writer_spec.is_binary_format: self._file = self.open(self._file_name, "wb") # type: ignore else: - self._file = self.open(self._file_name, "wt", encoding="utf-8", newline="") # type: ignore + self._file = self.open(self._file_name, "wt", encoding="utf-8", newline="") self._writer = self.writer_cls(self._file, caps=self._caps) # type: ignore[assignment] self._writer.write_header(self._current_columns) # write buffer diff --git a/dlt/common/destination/utils.py b/dlt/common/destination/utils.py index 0bad5b152e..c98344b687 100644 --- a/dlt/common/destination/utils.py +++ b/dlt/common/destination/utils.py @@ -38,7 +38,7 @@ def verify_schema_capabilities( exception_log: List[Exception] = [] # combined casing function case_identifier = lambda ident: capabilities.casefold_identifier( - (str if capabilities.has_case_sensitive_identifiers else str.casefold)(ident) # type: ignore + (str if capabilities.has_case_sensitive_identifiers else str.casefold)(ident) ) table_name_lookup: DictStrStr = {} # name collision explanation diff --git a/dlt/common/libs/deltalake.py b/dlt/common/libs/deltalake.py index 4047bc3a1a..0f938e7102 100644 --- a/dlt/common/libs/deltalake.py +++ b/dlt/common/libs/deltalake.py @@ -10,6 +10,7 @@ from dlt.common.exceptions import MissingDependencyException from dlt.common.storages import FilesystemConfiguration from dlt.common.utils import assert_min_pkg_version +from dlt.common.configuration.specs.mixins import WithObjectStoreRsCredentials from dlt.destinations.impl.filesystem.filesystem import FilesystemClient try: @@ -191,10 +192,9 @@ def get_delta_tables( def _deltalake_storage_options(config: FilesystemConfiguration) -> Dict[str, str]: """Returns dict that can be passed as `storage_options` in `deltalake` library.""" - creds = {} # type: ignore + creds = {} extra_options = {} - # TODO: create a mixin with to_object_store_rs_credentials for a proper discovery - if hasattr(config.credentials, "to_object_store_rs_credentials"): + if isinstance(config.credentials, WithObjectStoreRsCredentials): creds = config.credentials.to_object_store_rs_credentials() if config.deltalake_storage_options is not None: extra_options = config.deltalake_storage_options diff --git a/dlt/common/libs/pyiceberg.py b/dlt/common/libs/pyiceberg.py new file mode 100644 index 0000000000..19ce9abbf2 --- /dev/null +++ b/dlt/common/libs/pyiceberg.py @@ -0,0 +1,192 @@ +from typing import Dict, Any, List, Optional + +from dlt import version, Pipeline +from dlt.common.libs.pyarrow import cast_arrow_schema_types +from dlt.common.schema.typing import TWriteDisposition +from dlt.common.utils import assert_min_pkg_version +from dlt.common.exceptions import MissingDependencyException +from dlt.common.storages.configuration import FileSystemCredentials +from dlt.common.configuration.specs import CredentialsConfiguration +from dlt.common.configuration.specs.mixins import WithPyicebergConfig +from dlt.destinations.impl.filesystem.filesystem import FilesystemClient + + +try: + from pyiceberg.table import Table as IcebergTable + from pyiceberg.catalog import MetastoreCatalog + import pyarrow as pa +except ModuleNotFoundError: + raise MissingDependencyException( + "dlt pyiceberg helpers", + [f"{version.DLT_PKG_NAME}[pyiceberg]"], + "Install `pyiceberg` so dlt can create Iceberg tables in the `filesystem` destination.", + ) + + +def ensure_iceberg_compatible_arrow_schema(schema: pa.Schema) -> pa.Schema: + ARROW_TO_ICEBERG_COMPATIBLE_ARROW_TYPE_MAP = { + pa.types.is_time: pa.string(), + pa.types.is_decimal256: pa.string(), # pyarrow does not allow downcasting to decimal128 + } + return cast_arrow_schema_types(schema, ARROW_TO_ICEBERG_COMPATIBLE_ARROW_TYPE_MAP) + + +def ensure_iceberg_compatible_arrow_data(data: pa.Table) -> pa.Table: + schema = ensure_iceberg_compatible_arrow_schema(data.schema) + return data.cast(schema) + + +def write_iceberg_table( + table: IcebergTable, + data: pa.Table, + write_disposition: TWriteDisposition, +) -> None: + if write_disposition == "append": + table.append(ensure_iceberg_compatible_arrow_data(data)) + elif write_disposition == "replace": + table.overwrite(ensure_iceberg_compatible_arrow_data(data)) + + +def get_sql_catalog(credentials: FileSystemCredentials) -> "SqlCatalog": # type: ignore[name-defined] # noqa: F821 + assert_min_pkg_version( + pkg_name="sqlalchemy", + version="2.0.18", + msg=( + "`sqlalchemy>=2.0.18` is needed for `iceberg` table format on `filesystem` destination." + ), + ) + + from pyiceberg.catalog.sql import SqlCatalog + + return SqlCatalog( + "default", + uri="sqlite:///:memory:", + **_get_fileio_config(credentials), + ) + + +def create_or_evolve_table( + catalog: MetastoreCatalog, + client: FilesystemClient, + table_name: str, + namespace_name: Optional[str] = None, + schema: Optional[pa.Schema] = None, + partition_columns: Optional[List[str]] = None, +) -> MetastoreCatalog: + # add table to catalog + table_id = f"{namespace_name}.{table_name}" + table_path = f"{client.dataset_path}/{table_name}" + metadata_path = f"{table_path}/metadata" + if client.fs_client.exists(metadata_path): + # found metadata; register existing table + table = _register_table(table_id, metadata_path, catalog, client) + + # evolve schema + if schema is not None: + with table.update_schema() as update: + update.union_by_name(ensure_iceberg_compatible_arrow_schema(schema)) + else: + # found no metadata; create new table + assert schema is not None + with catalog.create_table_transaction( + table_id, + schema=ensure_iceberg_compatible_arrow_schema(schema), + location=_make_path(table_path, client), + ) as txn: + # add partitioning + with txn.update_spec() as update_spec: + for col in partition_columns: + update_spec.add_identity(col) + + return catalog + + +def get_catalog( + client: FilesystemClient, + table_name: str, + namespace_name: Optional[str] = None, + schema: Optional[pa.Schema] = None, + partition_columns: Optional[List[str]] = None, +) -> MetastoreCatalog: + """Returns single-table, ephemeral, in-memory Iceberg catalog.""" + + # create in-memory catalog + catalog: MetastoreCatalog = get_sql_catalog(client.config.credentials) + + # create namespace + if namespace_name is None: + namespace_name = client.dataset_name + catalog.create_namespace(namespace_name) + + # add table to catalog + catalog = create_or_evolve_table( + catalog=catalog, + client=client, + table_name=table_name, + namespace_name=namespace_name, + schema=schema, + partition_columns=partition_columns, + ) + + return catalog + + +def get_iceberg_tables( + pipeline: Pipeline, *tables: str, schema_name: Optional[str] = None +) -> Dict[str, IcebergTable]: + from dlt.common.schema.utils import get_table_format + + with pipeline.destination_client(schema_name=schema_name) as client: + assert isinstance( + client, FilesystemClient + ), "The `get_iceberg_tables` function requires a `filesystem` destination." + + schema_iceberg_tables = [ + t["name"] + for t in client.schema.tables.values() + if get_table_format(client.schema.tables, t["name"]) == "iceberg" + ] + if len(tables) > 0: + invalid_tables = set(tables) - set(schema_iceberg_tables) + if len(invalid_tables) > 0: + available_schemas = "" + if len(pipeline.schema_names) > 1: + available_schemas = f" Available schemas are {pipeline.schema_names}" + raise ValueError( + f"Schema {client.schema.name} does not contain Iceberg tables with these names:" + f" {', '.join(invalid_tables)}.{available_schemas}" + ) + schema_iceberg_tables = [t for t in schema_iceberg_tables if t in tables] + + return { + name: get_catalog(client, name).load_table(f"{pipeline.dataset_name}.{name}") + for name in schema_iceberg_tables + } + + +def _get_fileio_config(credentials: CredentialsConfiguration) -> Dict[str, Any]: + if isinstance(credentials, WithPyicebergConfig): + return credentials.to_pyiceberg_fileio_config() + return {} + + +def _get_last_metadata_file(metadata_path: str, client: FilesystemClient) -> str: + # TODO: implement faster way to obtain `last_metadata_file` (listing is slow) + metadata_files = [f for f in client.fs_client.ls(metadata_path) if f.endswith(".json")] + return _make_path(sorted(metadata_files)[-1], client) + + +def _register_table( + identifier: str, + metadata_path: str, + catalog: MetastoreCatalog, + client: FilesystemClient, +) -> IcebergTable: + last_metadata_file = _get_last_metadata_file(metadata_path, client) + return catalog.register_table(identifier, last_metadata_file) + + +def _make_path(path: str, client: FilesystemClient) -> str: + # don't use file protocol for local files because duckdb does not support it + # https://github.com/duckdb/duckdb/issues/13669 + return path if client.is_local_filesystem else client.config.make_url(path) diff --git a/dlt/common/logger.py b/dlt/common/logger.py index b163c15672..634e305805 100644 --- a/dlt/common/logger.py +++ b/dlt/common/logger.py @@ -47,7 +47,7 @@ def is_logging() -> bool: def log_level() -> str: if not LOGGER: raise RuntimeError("Logger not initialized") - return logging.getLevelName(LOGGER.level) # type: ignore + return logging.getLevelName(LOGGER.level) def is_json_logging(log_format: str) -> bool: diff --git a/dlt/common/metrics.py b/dlt/common/metrics.py index d6acf19d0d..2f9f574dd0 100644 --- a/dlt/common/metrics.py +++ b/dlt/common/metrics.py @@ -9,7 +9,7 @@ class DataWriterMetrics(NamedTuple): created: float last_modified: float - def __add__(self, other: Tuple[object, ...], /) -> Tuple[object, ...]: + def __add__(self, other: Tuple[object, ...], /) -> Tuple[object, ...]: # type: ignore[override] if isinstance(other, DataWriterMetrics): return DataWriterMetrics( self.file_path if self.file_path == other.file_path else "", diff --git a/dlt/common/reflection/utils.py b/dlt/common/reflection/utils.py index c612c5a4f1..27c7bd8758 100644 --- a/dlt/common/reflection/utils.py +++ b/dlt/common/reflection/utils.py @@ -90,24 +90,24 @@ def rewrite_python_script( last_line = -1 last_offset = -1 # sort transformed nodes by line and offset - for node, t_value in sorted(transformed_nodes, key=lambda n: (n[0].lineno, n[0].col_offset)): + for node, t_value in sorted(transformed_nodes, key=lambda n: (n[0].lineno, n[0].col_offset)): # type: ignore[attr-defined] # do we have a line changed - if last_line != node.lineno - 1: + if last_line != node.lineno - 1: # type: ignore[attr-defined] # add remainder from the previous line if last_offset >= 0: script_lines.append(source_script_lines[last_line][last_offset:]) # add all new lines from previous line to current - script_lines.extend(source_script_lines[last_line + 1 : node.lineno - 1]) + script_lines.extend(source_script_lines[last_line + 1 : node.lineno - 1]) # type: ignore[attr-defined] # add trailing characters until node in current line starts - script_lines.append(source_script_lines[node.lineno - 1][: node.col_offset]) + script_lines.append(source_script_lines[node.lineno - 1][: node.col_offset]) # type: ignore[attr-defined] elif last_offset >= 0: # no line change, add the characters from the end of previous node to the current - script_lines.append(source_script_lines[last_line][last_offset : node.col_offset]) + script_lines.append(source_script_lines[last_line][last_offset : node.col_offset]) # type: ignore[attr-defined] # replace node value script_lines.append(ast_unparse(t_value).strip()) - last_line = node.end_lineno - 1 - last_offset = node.end_col_offset + last_line = node.end_lineno - 1 # type: ignore[attr-defined] + last_offset = node.end_col_offset # type: ignore[attr-defined] # add all that was missing if last_offset >= 0: diff --git a/dlt/common/schema/schema.py b/dlt/common/schema/schema.py index d6031a08fa..276bbe9c09 100644 --- a/dlt/common/schema/schema.py +++ b/dlt/common/schema/schema.py @@ -525,7 +525,7 @@ def get_new_table_columns( Typically they come from the destination schema. Columns that are in `existing_columns` and not in `table_name` columns are ignored. Optionally includes incomplete columns (without data type)""" - casefold_f: Callable[[str], str] = str.casefold if not case_sensitive else str # type: ignore[assignment] + casefold_f: Callable[[str], str] = str.casefold if not case_sensitive else str casefold_existing = { casefold_f(col_name): col for col_name, col in existing_columns.items() } diff --git a/dlt/common/typing.py b/dlt/common/typing.py index a3364d1b07..8986d753f3 100644 --- a/dlt/common/typing.py +++ b/dlt/common/typing.py @@ -446,7 +446,7 @@ def get_generic_type_argument_from_instance( if cls_: orig_param_type = get_args(cls_)[0] if orig_param_type in (Any, CallableAny) and sample_value is not None: - orig_param_type = type(sample_value) + orig_param_type = type(sample_value) # type: ignore[assignment] return orig_param_type # type: ignore diff --git a/dlt/destinations/impl/filesystem/factory.py b/dlt/destinations/impl/filesystem/factory.py index 2463da58fa..906bd157e4 100644 --- a/dlt/destinations/impl/filesystem/factory.py +++ b/dlt/destinations/impl/filesystem/factory.py @@ -19,7 +19,7 @@ def filesystem_loader_file_format_selector( *, table_schema: TTableSchema, ) -> t.Tuple[TLoaderFileFormat, t.Sequence[TLoaderFileFormat]]: - if table_schema.get("table_format") == "delta": + if table_schema.get("table_format") in ("delta", "iceberg"): return ("parquet", ["parquet"]) return (preferred_loader_file_format, supported_loader_file_formats) @@ -43,7 +43,7 @@ def _raw_capabilities(self) -> DestinationCapabilitiesContext: caps = DestinationCapabilitiesContext.generic_capabilities( preferred_loader_file_format="jsonl", loader_file_format_selector=filesystem_loader_file_format_selector, - supported_table_formats=["delta"], + supported_table_formats=["delta", "iceberg"], supported_merge_strategies=["upsert"], merge_strategies_selector=filesystem_merge_strategies_selector, ) diff --git a/dlt/destinations/impl/filesystem/filesystem.py b/dlt/destinations/impl/filesystem/filesystem.py index 1739c87fb3..ccf764811b 100644 --- a/dlt/destinations/impl/filesystem/filesystem.py +++ b/dlt/destinations/impl/filesystem/filesystem.py @@ -119,16 +119,27 @@ def metrics(self) -> Optional[LoadJobMetrics]: return m._replace(remote_url=self.make_remote_url()) -class DeltaLoadFilesystemJob(FilesystemLoadJob): +class TableFormatLoadFilesystemJob(FilesystemLoadJob): def __init__(self, file_path: str) -> None: super().__init__(file_path=file_path) self.file_paths = ReferenceFollowupJobRequest.resolve_references(self._file_path) def make_remote_path(self) -> str: - # remote path is table dir - delta will create its file structure inside it return self._job_client.get_table_dir(self.load_table_name) + @property + def arrow_dataset(self) -> Any: + from dlt.common.libs.pyarrow import pyarrow + + return pyarrow.dataset.dataset(self.file_paths) + + @property + def _partition_columns(self) -> List[str]: + return get_columns_names_with_prop(self._load_table, "partition") + + +class DeltaLoadFilesystemJob(TableFormatLoadFilesystemJob): def run(self) -> None: # create Arrow dataset from Parquet files from dlt.common.libs.pyarrow import pyarrow as pa @@ -138,7 +149,7 @@ def run(self) -> None: f"Will copy file(s) {self.file_paths} to delta table {self.make_remote_url()} [arrow" f" buffer: {pa.total_allocated_bytes()}]" ) - source_ds = pa.dataset.dataset(self.file_paths) + source_ds = self.arrow_dataset delta_table = self._delta_table() # explicitly check if there is data @@ -148,9 +159,6 @@ def run(self) -> None: else: with source_ds.scanner().to_reader() as arrow_rbr: # RecordBatchReader if self._load_table["write_disposition"] == "merge" and delta_table is not None: - self._load_table["x-merge-strategy"] = resolve_merge_strategy( # type: ignore[typeddict-unknown-key] - self._schema.tables, self._load_table, self._job_client.capabilities - ) merge_delta_table( table=delta_table, data=arrow_rbr, @@ -188,10 +196,6 @@ def _delta_table(self) -> Optional["DeltaTable"]: # type: ignore[name-defined] else: return None - @property - def _partition_columns(self) -> List[str]: - return get_columns_names_with_prop(self._load_table, "partition") - def _create_or_evolve_delta_table(self, arrow_ds: "Dataset", delta_table: "DeltaTable") -> "DeltaTable": # type: ignore[name-defined] # noqa: F821 from dlt.common.libs.deltalake import ( DeltaTable, @@ -211,13 +215,36 @@ def _create_or_evolve_delta_table(self, arrow_ds: "Dataset", delta_table: "Delta return _evolve_delta_table_schema(delta_table, arrow_ds.schema) +class IcebergLoadFilesystemJob(TableFormatLoadFilesystemJob): + def run(self) -> None: + from dlt.common.libs.pyiceberg import write_iceberg_table + + write_iceberg_table( + table=self._iceberg_table(), + data=self.arrow_dataset.to_table(), + write_disposition=self._load_table["write_disposition"], + ) + + def _iceberg_table(self) -> "pyiceberg.table.Table": # type: ignore[name-defined] # noqa: F821 + from dlt.common.libs.pyiceberg import get_catalog + + catalog = get_catalog( + client=self._job_client, + table_name=self.load_table_name, + schema=self.arrow_dataset.schema, + partition_columns=self._partition_columns, + ) + return catalog.load_table(self.table_identifier) + + @property + def table_identifier(self) -> str: + return f"{self._job_client.dataset_name}.{self.load_table_name}" + + class FilesystemLoadJobWithFollowup(HasFollowupJobs, FilesystemLoadJob): def create_followup_jobs(self, final_state: TLoadJobState) -> List[FollowupJobRequest]: jobs = super().create_followup_jobs(final_state) - if self._load_table.get("table_format") == "delta": - # delta table jobs only require table chain followup jobs - pass - elif final_state == "completed": + if final_state == "completed": ref_job = ReferenceFollowupJobRequest( original_file_name=self.file_name(), remote_paths=[self._job_client.make_remote_url(self.make_remote_path())], @@ -394,6 +421,13 @@ def prepare_load_table(self, table_name: str) -> PreparedTableSchema: if table["write_disposition"] == "merge": table["write_disposition"] = "append" table.pop("table_format", None) + merge_strategy = resolve_merge_strategy(self.schema.tables, table, self.capabilities) + if table["write_disposition"] == "merge": + if merge_strategy is None: + # no supported merge strategies, fall back to append + table["write_disposition"] = "append" + else: + table["x-merge-strategy"] = merge_strategy # type: ignore[typeddict-unknown-key] return table def get_table_dir(self, table_name: str, remote: bool = False) -> str: @@ -458,12 +492,20 @@ def create_load_job( # where we want to load the state the regular way if table["name"] == self.schema.state_table_name and not self.config.as_staging_destination: return FinalizedLoadJob(file_path) - if table.get("table_format") == "delta": - import dlt.common.libs.deltalake # assert dependencies are installed + table_format = table.get("table_format") + if table_format in ("delta", "iceberg"): # a reference job for a delta table indicates a table chain followup job if ReferenceFollowupJobRequest.is_reference_job(file_path): - return DeltaLoadFilesystemJob(file_path) + if table_format == "delta": + import dlt.common.libs.deltalake + + return DeltaLoadFilesystemJob(file_path) + elif table_format == "iceberg": + import dlt.common.libs.pyiceberg + + return IcebergLoadFilesystemJob(file_path) + # otherwise just continue return FinalizedLoadJobWithFollowupJobs(file_path) @@ -494,10 +536,10 @@ def should_load_data_to_staging_dataset(self, table_name: str) -> bool: def should_truncate_table_before_load(self, table_name: str) -> bool: table = self.prepare_load_table(table_name) - return ( - table["write_disposition"] == "replace" - and not table.get("table_format") == "delta" # Delta can do a logical replace - ) + return table["write_disposition"] == "replace" and not table.get("table_format") in ( + "delta", + "iceberg", + ) # Delta/Iceberg can do a logical replace # # state stuff @@ -718,7 +760,7 @@ def create_table_chain_completed_followup_jobs( jobs = super().create_table_chain_completed_followup_jobs( table_chain, completed_table_chain_jobs ) - if table_chain[0].get("table_format") == "delta": + if table_chain[0].get("table_format") in ("delta", "iceberg"): for table in table_chain: table_job_paths = [ job.file_path diff --git a/dlt/destinations/impl/filesystem/sql_client.py b/dlt/destinations/impl/filesystem/sql_client.py index d03a00b418..d39f4c3431 100644 --- a/dlt/destinations/impl/filesystem/sql_client.py +++ b/dlt/destinations/impl/filesystem/sql_client.py @@ -13,6 +13,7 @@ from dlt.common.destination.reference import DBApiCursor +from dlt.common.storages.fsspec_filesystem import AZURE_BLOB_STORAGE_PROTOCOLS from dlt.destinations.sql_client import raise_database_error from dlt.destinations.impl.duckdb.sql_client import DuckDbSqlClient @@ -169,8 +170,9 @@ def create_authentication(self, persistent: bool = False, secret_name: str = Non # native google storage implementation is not supported.. elif self.fs_client.config.protocol in ["gs", "gcs"]: logger.warn( - "For gs/gcs access via duckdb please use the gs/gcs s3 compatibility layer. Falling" - " back to fsspec." + "For gs/gcs access via duckdb please use the gs/gcs s3 compatibility layer if" + " possible (not supported when using `iceberg` table format). Falling back to" + " fsspec." ) self._conn.register_filesystem(self.fs_client.fs_client) @@ -192,7 +194,7 @@ def open_connection(self) -> duckdb.DuckDBPyConnection: # the line below solves problems with certificate path lookup on linux # see duckdb docs - if self.fs_client.config.protocol in ["az", "abfss"]: + if self.fs_client.config.protocol in AZURE_BLOB_STORAGE_PROTOCOLS: self._conn.sql("SET azure_transport_option_type = 'curl';") return self._conn @@ -258,6 +260,13 @@ def create_views_for_tables(self, tables: Dict[str, str]) -> None: from_statement = "" if schema_table.get("table_format") == "delta": from_statement = f"delta_scan('{resolved_folder}')" + elif schema_table.get("table_format") == "iceberg": + from dlt.common.libs.pyiceberg import _get_last_metadata_file + + self._setup_iceberg(self._conn) + metadata_path = f"{resolved_folder}/metadata" + last_metadata_file = _get_last_metadata_file(metadata_path, self.fs_client) + from_statement = f"iceberg_scan('{last_metadata_file}')" elif first_file_type == "parquet": from_statement = f"read_parquet([{resolved_files_string}])" elif first_file_type == "jsonl": @@ -267,7 +276,7 @@ def create_views_for_tables(self, tables: Dict[str, str]) -> None: else: raise NotImplementedError( f"Unknown filetype {first_file_type} for table {table_name}. Currently only" - " jsonl and parquet files as well as delta tables are supported." + " jsonl and parquet files as well as delta and iceberg tables are supported." ) # create table @@ -299,6 +308,16 @@ def execute_query(self, query: AnyStr, *args: Any, **kwargs: Any) -> Iterator[DB with super().execute_query(query, *args, **kwargs) as cursor: yield cursor + @staticmethod + def _setup_iceberg(conn: duckdb.DuckDBPyConnection) -> None: + # needed to make persistent secrets work in new connection + # https://github.com/duckdb/duckdb_iceberg/issues/83 + conn.execute("FROM duckdb_secrets();") + + # `duckdb_iceberg` extension does not support autoloading + # https://github.com/duckdb/duckdb_iceberg/issues/71 + conn.execute("INSTALL iceberg; LOAD iceberg;") + def __del__(self) -> None: if self.memory_db: self.memory_db.close() diff --git a/dlt/destinations/impl/sqlalchemy/factory.py b/dlt/destinations/impl/sqlalchemy/factory.py index edd827ed00..e61ac1fb6a 100644 --- a/dlt/destinations/impl/sqlalchemy/factory.py +++ b/dlt/destinations/impl/sqlalchemy/factory.py @@ -81,6 +81,9 @@ def adjust_capabilities( caps.max_column_identifier_length = dialect.max_identifier_length caps.supports_native_boolean = dialect.supports_native_boolean if dialect.name == "mysql": + # correct max identifier length + # dialect uses 255 (max length for aliases) instead of 64 (max length of identifiers) + caps.max_identifier_length = 64 caps.format_datetime_literal = _format_mysql_datetime_literal return caps diff --git a/dlt/extract/incremental/lag.py b/dlt/extract/incremental/lag.py index ee102a9961..dfafa2cd11 100644 --- a/dlt/extract/incremental/lag.py +++ b/dlt/extract/incremental/lag.py @@ -20,7 +20,7 @@ def _apply_lag_to_value( parsed_value = ensure_pendulum_datetime(value) if is_str else value if isinstance(parsed_value, (datetime, date)): - parsed_value = _apply_lag_to_datetime(lag, parsed_value, last_value_func, is_str_date) + parsed_value = _apply_lag_to_datetime(lag, parsed_value, last_value_func, is_str_date) # type: ignore[assignment] # go back to string or pass exact type value = parsed_value.strftime(value_format) if value_format else parsed_value # type: ignore[assignment] diff --git a/dlt/helpers/airflow_helper.py b/dlt/helpers/airflow_helper.py index 99458a3949..aaa19ea97d 100644 --- a/dlt/helpers/airflow_helper.py +++ b/dlt/helpers/airflow_helper.py @@ -18,7 +18,7 @@ from airflow.configuration import conf from airflow.models import TaskInstance from airflow.utils.task_group import TaskGroup - from airflow.operators.dummy import DummyOperator # type: ignore + from airflow.operators.dummy import DummyOperator from airflow.operators.python import PythonOperator, get_current_context except ModuleNotFoundError: raise MissingDependencyException("Airflow", ["apache-airflow>=2.5"]) @@ -255,7 +255,7 @@ def _run( # use task logger if self.use_task_logger: - ti: TaskInstance = get_current_context()["ti"] # type: ignore + ti: TaskInstance = get_current_context()["ti"] # type: ignore[assignment,unused-ignore] logger.LOGGER = ti.log # set global number of buffered items diff --git a/dlt/helpers/dbt/profiles.yml b/dlt/helpers/dbt/profiles.yml index a2a0014e4e..fd114478fb 100644 --- a/dlt/helpers/dbt/profiles.yml +++ b/dlt/helpers/dbt/profiles.yml @@ -83,6 +83,7 @@ duckdb: extensions: - httpfs - parquet + - iceberg # TODO: emit the config of duck db motherduck: diff --git a/docs/website/docs/dlt-ecosystem/destinations/delta-iceberg.md b/docs/website/docs/dlt-ecosystem/destinations/delta-iceberg.md new file mode 100644 index 0000000000..7a056d6b40 --- /dev/null +++ b/docs/website/docs/dlt-ecosystem/destinations/delta-iceberg.md @@ -0,0 +1,168 @@ +--- +title: Delta / Iceberg +description: Delta / Iceberg `dlt` destination +keywords: [delta, iceberg, destination, data warehouse] +--- + +# Delta and Iceberg table formats +`dlt` supports writing [Delta](https://delta.io/) and [Iceberg](https://iceberg.apache.org/) tables when using the [filesystem](./filesystem.md) destination. + +## How it works +`dlt` uses the [deltalake](https://pypi.org/project/deltalake/) and [pyiceberg](https://pypi.org/project/pyiceberg/) libraries to write Delta and Iceberg tables, respectively. One or multiple Parquet files are prepared during the extract and normalize steps. In the load step, these Parquet files are exposed as an Arrow data structure and fed into `deltalake` or `pyiceberg`. + +## Iceberg single-user ephemeral catalog +`dlt` uses single-table, ephemeral, in-memory, sqlite-based [Iceberg catalog](https://iceberg.apache.org/concepts/catalog/)s. These catalogs are created "on demand" when a pipeline is run, and do not persist afterwards. If a table already exists in the filesystem, it gets registered into the catalog using its latest metadata file. This allows for a serverless setup. It is currently not possible to connect your own Iceberg catalog. + +:::caution +While ephemeral catalogs make it easy to get started with Iceberg, it comes with limitations: +- concurrent writes are not handled and may lead to corrupt table state +- we cannot guarantee that reads concurrent with writes are clean +- the latest manifest file needs to be searched for using file listing—this can become slow with large tables, especially in cloud object stores +::: + +## Delta dependencies + +You need the `deltalake` package to use this format: + +```sh +pip install "dlt[deltalake]" +``` + +You also need `pyarrow>=17.0.0`: + +```sh +pip install 'pyarrow>=17.0.0' +``` + +## Iceberg dependencies + +You need Python version 3.9 or higher and the `pyiceberg` package to use this format: + +```sh +pip install "dlt[pyiceberg]" +``` + +You also need `sqlalchemy>=2.0.18`: + +```sh +pip install 'sqlalchemy>=2.0.18' +``` + +## Set table format + +Set the `table_format` argument to `delta` or `iceberg` when defining your resource: + +```py +@dlt.resource(table_format="delta") +def my_delta_resource(): + ... +``` + +or when calling `run` on your pipeline: + +```py +pipeline.run(my_resource, table_format="delta") +``` + +:::note +`dlt` always uses Parquet as `loader_file_format` when using the `delta` or `iceberg` table format. Any setting of `loader_file_format` is disregarded. +::: + + +## Table format partitioning +Both `delta` and `iceberg` tables can be partitioned by specifying one or more `partition` column hints. This example partitions a Delta table by the `foo` column: + +```py +@dlt.resource( + table_format="delta", + columns={"foo": {"partition": True}} +) +def my_delta_resource(): + ... +``` + +:::note +Delta uses [Hive-style partitioning](https://delta.io/blog/pros-cons-hive-style-partionining/), while Iceberg uses [hidden partioning](https://iceberg.apache.org/docs/latest/partitioning/). +::: + +:::caution +Partition evolution (changing partition columns after a table has been created) is not supported. +::: + +## Table access helper functions +You can use the `get_delta_tables` and `get_iceberg_tables` helper functions to acccess native table objects. For `delta` these are `deltalake` [DeltaTable](https://delta-io.github.io/delta-rs/api/delta_table/) objects, for `iceberg` these are `pyiceberg` [Table](https://py.iceberg.apache.org/reference/pyiceberg/table/#pyiceberg.table.Table) objects. + +```py +from dlt.common.libs.deltalake import get_delta_tables +# from dlt.common.libs.pyiceberg import get_iceberg_tables + +... + +# get dictionary of DeltaTable objects +delta_tables = get_delta_tables(pipeline) + +# execute operations on DeltaTable objects +delta_tables["my_delta_table"].optimize.compact() +delta_tables["another_delta_table"].optimize.z_order(["col_a", "col_b"]) +# delta_tables["my_delta_table"].vacuum() +# etc. +``` + +## Table format Google Cloud Storage authentication + +Note that not all authentication methods are supported when using table formats on Google Cloud Storage: + +| Authentication method | `delta` | `iceberg` | +| -- | -- | -- | +| [Service Account](bigquery.md#setup-guide) | ✅ | ❌ | +| [OAuth](../destinations/bigquery.md#oauth-20-authentication) | ❌ | ✅ | +| [Application Default Credentials](bigquery.md#using-default-credentials) | ✅ | ❌ | + +:::note +The [S3-compatible](#using-s3-compatible-storage) interface for Google Cloud Storage is not supported when using `iceberg`. +::: + +## Iceberg Azure scheme +The `az` [scheme](#supported-schemes) is not supported when using the `iceberg` table format. Please use the `abfss` scheme. This is because `pyiceberg`, which `dlt` used under the hood, currently does not support `az`. + +## Table format `merge` support (**experimental**) +The [`upsert`](../../general-usage/incremental-loading.md#upsert-strategy) merge strategy is supported for `delta`. For `iceberg`, the `merge` write disposition is not supported and falls back to `append`. + +:::caution +The `upsert` merge strategy for the filesystem destination with Delta table format is **experimental**. +::: + +```py +@dlt.resource( + write_disposition={"disposition": "merge", "strategy": "upsert"}, + primary_key="my_primary_key", + table_format="delta" +) +def my_upsert_resource(): + ... +... +``` + +### Known limitations +- `hard_delete` hint not supported +- Deleting records from nested tables not supported + - This means updates to JSON columns that involve element removals are not propagated. For example, if you first load `{"key": 1, "nested": [1, 2]}` and then load `{"key": 1, "nested": [1]}`, then the record for element `2` will not be deleted from the nested table. + +## Delta table format storage options +You can pass storage options by configuring `destination.filesystem.deltalake_storage_options`: + +```toml +[destination.filesystem] +deltalake_storage_options = '{"AWS_S3_LOCKING_PROVIDER": "dynamodb", "DELTA_DYNAMO_TABLE_NAME": "custom_table_name"}' +``` + +`dlt` passes these options to the `storage_options` argument of the `write_deltalake` method in the `deltalake` library. Look at their [documentation](https://delta-io.github.io/delta-rs/api/delta_writer/#deltalake.write_deltalake) to see which options can be used. + +You don't need to specify credentials here. `dlt` merges the required credentials with the options you provided before passing it as `storage_options`. + +>❗When using `s3`, you need to specify storage options to [configure](https://delta-io.github.io/delta-rs/usage/writing/writing-to-s3-with-locking-provider/) locking behavior. + +## Delta table format memory usage +:::caution +Beware that when loading a large amount of data for one table, the underlying rust implementation will consume a lot of memory. This is a known issue and the maintainers are actively working on a solution. You can track the progress [here](https://github.com/delta-io/delta-rs/pull/2289). Until the issue is resolved, you can mitigate the memory consumption by doing multiple smaller incremental pipeline runs. +::: \ No newline at end of file diff --git a/docs/website/docs/dlt-ecosystem/destinations/filesystem.md b/docs/website/docs/dlt-ecosystem/destinations/filesystem.md index 9b243b9429..de3d12e8e1 100644 --- a/docs/website/docs/dlt-ecosystem/destinations/filesystem.md +++ b/docs/website/docs/dlt-ecosystem/destinations/filesystem.md @@ -108,7 +108,8 @@ You need to create an S3 bucket and a user who can access that bucket. dlt does #### Using S3 compatible storage -To use an S3 compatible storage other than AWS S3, such as [MinIO](https://min.io/) or [Cloudflare R2](https://www.cloudflare.com/en-ca/developer-platform/r2/), you may supply an `endpoint_url` in the config. This should be set along with AWS credentials: +To use an S3 compatible storage other than AWS S3, such as [MinIO](https://min.io/), [Cloudflare R2](https://www.cloudflare.com/en-ca/developer-platform/r2/) or [Google +Cloud Storage](https://cloud.google.com/storage/docs/interoperability), you may supply an `endpoint_url` in the config. This should be set along with AWS credentials: ```toml [destination.filesystem] @@ -166,6 +167,8 @@ Run `pip install "dlt[az]"` which will install the `adlfs` package to interface Edit the credentials in `.dlt/secrets.toml`, you'll see AWS credentials by default; replace them with your Azure credentials. +#### Supported schemes + `dlt` supports both forms of the blob storage urls: ```toml [destination.filesystem] @@ -404,29 +407,6 @@ The filesystem destination handles the write dispositions as follows: - `replace` - all files that belong to such tables are deleted from the dataset folder, and then the current set of files is added. - `merge` - falls back to `append` -### Merge with Delta table format (experimental) -The [`upsert`](../../general-usage/incremental-loading.md#upsert-strategy) merge strategy is supported when using the [Delta table format](#delta-table-format). - -:::caution -The `upsert` merge strategy for the filesystem destination with Delta table format is experimental. -::: - -```py -@dlt.resource( - write_disposition={"disposition": "merge", "strategy": "upsert"}, - primary_key="my_primary_key", - table_format="delta" -) -def my_upsert_resource(): - ... -... -``` - -#### Known limitations -- `hard_delete` hint not supported -- Deleting records from nested tables not supported - - This means updates to JSON columns that involve element removals are not propagated. For example, if you first load `{"key": 1, "nested": [1, 2]}` and then load `{"key": 1, "nested": [1]}`, then the record for element `2` will not be deleted from the nested table. - ## File compression The filesystem destination in the dlt library uses `gzip` compression by default for efficiency, which may result in the files being stored in a compressed format. This format may not be easily readable as plain text or JSON Lines (`jsonl`) files. If you encounter files that seem unreadable, they may be compressed. @@ -645,88 +625,9 @@ You can choose the following file formats: ## Supported table formats -You can choose the following table formats: -* [Delta table](../table-formats/delta.md) is supported - -### Delta table format - -You need the `deltalake` package to use this format: - -```sh -pip install "dlt[deltalake]" -``` - -You also need `pyarrow>=17.0.0`: - -```sh -pip install 'pyarrow>=17.0.0' -``` - -Set the `table_format` argument to `delta` when defining your resource: - -```py -@dlt.resource(table_format="delta") -def my_delta_resource(): - ... -``` - -:::note -`dlt` always uses Parquet as `loader_file_format` when using the `delta` table format. Any setting of `loader_file_format` is disregarded. -::: - -:::caution -Beware that when loading a large amount of data for one table, the underlying rust implementation will consume a lot of memory. This is a known issue and the maintainers are actively working on a solution. You can track the progress [here](https://github.com/delta-io/delta-rs/pull/2289). Until the issue is resolved, you can mitigate the memory consumption by doing multiple smaller incremental pipeline runs. -::: - -#### Delta table partitioning -A Delta table can be partitioned ([Hive-style partitioning](https://delta.io/blog/pros-cons-hive-style-partionining/)) by specifying one or more `partition` column hints. This example partitions the Delta table by the `foo` column: - -```py -@dlt.resource( - table_format="delta", - columns={"foo": {"partition": True}} -) -def my_delta_resource(): - ... -``` - -:::caution -It is **not** possible to change partition columns after the Delta table has been created. Trying to do so causes an error stating that the partition columns don't match. -::: - - -#### Storage options -You can pass storage options by configuring `destination.filesystem.deltalake_storage_options`: - -```toml -[destination.filesystem] -deltalake_storage_options = '{"AWS_S3_LOCKING_PROVIDER": "dynamodb", "DELTA_DYNAMO_TABLE_NAME": "custom_table_name"}' -``` - -`dlt` passes these options to the `storage_options` argument of the `write_deltalake` method in the `deltalake` library. Look at their [documentation](https://delta-io.github.io/delta-rs/api/delta_writer/#deltalake.write_deltalake) to see which options can be used. - -You don't need to specify credentials here. `dlt` merges the required credentials with the options you provided before passing it as `storage_options`. - ->❗When using `s3`, you need to specify storage options to [configure](https://delta-io.github.io/delta-rs/usage/writing/writing-to-s3-with-locking-provider/) locking behavior. - -#### `get_delta_tables` helper -You can use the `get_delta_tables` helper function to get `deltalake` [DeltaTable](https://delta-io.github.io/delta-rs/api/delta_table/) objects for your Delta tables: - -```py -from dlt.common.libs.deltalake import get_delta_tables - -... - -# get dictionary of DeltaTable objects -delta_tables = get_delta_tables(pipeline) - -# execute operations on DeltaTable objects -delta_tables["my_delta_table"].optimize.compact() -delta_tables["another_delta_table"].optimize.z_order(["col_a", "col_b"]) -# delta_tables["my_delta_table"].vacuum() -# etc. - -``` +You can choose the following [table formats](./delta-iceberg.md): +* Delta table +* Iceberg ## Syncing of dlt state This destination fully supports [dlt state sync](../../general-usage/state#syncing-state-with-destination). To this end, special folders and files will be created at your destination which hold information about your pipeline state, schemas, and completed loads. These folders DO NOT respect your settings in the layout section. When using filesystem as a staging destination, not all of these folders are created, as the state and schemas are managed in the regular way by the final destination you have configured. diff --git a/docs/website/docs/dlt-ecosystem/table-formats/iceberg.md b/docs/website/docs/dlt-ecosystem/table-formats/iceberg.md index 233ae0ce21..edca521e52 100644 --- a/docs/website/docs/dlt-ecosystem/table-formats/iceberg.md +++ b/docs/website/docs/dlt-ecosystem/table-formats/iceberg.md @@ -10,5 +10,5 @@ keywords: [iceberg, table formats] ## Supported destinations -Supported by: **Athena** +Supported by: **Athena**, **filesystem** diff --git a/docs/website/docs/general-usage/dataset-access/ibis-backend.md b/docs/website/docs/general-usage/dataset-access/ibis-backend.md index 8f4b0fb6b6..9f9b65e9c0 100644 --- a/docs/website/docs/general-usage/dataset-access/ibis-backend.md +++ b/docs/website/docs/general-usage/dataset-access/ibis-backend.md @@ -6,7 +6,7 @@ keywords: [data, dataset, ibis] # Ibis -Ibis is a powerful portable Python dataframe library. Learn more about what it is and how to use it in the [official documentation](https://ibis-project.org/). +Ibis is a powerful portable Python dataframe library. Learn more about what it is and how to use it in the [official documentation](https://ibis-project.org/). `dlt` provides an easy way to hand over your loaded dataset to an Ibis backend connection. @@ -46,4 +46,3 @@ print(table.limit(10).execute()) # Visit the ibis docs to learn more about the available methods ``` - diff --git a/docs/website/sidebars.js b/docs/website/sidebars.js index 274f3e82b3..8e8c11fc09 100644 --- a/docs/website/sidebars.js +++ b/docs/website/sidebars.js @@ -167,6 +167,7 @@ const sidebars = { 'dlt-ecosystem/destinations/synapse', 'dlt-ecosystem/destinations/clickhouse', 'dlt-ecosystem/destinations/filesystem', + 'dlt-ecosystem/destinations/delta-iceberg', 'dlt-ecosystem/destinations/postgres', 'dlt-ecosystem/destinations/redshift', 'dlt-ecosystem/destinations/snowflake', diff --git a/mypy.ini b/mypy.ini index 769e84b13a..fdf0ceb1e6 100644 --- a/mypy.ini +++ b/mypy.ini @@ -135,3 +135,9 @@ ignore_missing_imports = True [mypy-time_machine.*] ignore_missing_imports = True + +[mypy-pyiceberg.*] +ignore_missing_imports = True + +[mypy-airflow.*] +ignore_missing_imports = True diff --git a/poetry.lock b/poetry.lock index e5d2eafdd9..25dbf5a3b2 100644 --- a/poetry.lock +++ b/poetry.lock @@ -1543,13 +1543,13 @@ files = [ [[package]] name = "cachetools" -version = "5.3.1" +version = "5.5.0" description = "Extensible memoizing collections and decorators" optional = false python-versions = ">=3.7" files = [ - {file = "cachetools-5.3.1-py3-none-any.whl", hash = "sha256:95ef631eeaea14ba2e36f06437f36463aac3a096799e876ee55e5cdccb102590"}, - {file = "cachetools-5.3.1.tar.gz", hash = "sha256:dce83f2d9b4e1f732a8cd44af8e8fab2dbe46201467fc98b3ef8f269092bf62b"}, + {file = "cachetools-5.5.0-py3-none-any.whl", hash = "sha256:02134e8439cdc2ffb62023ce1debca2944c3f289d66bb17ead3ab3dede74b292"}, + {file = "cachetools-5.5.0.tar.gz", hash = "sha256:2cc24fb4cbe39633fb7badd9db9ca6295d766d9c2995f245725a46715d050f2a"}, ] [[package]] @@ -5892,44 +5892,49 @@ files = [ [[package]] name = "mypy" -version = "1.10.0" +version = "1.12.1" description = "Optional static typing for Python" optional = false python-versions = ">=3.8" files = [ - {file = "mypy-1.10.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:da1cbf08fb3b851ab3b9523a884c232774008267b1f83371ace57f412fe308c2"}, - {file = "mypy-1.10.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:12b6bfc1b1a66095ab413160a6e520e1dc076a28f3e22f7fb25ba3b000b4ef99"}, - {file = "mypy-1.10.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9e36fb078cce9904c7989b9693e41cb9711e0600139ce3970c6ef814b6ebc2b2"}, - {file = "mypy-1.10.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:2b0695d605ddcd3eb2f736cd8b4e388288c21e7de85001e9f85df9187f2b50f9"}, - {file = "mypy-1.10.0-cp310-cp310-win_amd64.whl", hash = "sha256:cd777b780312ddb135bceb9bc8722a73ec95e042f911cc279e2ec3c667076051"}, - {file = "mypy-1.10.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:3be66771aa5c97602f382230165b856c231d1277c511c9a8dd058be4784472e1"}, - {file = "mypy-1.10.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:8b2cbaca148d0754a54d44121b5825ae71868c7592a53b7292eeb0f3fdae95ee"}, - {file = "mypy-1.10.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1ec404a7cbe9fc0e92cb0e67f55ce0c025014e26d33e54d9e506a0f2d07fe5de"}, - {file = "mypy-1.10.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:e22e1527dc3d4aa94311d246b59e47f6455b8729f4968765ac1eacf9a4760bc7"}, - {file = "mypy-1.10.0-cp311-cp311-win_amd64.whl", hash = "sha256:a87dbfa85971e8d59c9cc1fcf534efe664d8949e4c0b6b44e8ca548e746a8d53"}, - {file = "mypy-1.10.0-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:a781f6ad4bab20eef8b65174a57e5203f4be627b46291f4589879bf4e257b97b"}, - {file = "mypy-1.10.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:b808e12113505b97d9023b0b5e0c0705a90571c6feefc6f215c1df9381256e30"}, - {file = "mypy-1.10.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8f55583b12156c399dce2df7d16f8a5095291354f1e839c252ec6c0611e86e2e"}, - {file = "mypy-1.10.0-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:4cf18f9d0efa1b16478c4c129eabec36148032575391095f73cae2e722fcf9d5"}, - {file = "mypy-1.10.0-cp312-cp312-win_amd64.whl", hash = "sha256:bc6ac273b23c6b82da3bb25f4136c4fd42665f17f2cd850771cb600bdd2ebeda"}, - {file = "mypy-1.10.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:9fd50226364cd2737351c79807775136b0abe084433b55b2e29181a4c3c878c0"}, - {file = "mypy-1.10.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:f90cff89eea89273727d8783fef5d4a934be2fdca11b47def50cf5d311aff727"}, - {file = "mypy-1.10.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fcfc70599efde5c67862a07a1aaf50e55bce629ace26bb19dc17cece5dd31ca4"}, - {file = "mypy-1.10.0-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:075cbf81f3e134eadaf247de187bd604748171d6b79736fa9b6c9685b4083061"}, - {file = "mypy-1.10.0-cp38-cp38-win_amd64.whl", hash = "sha256:3f298531bca95ff615b6e9f2fc0333aae27fa48052903a0ac90215021cdcfa4f"}, - {file = "mypy-1.10.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:fa7ef5244615a2523b56c034becde4e9e3f9b034854c93639adb667ec9ec2976"}, - {file = "mypy-1.10.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:3236a4c8f535a0631f85f5fcdffba71c7feeef76a6002fcba7c1a8e57c8be1ec"}, - {file = "mypy-1.10.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4a2b5cdbb5dd35aa08ea9114436e0d79aceb2f38e32c21684dcf8e24e1e92821"}, - {file = "mypy-1.10.0-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:92f93b21c0fe73dc00abf91022234c79d793318b8a96faac147cd579c1671746"}, - {file = "mypy-1.10.0-cp39-cp39-win_amd64.whl", hash = "sha256:28d0e038361b45f099cc086d9dd99c15ff14d0188f44ac883010e172ce86c38a"}, - {file = "mypy-1.10.0-py3-none-any.whl", hash = "sha256:f8c083976eb530019175aabadb60921e73b4f45736760826aa1689dda8208aee"}, - {file = "mypy-1.10.0.tar.gz", hash = "sha256:3d087fcbec056c4ee34974da493a826ce316947485cef3901f511848e687c131"}, + {file = "mypy-1.12.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:3d7d4371829184e22fda4015278fbfdef0327a4b955a483012bd2d423a788801"}, + {file = "mypy-1.12.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:f59f1dfbf497d473201356966e353ef09d4daec48caeacc0254db8ef633a28a5"}, + {file = "mypy-1.12.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:b947097fae68004b8328c55161ac9db7d3566abfef72d9d41b47a021c2fba6b1"}, + {file = "mypy-1.12.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:96af62050971c5241afb4701c15189ea9507db89ad07794a4ee7b4e092dc0627"}, + {file = "mypy-1.12.1-cp310-cp310-win_amd64.whl", hash = "sha256:d90da248f4c2dba6c44ddcfea94bb361e491962f05f41990ff24dbd09969ce20"}, + {file = "mypy-1.12.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:1230048fec1380faf240be6385e709c8570604d2d27ec6ca7e573e3bc09c3735"}, + {file = "mypy-1.12.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:02dcfe270c6ea13338210908f8cadc8d31af0f04cee8ca996438fe6a97b4ec66"}, + {file = "mypy-1.12.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:a5a437c9102a6a252d9e3a63edc191a3aed5f2fcb786d614722ee3f4472e33f6"}, + {file = "mypy-1.12.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:186e0c8346efc027ee1f9acf5ca734425fc4f7dc2b60144f0fbe27cc19dc7931"}, + {file = "mypy-1.12.1-cp311-cp311-win_amd64.whl", hash = "sha256:673ba1140a478b50e6d265c03391702fa11a5c5aff3f54d69a62a48da32cb811"}, + {file = "mypy-1.12.1-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:9fb83a7be97c498176fb7486cafbb81decccaef1ac339d837c377b0ce3743a7f"}, + {file = "mypy-1.12.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:389e307e333879c571029d5b93932cf838b811d3f5395ed1ad05086b52148fb0"}, + {file = "mypy-1.12.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:94b2048a95a21f7a9ebc9fbd075a4fcd310410d078aa0228dbbad7f71335e042"}, + {file = "mypy-1.12.1-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:4ee5932370ccf7ebf83f79d1c157a5929d7ea36313027b0d70a488493dc1b179"}, + {file = "mypy-1.12.1-cp312-cp312-win_amd64.whl", hash = "sha256:19bf51f87a295e7ab2894f1d8167622b063492d754e69c3c2fed6563268cb42a"}, + {file = "mypy-1.12.1-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:d34167d43613ffb1d6c6cdc0cc043bb106cac0aa5d6a4171f77ab92a3c758bcc"}, + {file = "mypy-1.12.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:427878aa54f2e2c5d8db31fa9010c599ed9f994b3b49e64ae9cd9990c40bd635"}, + {file = "mypy-1.12.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:5fcde63ea2c9f69d6be859a1e6dd35955e87fa81de95bc240143cf00de1f7f81"}, + {file = "mypy-1.12.1-cp313-cp313-musllinux_1_1_x86_64.whl", hash = "sha256:d54d840f6c052929f4a3d2aab2066af0f45a020b085fe0e40d4583db52aab4e4"}, + {file = "mypy-1.12.1-cp313-cp313-win_amd64.whl", hash = "sha256:20db6eb1ca3d1de8ece00033b12f793f1ea9da767334b7e8c626a4872090cf02"}, + {file = "mypy-1.12.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:b16fe09f9c741d85a2e3b14a5257a27a4f4886c171d562bc5a5e90d8591906b8"}, + {file = "mypy-1.12.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:0dcc1e843d58f444fce19da4cce5bd35c282d4bde232acdeca8279523087088a"}, + {file = "mypy-1.12.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:e10ba7de5c616e44ad21005fa13450cd0de7caaa303a626147d45307492e4f2d"}, + {file = "mypy-1.12.1-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:0e6fe449223fa59fbee351db32283838a8fee8059e0028e9e6494a03802b4004"}, + {file = "mypy-1.12.1-cp38-cp38-win_amd64.whl", hash = "sha256:dc6e2a2195a290a7fd5bac3e60b586d77fc88e986eba7feced8b778c373f9afe"}, + {file = "mypy-1.12.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:de5b2a8988b4e1269a98beaf0e7cc71b510d050dce80c343b53b4955fff45f19"}, + {file = "mypy-1.12.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:843826966f1d65925e8b50d2b483065c51fc16dc5d72647e0236aae51dc8d77e"}, + {file = "mypy-1.12.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:9fe20f89da41a95e14c34b1ddb09c80262edcc295ad891f22cc4b60013e8f78d"}, + {file = "mypy-1.12.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:8135ffec02121a75f75dc97c81af7c14aa4ae0dda277132cfcd6abcd21551bfd"}, + {file = "mypy-1.12.1-cp39-cp39-win_amd64.whl", hash = "sha256:a7b76fa83260824300cc4834a3ab93180db19876bce59af921467fd03e692810"}, + {file = "mypy-1.12.1-py3-none-any.whl", hash = "sha256:ce561a09e3bb9863ab77edf29ae3a50e65685ad74bba1431278185b7e5d5486e"}, + {file = "mypy-1.12.1.tar.gz", hash = "sha256:f5b3936f7a6d0e8280c9bdef94c7ce4847f5cdfc258fbb2c29a8c1711e8bb96d"}, ] [package.dependencies] mypy-extensions = ">=1.0.0" tomli = {version = ">=1.1.0", markers = "python_version < \"3.11\""} -typing-extensions = ">=4.1.0" +typing-extensions = ">=4.6.0" [package.extras] dmypy = ["psutil (>=4.0)"] @@ -7541,6 +7546,74 @@ files = [ [package.extras] plugins = ["importlib-metadata"] +[[package]] +name = "pyiceberg" +version = "0.8.1" +description = "Apache Iceberg is an open table format for huge analytic datasets" +optional = true +python-versions = "!=2.7.*,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*,!=3.5.*,!=3.6.*,!=3.7.*,!=3.8.*,>=3.9" +files = [ + {file = "pyiceberg-0.8.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:7c121d1d3baf64510db94740ad870ae4b6eb9eb59a5ff7ecb4e96f7510666b2f"}, + {file = "pyiceberg-0.8.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:2a6f14aa588a3883fc7fddc136ca75b75660b4abb0b55b4c541619953f8971e7"}, + {file = "pyiceberg-0.8.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7c720c2a191ac6faf01fe4c0f4c01c64b94bf064185b0292003d42939049277c"}, + {file = "pyiceberg-0.8.1-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:d421d6e51ac1c581cba9fce96aa6b9118cf4a02270066a7fdc9490ab5d57ece9"}, + {file = "pyiceberg-0.8.1-cp310-cp310-win_amd64.whl", hash = "sha256:ae11fb0515ea0a046370e09a7f6039a7e86622ab910360eaa732f0106b8f00c7"}, + {file = "pyiceberg-0.8.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:9488954c9eb5ce42ca6b816fc61873f219414cfdb9e9928d1c4a302702be1d89"}, + {file = "pyiceberg-0.8.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:44179e0fb844887b440c162279ba526dfe0e0f72d32945236528838518b55af0"}, + {file = "pyiceberg-0.8.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3e121c6f5505d8ec711a1dd1690e07156cd54fb3d0844d5d991e02f1593f2708"}, + {file = "pyiceberg-0.8.1-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:5961a288f2d4bbb2ab300c803da1bf0e70cea837e3f14b14108827cc821af252"}, + {file = "pyiceberg-0.8.1-cp311-cp311-win_amd64.whl", hash = "sha256:dbe192324a6fb552c2fd29cab51086e21fa248ea2a0b95fbab921dede49e5a69"}, + {file = "pyiceberg-0.8.1-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:60430f0d8f6d650ed7d1893d038b847565a8e9ac135a1cc812e57d24f0482f6c"}, + {file = "pyiceberg-0.8.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:f0f697977dac672d8b00e125836423585a97ebf59a28b865b1296a2b6ee81c51"}, + {file = "pyiceberg-0.8.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:370de7c230970ff858f713d150164d492ba8450e771e59a0c520520b13ea6226"}, + {file = "pyiceberg-0.8.1-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:3036ed226020d50e30648a71f968cf78bde5d6b609294508e60754e100e5ef36"}, + {file = "pyiceberg-0.8.1-cp312-cp312-win_amd64.whl", hash = "sha256:9ac9555f3bd25a31059229089ae639cf738a8e8286a175cea128561ac1ed9452"}, + {file = "pyiceberg-0.8.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:51da3a553d3a881042bf436e66a91cc2b6c4a3fea0e174cd73af2eb6ed255323"}, + {file = "pyiceberg-0.8.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:863f1dce7340e6ed870706a3fa4a73457178dae8529725bb80522ddcd4253afb"}, + {file = "pyiceberg-0.8.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4dbf52b39080a6a2cda6a5126a74e3a88d5b206f609c128d001a728b36b81075"}, + {file = "pyiceberg-0.8.1-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:eb77d65e8efbb883c163817e4a9c373d907110ab6343c1b816b48f336955d4d7"}, + {file = "pyiceberg-0.8.1-cp39-cp39-win_amd64.whl", hash = "sha256:1fcd35b7de0eddc3fd8fd0c38b98741217ef6de4eeb0e72b798b4007692aa76c"}, + {file = "pyiceberg-0.8.1-pp310-pypy310_pp73-macosx_10_15_x86_64.whl", hash = "sha256:6f0f56f8fc61bcd795f6a3d03e8ce6bee09ebaa64425eb08327e975f906d98be"}, + {file = "pyiceberg-0.8.1-pp310-pypy310_pp73-macosx_11_0_arm64.whl", hash = "sha256:d7099c6631743ad29c451de2bebd9ed3c96c42bcb1fe5d5d5c93aec895858e3f"}, + {file = "pyiceberg-0.8.1-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d6436f5a782491115f64131882a737d77c9dc0040493e1b7f9b3081ea8cf6a26"}, + {file = "pyiceberg-0.8.1-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:c1d75b40a98a327f7436eb0d6187c51834c44b79adf61c6945b33645f4afbf17"}, + {file = "pyiceberg-0.8.1-pp39-pypy39_pp73-macosx_10_15_x86_64.whl", hash = "sha256:8de988fa2363e6a51b40b85b5ff1e8261cda5bfc14ac54dd4ebe58391b95acae"}, + {file = "pyiceberg-0.8.1-pp39-pypy39_pp73-macosx_11_0_arm64.whl", hash = "sha256:dd06c5b606011155aa0b76e7b001e30f1c40ab2fb3eeb8a0652b88629259c2bb"}, + {file = "pyiceberg-0.8.1-pp39-pypy39_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e8142f0dbc12dda0e6d7aaf564a3fbb0f17fc934630e7cf866773c8caaebf666"}, + {file = "pyiceberg-0.8.1-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:6126ee3a46ff975f15abf2085f184591d21643bffb96330907e003eea0b63005"}, + {file = "pyiceberg-0.8.1.tar.gz", hash = "sha256:4502f0cfddf6f7cd48b9cd54016bce0ab94052b0ab01efcfa515879074f4c8e3"}, +] + +[package.dependencies] +cachetools = ">=5.5.0,<6.0.0" +click = ">=7.1.1,<9.0.0" +fsspec = ">=2023.1.0" +mmh3 = ">=4.0.0,<6.0.0" +pydantic = ">=2.0,<2.4.0 || >2.4.0,<2.4.1 || >2.4.1,<3.0" +pyparsing = ">=3.1.0,<4.0.0" +requests = ">=2.20.0,<3.0.0" +rich = ">=10.11.0,<14.0.0" +sortedcontainers = "2.4.0" +strictyaml = ">=1.7.0,<2.0.0" +tenacity = ">=8.2.3,<10.0.0" + +[package.extras] +adlfs = ["adlfs (>=2023.1.0)"] +daft = ["getdaft (>=0.2.12)"] +duckdb = ["duckdb (>=0.5.0,<2.0.0)", "pyarrow (>=14.0.0,<19.0.0)"] +dynamodb = ["boto3 (>=1.24.59)"] +gcsfs = ["gcsfs (>=2023.1.0)"] +glue = ["boto3 (>=1.24.59)", "mypy-boto3-glue (>=1.28.18)"] +hive = ["thrift (>=0.13.0,<1.0.0)"] +pandas = ["pandas (>=1.0.0,<3.0.0)", "pyarrow (>=14.0.0,<19.0.0)"] +pyarrow = ["pyarrow (>=14.0.0,<19.0.0)"] +ray = ["pandas (>=1.0.0,<3.0.0)", "pyarrow (>=14.0.0,<19.0.0)", "ray (==2.10.0)", "ray (>=2.10.0,<3.0.0)"] +s3fs = ["s3fs (>=2023.1.0)"] +snappy = ["python-snappy (>=0.6.0,<1.0.0)"] +sql-postgres = ["psycopg2-binary (>=2.9.6)", "sqlalchemy (>=2.0.18,<3.0.0)"] +sql-sqlite = ["sqlalchemy (>=2.0.18,<3.0.0)"] +zstandard = ["zstandard (>=0.13.0,<1.0.0)"] + [[package]] name = "pyjwt" version = "2.8.0" @@ -9347,6 +9420,20 @@ files = [ [package.dependencies] pbr = ">=2.0.0,<2.1.0 || >2.1.0" +[[package]] +name = "strictyaml" +version = "1.7.3" +description = "Strict, typed YAML parser" +optional = true +python-versions = ">=3.7.0" +files = [ + {file = "strictyaml-1.7.3-py3-none-any.whl", hash = "sha256:fb5c8a4edb43bebb765959e420f9b3978d7f1af88c80606c03fb420888f5d1c7"}, + {file = "strictyaml-1.7.3.tar.gz", hash = "sha256:22f854a5fcab42b5ddba8030a0e4be51ca89af0267961c8d6cfa86395586c407"}, +] + +[package.dependencies] +python-dateutil = ">=2.6.0" + [[package]] name = "sympy" version = "1.12" @@ -10626,6 +10713,7 @@ mssql = ["pyodbc"] parquet = ["pyarrow"] postgis = ["psycopg2-binary", "psycopg2cffi"] postgres = ["psycopg2-binary", "psycopg2cffi"] +pyiceberg = ["pyarrow", "pyiceberg", "sqlalchemy"] qdrant = ["qdrant-client"] redshift = ["psycopg2-binary", "psycopg2cffi"] s3 = ["botocore", "s3fs"] @@ -10639,4 +10727,4 @@ weaviate = ["weaviate-client"] [metadata] lock-version = "2.0" python-versions = ">=3.8.1,<3.13" -content-hash = "a7cd6b599326d80b5beb8d4a3d3e3b4074eda6dc53daa5c296ef8d54002c5f78" +content-hash = "84e8b8eccd9b8ee104a2dc08f5b83987aeb06540d61330390ce849cc1ad6acb4" diff --git a/pyproject.toml b/pyproject.toml index 4e10e96b93..4de92014a8 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -75,7 +75,7 @@ cron-descriptor = {version = ">=1.2.32", optional = true} pipdeptree = {version = ">=2.9.0,<2.10", optional = true} pyathena = {version = ">=2.9.6", optional = true} weaviate-client = {version = ">=3.22", optional = true} -adlfs = {version = ">=2022.4.0", optional = true} +adlfs = {version = ">=2024.7.0", optional = true} pyodbc = {version = ">=4.0.39", optional = true} qdrant-client = {version = ">=1.8", optional = true, extras = ["fastembed"]} databricks-sql-connector = {version = ">=2.9.3", optional = true} @@ -91,6 +91,12 @@ sqlglot = {version = ">=20.0.0", optional = true} db-dtypes = { version = ">=1.2.0", optional = true } aiohttp = { version = ">=3.9", optional = true } databricks-sdk = {version = ">=0.38.0", optional = true} +# `sql-sqlite` extra leads to dependency conflict with `apache-airflow` because `apache-airflow` +# requires `sqlalchemy<2.0.0` while the extra requires `sqlalchemy>=2.0.18` +# https://github.com/apache/airflow/issues/28723 +# pyiceberg = { version = ">=0.7.1", optional = true, extras = ["sql-sqlite"] } +# we will rely on manual installation of `sqlalchemy>=2.0.18` instead +pyiceberg = { version = ">=0.8.1", python = ">=3.9", optional = true } [tool.poetry.extras] gcp = ["grpcio", "google-cloud-bigquery", "db-dtypes", "gcsfs"] @@ -120,6 +126,7 @@ lancedb = ["lancedb", "pyarrow", "tantivy"] deltalake = ["deltalake", "pyarrow"] sql_database = ["sqlalchemy"] sqlalchemy = ["sqlalchemy", "alembic"] +pyiceberg = ["pyiceberg", "pyarrow", "sqlalchemy"] postgis = ["psycopg2-binary", "psycopg2cffi"] [tool.poetry.scripts] @@ -136,7 +143,7 @@ sqlfluff = "^2.3.2" types-deprecated = "^1.2.9.2" pytest-console-scripts = "^1.4.1" pytest = "^7.0.0" -mypy = "^1.10.0" +mypy = ">=1.11.0,<1.13.0" flake8 = "^5.0.0" bandit = "^1.7.0" black = "^23.7.0" diff --git a/tests/conftest.py b/tests/conftest.py index 6088fa976c..a5a349f8d9 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -120,6 +120,9 @@ def _create_pipeline_instance_id(self) -> str: # disable googleapiclient logging logging.getLogger("googleapiclient.discovery_cache").setLevel("WARNING") + # disable pyiceberg logging + logging.getLogger("pyiceberg").setLevel("WARNING") + # reset and init airflow db import warnings diff --git a/tests/libs/test_csv_writer.py b/tests/libs/test_csv_writer.py index 3c30123e1c..a120cd048e 100644 --- a/tests/libs/test_csv_writer.py +++ b/tests/libs/test_csv_writer.py @@ -178,7 +178,7 @@ def test_non_utf8_binary(item_type: TestDataItemFormat) -> None: table = pq.read_table(f) else: table = data - writer_type: Type[DataWriter] = ArrowToCsvWriter if item_type == "arrow-table" else CsvWriter # type: ignore + writer_type: Type[DataWriter] = ArrowToCsvWriter if item_type == "arrow-table" else CsvWriter with pytest.raises(InvalidDataItem) as inv_ex: with get_writer(writer_type, disable_compression=True) as writer: @@ -195,7 +195,7 @@ def test_arrow_struct() -> None: @pytest.mark.parametrize("item_type", ["object", "arrow-table"]) def test_csv_writer_empty(item_type: TestDataItemFormat) -> None: - writer_type: Type[DataWriter] = ArrowToCsvWriter if item_type == "arrow-table" else CsvWriter # type: ignore + writer_type: Type[DataWriter] = ArrowToCsvWriter if item_type == "arrow-table" else CsvWriter with get_writer(writer_type, disable_compression=True) as writer: writer.write_empty_file(TABLE_UPDATE_COLUMNS_SCHEMA) diff --git a/tests/load/filesystem/test_object_store_rs_credentials.py b/tests/load/filesystem/test_credentials_mixins.py similarity index 50% rename from tests/load/filesystem/test_object_store_rs_credentials.py rename to tests/load/filesystem/test_credentials_mixins.py index f23187a269..c1fb02c152 100644 --- a/tests/load/filesystem/test_object_store_rs_credentials.py +++ b/tests/load/filesystem/test_credentials_mixins.py @@ -1,12 +1,8 @@ -"""Tests translation of `dlt` credentials into `object_store` Rust crate credentials.""" - -from typing import Any, Dict +from typing import Any, Dict, Union, Type, get_args, cast import os import json # noqa: I251 import pytest -from deltalake import DeltaTable -from deltalake.exceptions import TableNotFoundError import dlt from dlt.common.configuration import resolve_configuration @@ -23,10 +19,15 @@ from dlt.common.utils import custom_environ from dlt.common.configuration.resolve import resolve_configuration from dlt.common.configuration.specs.gcp_credentials import GcpDefaultCredentials -from dlt.common.configuration.specs.exceptions import ObjectStoreRsCredentialsException +from dlt.common.configuration.specs.exceptions import ( + ObjectStoreRsCredentialsException, + UnsupportedAuthenticationMethodException, +) +from dlt.common.configuration.specs.mixins import WithObjectStoreRsCredentials, WithPyicebergConfig from tests.load.utils import ( AZ_BUCKET, + ABFS_BUCKET, AWS_BUCKET, GCS_BUCKET, R2_BUCKET_CONFIG, @@ -34,6 +35,9 @@ ) +TCredentialsMixin = Union[WithObjectStoreRsCredentials, WithPyicebergConfig] +ALL_CREDENTIALS_MIXINS = get_args(TCredentialsMixin) + pytestmark = pytest.mark.essential if all(driver not in ALL_FILESYSTEM_DRIVERS for driver in ("az", "s3", "gs", "r2")): @@ -53,11 +57,27 @@ def fs_creds() -> Dict[str, Any]: return creds -def can_connect(bucket_url: str, object_store_rs_credentials: Dict[str, str]) -> bool: - """Returns True if client can connect to object store, False otherwise. +def can_connect(bucket_url: str, credentials: TCredentialsMixin, mixin: Type[TCredentialsMixin]) -> bool: # type: ignore[return] + """Returns True if client can connect to object store, False otherwise.""" + if mixin == WithObjectStoreRsCredentials: + credentials = cast(WithObjectStoreRsCredentials, credentials) + return can_connect_object_store_rs_credentials( + bucket_url, credentials.to_object_store_rs_credentials() + ) + elif mixin == WithPyicebergConfig: + credentials = cast(WithPyicebergConfig, credentials) + return can_connect_pyiceberg_fileio_config( + bucket_url, credentials.to_pyiceberg_fileio_config() + ) + + +def can_connect_object_store_rs_credentials( + bucket_url: str, object_store_rs_credentials: Dict[str, str] +) -> bool: + # uses `deltatable` library as Python interface to `object_store` Rust crate + from deltalake import DeltaTable + from deltalake.exceptions import TableNotFoundError - Uses `deltatable` library as Python interface to `object_store` Rust crate. - """ try: DeltaTable( bucket_url, @@ -70,16 +90,40 @@ def can_connect(bucket_url: str, object_store_rs_credentials: Dict[str, str]) -> return False +def can_connect_pyiceberg_fileio_config( + bucket_url: str, pyiceberg_fileio_config: Dict[str, str] +) -> bool: + from pyiceberg.table import StaticTable + + try: + StaticTable.from_metadata( + f"{bucket_url}/non_existing_metadata_file.json", + properties=pyiceberg_fileio_config, + ) + except FileNotFoundError: + # this error implies the connection was successful + # there is no Iceberg metadata file at the specified path + return True + return False + + @pytest.mark.parametrize( - "driver", [driver for driver in ALL_FILESYSTEM_DRIVERS if driver in ("az")] + "driver", [driver for driver in ALL_FILESYSTEM_DRIVERS if driver in ("az", "abfss")] ) -def test_azure_object_store_rs_credentials(driver: str, fs_creds: Dict[str, Any]) -> None: +@pytest.mark.parametrize("mixin", ALL_CREDENTIALS_MIXINS) +def test_azure_credentials_mixins( + driver: str, fs_creds: Dict[str, Any], mixin: Type[TCredentialsMixin] +) -> None: + if mixin == WithPyicebergConfig and driver == "az": + pytest.skip("`pyiceberg` does not support `az` scheme") + + buckets = {"az": AZ_BUCKET, "abfss": ABFS_BUCKET} creds: AnyAzureCredentials creds = AzureServicePrincipalCredentialsWithoutDefaults( **dlt.secrets.get("destination.fsazureprincipal.credentials") ) - assert can_connect(AZ_BUCKET, creds.to_object_store_rs_credentials()) + assert can_connect(buckets[driver], creds, mixin) # without SAS token creds = AzureCredentialsWithoutDefaults( @@ -87,18 +131,21 @@ def test_azure_object_store_rs_credentials(driver: str, fs_creds: Dict[str, Any] azure_storage_account_key=fs_creds["azure_storage_account_key"], ) assert creds.azure_storage_sas_token is None - assert can_connect(AZ_BUCKET, creds.to_object_store_rs_credentials()) + assert can_connect(buckets[driver], creds, mixin) # with SAS token creds = resolve_configuration(creds) assert creds.azure_storage_sas_token is not None - assert can_connect(AZ_BUCKET, creds.to_object_store_rs_credentials()) + assert can_connect(buckets[driver], creds, mixin) @pytest.mark.parametrize( "driver", [driver for driver in ALL_FILESYSTEM_DRIVERS if driver in ("s3", "r2")] ) -def test_aws_object_store_rs_credentials(driver: str, fs_creds: Dict[str, Any]) -> None: +@pytest.mark.parametrize("mixin", ALL_CREDENTIALS_MIXINS) +def test_aws_credentials_mixins( + driver: str, fs_creds: Dict[str, Any], mixin: Type[TCredentialsMixin] +) -> None: creds: AwsCredentialsWithoutDefaults if driver == "r2": @@ -112,9 +159,11 @@ def test_aws_object_store_rs_credentials(driver: str, fs_creds: Dict[str, Any]) endpoint_url=fs_creds.get("endpoint_url"), ) assert creds.aws_session_token is None - object_store_rs_creds = creds.to_object_store_rs_credentials() - assert "aws_session_token" not in object_store_rs_creds # no auto-generated token - assert can_connect(AWS_BUCKET, object_store_rs_creds) + if mixin == WithObjectStoreRsCredentials: + assert ( + "aws_session_token" not in creds.to_object_store_rs_credentials() + ) # no auto-generated token + assert can_connect(AWS_BUCKET, creds, mixin) # AwsCredentials: no user-provided session token creds = AwsCredentials( @@ -124,24 +173,27 @@ def test_aws_object_store_rs_credentials(driver: str, fs_creds: Dict[str, Any]) endpoint_url=fs_creds.get("endpoint_url"), ) assert creds.aws_session_token is None - object_store_rs_creds = creds.to_object_store_rs_credentials() - assert "aws_session_token" not in object_store_rs_creds # no auto-generated token - assert can_connect(AWS_BUCKET, object_store_rs_creds) - - # exception should be raised if both `endpoint_url` and `region_name` are - # not provided - with pytest.raises(ObjectStoreRsCredentialsException): - AwsCredentials( - aws_access_key_id=fs_creds["aws_access_key_id"], - aws_secret_access_key=fs_creds["aws_secret_access_key"], - ).to_object_store_rs_credentials() - - if "endpoint_url" in object_store_rs_creds: - # TODO: make sure this case is tested on GitHub CI, e.g. by adding - # a local MinIO bucket to the set of tested buckets - if object_store_rs_creds["endpoint_url"].startswith("http://"): + assert can_connect(AWS_BUCKET, creds, mixin) + if mixin == WithObjectStoreRsCredentials: + object_store_rs_creds = creds.to_object_store_rs_credentials() + assert "aws_session_token" not in object_store_rs_creds # no auto-generated token + + # exception should be raised if both `endpoint_url` and `region_name` are + # not provided + with pytest.raises(ObjectStoreRsCredentialsException): + AwsCredentials( + aws_access_key_id=fs_creds["aws_access_key_id"], + aws_secret_access_key=fs_creds["aws_secret_access_key"], + ).to_object_store_rs_credentials() + + if "endpoint_url" in object_store_rs_creds and object_store_rs_creds[ + "endpoint_url" + ].startswith("http://"): + # TODO: make sure this case is tested on GitHub CI, e.g. by adding + # a local MinIO bucket to the set of tested buckets assert object_store_rs_creds["aws_allow_http"] == "true" + if creds.endpoint_url is not None: # remainder of tests use session tokens # we don't run them on S3 compatible storage because session tokens # may not be available @@ -158,9 +210,10 @@ def test_aws_object_store_rs_credentials(driver: str, fs_creds: Dict[str, Any]) region_name=fs_creds["region_name"], ) assert creds.aws_session_token is not None - object_store_rs_creds = creds.to_object_store_rs_credentials() - assert object_store_rs_creds["aws_session_token"] is not None - assert can_connect(AWS_BUCKET, object_store_rs_creds) + assert can_connect(AWS_BUCKET, creds, mixin) + if mixin == WithObjectStoreRsCredentials: + object_store_rs_creds = creds.to_object_store_rs_credentials() + assert object_store_rs_creds["aws_session_token"] is not None # AwsCredentialsWithoutDefaults: user-provided session token creds = AwsCredentialsWithoutDefaults( @@ -170,15 +223,19 @@ def test_aws_object_store_rs_credentials(driver: str, fs_creds: Dict[str, Any]) region_name=fs_creds["region_name"], ) assert creds.aws_session_token is not None - object_store_rs_creds = creds.to_object_store_rs_credentials() - assert object_store_rs_creds["aws_session_token"] is not None - assert can_connect(AWS_BUCKET, object_store_rs_creds) + assert can_connect(AWS_BUCKET, creds, mixin) + if mixin == WithObjectStoreRsCredentials: + object_store_rs_creds = creds.to_object_store_rs_credentials() + assert object_store_rs_creds["aws_session_token"] is not None @pytest.mark.parametrize( "driver", [driver for driver in ALL_FILESYSTEM_DRIVERS if driver in ("gs")] ) -def test_gcp_object_store_rs_credentials(driver, fs_creds: Dict[str, Any]) -> None: +@pytest.mark.parametrize("mixin", ALL_CREDENTIALS_MIXINS) +def test_gcp_credentials_mixins( + driver, fs_creds: Dict[str, Any], mixin: Type[TCredentialsMixin] +) -> None: creds: GcpCredentials # GcpServiceAccountCredentialsWithoutDefaults @@ -189,7 +246,11 @@ def test_gcp_object_store_rs_credentials(driver, fs_creds: Dict[str, Any]) -> No private_key_id=fs_creds["private_key_id"], client_email=fs_creds["client_email"], ) - assert can_connect(GCS_BUCKET, creds.to_object_store_rs_credentials()) + if mixin == WithPyicebergConfig: + with pytest.raises(UnsupportedAuthenticationMethodException): + assert can_connect(GCS_BUCKET, creds, mixin) + elif mixin == WithObjectStoreRsCredentials: + assert can_connect(GCS_BUCKET, creds, mixin) # GcpDefaultCredentials @@ -197,7 +258,7 @@ def test_gcp_object_store_rs_credentials(driver, fs_creds: Dict[str, Any]) -> No GcpDefaultCredentials._LAST_FAILED_DEFAULT = 0 # write service account key to JSON file - service_json = json.loads(creds.to_object_store_rs_credentials()["service_account_key"]) + service_json = json.loads(creds.to_native_representation()) path = "_secrets/service.json" os.makedirs(os.path.dirname(path), exist_ok=True) with open(path, "w", encoding="utf-8") as f: @@ -206,8 +267,18 @@ def test_gcp_object_store_rs_credentials(driver, fs_creds: Dict[str, Any]) -> No with custom_environ({"GOOGLE_APPLICATION_CREDENTIALS": path}): creds = GcpDefaultCredentials() resolve_configuration(creds) - can_connect(GCS_BUCKET, creds.to_object_store_rs_credentials()) - - # GcpOAuthCredentialsWithoutDefaults is currently not supported - with pytest.raises(NotImplementedError): - GcpOAuthCredentialsWithoutDefaults().to_object_store_rs_credentials() + if mixin == WithPyicebergConfig: + with pytest.raises(UnsupportedAuthenticationMethodException): + assert can_connect(GCS_BUCKET, creds, mixin) + elif mixin == WithObjectStoreRsCredentials: + assert can_connect(GCS_BUCKET, creds, mixin) + + # GcpOAuthCredentialsWithoutDefaults + creds = resolve_configuration( + GcpOAuthCredentialsWithoutDefaults(), sections=("destination", "fsgcpoauth") + ) + if mixin == WithPyicebergConfig: + assert can_connect(GCS_BUCKET, creds, mixin) + elif mixin == WithObjectStoreRsCredentials: + with pytest.raises(UnsupportedAuthenticationMethodException): + assert can_connect(GCS_BUCKET, creds, mixin) diff --git a/tests/load/filesystem/test_sql_client.py b/tests/load/filesystem/test_sql_client.py index ac2ada2551..a73b0f7e31 100644 --- a/tests/load/filesystem/test_sql_client.py +++ b/tests/load/filesystem/test_sql_client.py @@ -1,17 +1,17 @@ """Test the duckdb supported sql client for special internal features""" -from typing import Any +from typing import Optional import pytest import dlt import os import shutil -import logging from dlt import Pipeline from dlt.common.utils import uniq_id +from dlt.common.schema.typing import TTableFormat from tests.load.utils import ( destinations_configs, @@ -19,7 +19,6 @@ GCS_BUCKET, SFTP_BUCKET, MEMORY_BUCKET, - AWS_BUCKET, ) from dlt.destinations import filesystem from tests.utils import TEST_STORAGE_ROOT @@ -37,7 +36,7 @@ def _run_dataset_checks( pipeline: Pipeline, destination_config: DestinationTestConfiguration, secret_directory: str, - table_format: Any = None, + table_format: Optional[TTableFormat] = None, alternate_access_pipeline: Pipeline = None, ) -> None: total_records = 200 @@ -144,6 +143,8 @@ def _external_duckdb_connection() -> duckdb.DuckDBPyConnection: # the line below solves problems with certificate path lookup on linux, see duckdb docs external_db.sql("SET azure_transport_option_type = 'curl';") external_db.sql(f"SET secret_directory = '{secret_directory}';") + if table_format == "iceberg": + FilesystemSqlClient._setup_iceberg(external_db) return external_db def _fs_sql_client_for_external_db( @@ -283,13 +284,13 @@ def test_read_interfaces_filesystem( "destination_config", destinations_configs( table_format_filesystem_configs=True, - with_table_format="delta", + with_table_format=("delta", "iceberg"), bucket_exclude=[SFTP_BUCKET, MEMORY_BUCKET], # NOTE: delta does not work on memory buckets ), ids=lambda x: x.name, ) -def test_delta_tables( +def test_table_formats( destination_config: DestinationTestConfiguration, secret_directory: str ) -> None: os.environ["DATA_WRITER__FILE_MAX_ITEMS"] = "700" @@ -302,8 +303,9 @@ def test_delta_tables( # in case of gcs we use the s3 compat layer for reading # for writing we still need to use the gc authentication, as delta_rs seems to use # methods on the s3 interface that are not implemented by gcs + # s3 compat layer does not work with `iceberg` table format access_pipeline = pipeline - if destination_config.bucket_url == GCS_BUCKET: + if destination_config.bucket_url == GCS_BUCKET and destination_config.table_format != "iceberg": gcp_bucket = filesystem( GCS_BUCKET.replace("gs://", "s3://"), destination_name="filesystem_s3_gcs_comp" ) @@ -315,7 +317,7 @@ def test_delta_tables( pipeline, destination_config, secret_directory=secret_directory, - table_format="delta", + table_format=destination_config.table_format, alternate_access_pipeline=access_pipeline, ) diff --git a/tests/load/pipeline/test_filesystem_pipeline.py b/tests/load/pipeline/test_filesystem_pipeline.py index 8d890642ee..c70fa5ab5d 100644 --- a/tests/load/pipeline/test_filesystem_pipeline.py +++ b/tests/load/pipeline/test_filesystem_pipeline.py @@ -2,7 +2,7 @@ import os import posixpath from pathlib import Path -from typing import Any, Callable, List, Dict, cast +from typing import Any, Callable, List, Dict, cast, Tuple from importlib.metadata import version as pkg_version from packaging.version import Version @@ -15,7 +15,7 @@ from dlt.common.storages.configuration import FilesystemConfiguration from dlt.common.storages.load_package import ParsedLoadJobFileName from dlt.common.utils import uniq_id -from dlt.common.schema.typing import TWriteDisposition +from dlt.common.schema.typing import TWriteDisposition, TTableFormat from dlt.common.configuration.exceptions import ConfigurationValueError from dlt.destinations import filesystem from dlt.destinations.impl.filesystem.filesystem import FilesystemClient @@ -223,6 +223,48 @@ def some_source(): assert table.column("value").to_pylist() == [1, 2, 3, 4, 5] +# here start the `table_format` tests + + +def get_expected_actual( + pipeline: dlt.Pipeline, + table_name: str, + table_format: TTableFormat, + arrow_table: "pyarrow.Table", # type: ignore[name-defined] # noqa: F821 +) -> Tuple["pyarrow.Table", "pyarrow.Table"]: # type: ignore[name-defined] # noqa: F821 + from dlt.common.libs.pyarrow import pyarrow, cast_arrow_schema_types + + if table_format == "delta": + from dlt.common.libs.deltalake import ( + get_delta_tables, + ensure_delta_compatible_arrow_data, + ) + + dt = get_delta_tables(pipeline, table_name)[table_name] + expected = ensure_delta_compatible_arrow_data(arrow_table) + actual = dt.to_pyarrow_table() + elif table_format == "iceberg": + from dlt.common.libs.pyiceberg import ( + get_iceberg_tables, + ensure_iceberg_compatible_arrow_data, + ) + + it = get_iceberg_tables(pipeline, table_name)[table_name] + expected = ensure_iceberg_compatible_arrow_data(arrow_table) + actual = it.scan().to_arrow() + + # work around pyiceberg bug https://github.com/apache/iceberg-python/issues/1128 + schema = cast_arrow_schema_types( + actual.schema, + { + pyarrow.types.is_large_string: pyarrow.string(), + pyarrow.types.is_large_binary: pyarrow.binary(), + }, + ) + actual = actual.cast(schema) + return (expected, actual) + + @pytest.mark.skip( reason="pyarrow version check not needed anymore, since we have 17 as a dependency" ) @@ -258,44 +300,44 @@ def foo(): "destination_config", destinations_configs( table_format_filesystem_configs=True, - with_table_format="delta", + with_table_format=("delta", "iceberg"), bucket_exclude=(MEMORY_BUCKET, SFTP_BUCKET), ), ids=lambda x: x.name, ) -def test_delta_table_core( +def test_table_format_core( destination_config: DestinationTestConfiguration, ) -> None: - """Tests core functionality for `delta` table format. + """Tests core functionality for `delta` and `iceberg` table formats. Tests all data types, all filesystems. Tests `append` and `replace` write dispositions (`merge` is tested elsewhere). """ - - from dlt.common.libs.deltalake import get_delta_tables + if destination_config.table_format == "delta": + from dlt.common.libs.deltalake import get_delta_tables # create resource that yields rows with all data types column_schemas, row = table_update_and_row() - @dlt.resource(columns=column_schemas, table_format="delta") + @dlt.resource(columns=column_schemas, table_format=destination_config.table_format) def data_types(): nonlocal row yield [row] * 10 pipeline = destination_config.setup_pipeline("fs_pipe", dev_mode=True) - # run pipeline, this should create Delta table + # run pipeline, this should create table info = pipeline.run(data_types()) assert_load_info(info) - # `delta` table format should use `parquet` file format + # table formats should use `parquet` file format completed_jobs = info.load_packages[0].jobs["completed_jobs"] data_types_jobs = [ job for job in completed_jobs if job.job_file_info.table_name == "data_types" ] assert all([job.file_path.endswith((".parquet", ".reference")) for job in data_types_jobs]) - # 10 rows should be loaded to the Delta table and the content of the first + # 10 rows should be loaded to the table and the content of the first # row should match expected values rows = load_tables_to_dicts(pipeline, "data_types", exclude_system_cols=True)["data_types"] assert len(rows) == 10 @@ -322,7 +364,8 @@ def data_types(): # should do logical replace, increasing the table version info = pipeline.run(data_types(), write_disposition="replace") assert_load_info(info) - assert get_delta_tables(pipeline, "data_types")["data_types"].version() == 2 + if destination_config.table_format == "delta": + assert get_delta_tables(pipeline, "data_types")["data_types"].version() == 2 rows = load_tables_to_dicts(pipeline, "data_types", exclude_system_cols=True)["data_types"] assert len(rows) == 10 @@ -331,15 +374,16 @@ def data_types(): "destination_config", destinations_configs( table_format_filesystem_configs=True, + # job orchestration is same across table formats—no need to test all formats with_table_format="delta", bucket_subset=(FILE_BUCKET), ), ids=lambda x: x.name, ) -def test_delta_table_does_not_contain_job_files( +def test_table_format_does_not_contain_job_files( destination_config: DestinationTestConfiguration, ) -> None: - """Asserts Parquet job files do not end up in Delta table.""" + """Asserts Parquet job files do not end up in table.""" pipeline = destination_config.setup_pipeline("fs_pipe", dev_mode=True) @@ -376,17 +420,18 @@ def delta_table(): "destination_config", destinations_configs( table_format_filesystem_configs=True, + # job orchestration is same across table formats—no need to test all formats with_table_format="delta", bucket_subset=(FILE_BUCKET), ), ids=lambda x: x.name, ) -def test_delta_table_multiple_files( +def test_table_format_multiple_files( destination_config: DestinationTestConfiguration, ) -> None: - """Tests loading multiple files into a Delta table. + """Tests loading multiple files into a table. - Files should be loaded into the Delta table in a single commit. + Files should be loaded into the table in a single commit. """ from dlt.common.libs.deltalake import get_delta_tables @@ -422,17 +467,17 @@ def delta_table(): "destination_config", destinations_configs( table_format_filesystem_configs=True, - with_table_format="delta", + with_table_format=("delta", "iceberg"), bucket_subset=(FILE_BUCKET), ), ids=lambda x: x.name, ) -def test_delta_table_child_tables( +def test_table_format_child_tables( destination_config: DestinationTestConfiguration, ) -> None: - """Tests child table handling for `delta` table format.""" + """Tests child table handling for `delta` and `iceberg` table formats.""" - @dlt.resource(table_format="delta") + @dlt.resource(table_format=destination_config.table_format) def nested_table(): yield [ { @@ -494,49 +539,63 @@ def nested_table(): assert len(rows_dict["nested_table__child"]) == 3 assert len(rows_dict["nested_table__child__grandchild"]) == 5 - # now drop children and grandchildren, use merge write disposition to create and pass full table chain - # also for tables that do not have jobs - info = pipeline.run( - [{"foo": 3}] * 10000, - table_name="nested_table", - primary_key="foo", - write_disposition="merge", - ) - assert_load_info(info) + if destination_config.supports_merge: + # now drop children and grandchildren, use merge write disposition to create and pass full table chain + # also for tables that do not have jobs + info = pipeline.run( + [{"foo": 3}] * 10000, + table_name="nested_table", + primary_key="foo", + write_disposition="merge", + ) + assert_load_info(info) @pytest.mark.parametrize( "destination_config", destinations_configs( table_format_filesystem_configs=True, - with_table_format="delta", + with_table_format=("delta", "iceberg"), bucket_subset=(FILE_BUCKET), ), ids=lambda x: x.name, ) -def test_delta_table_partitioning( +def test_table_format_partitioning( destination_config: DestinationTestConfiguration, ) -> None: - """Tests partitioning for `delta` table format.""" + """Tests partitioning for `delta` and `iceberg` table formats.""" - from dlt.common.libs.deltalake import get_delta_tables from tests.pipeline.utils import users_materialize_table_schema + def assert_partition_columns( + table_name: str, table_format: TTableFormat, expected_partition_columns: List[str] + ) -> None: + if table_format == "delta": + from dlt.common.libs.deltalake import get_delta_tables + + dt = get_delta_tables(pipeline, table_name)[table_name] + actual_partition_columns = dt.metadata().partition_columns + elif table_format == "iceberg": + from dlt.common.libs.pyiceberg import get_iceberg_tables + + it = get_iceberg_tables(pipeline, table_name)[table_name] + actual_partition_columns = [f.name for f in it.metadata.specs_struct().fields] + assert actual_partition_columns == expected_partition_columns + pipeline = destination_config.setup_pipeline("fs_pipe", dev_mode=True) # zero partition columns - @dlt.resource(table_format="delta") + @dlt.resource(table_format=destination_config.table_format) def zero_part(): yield {"foo": 1, "bar": 1} info = pipeline.run(zero_part()) assert_load_info(info) - dt = get_delta_tables(pipeline, "zero_part")["zero_part"] - assert dt.metadata().partition_columns == [] + assert_partition_columns("zero_part", destination_config.table_format, []) assert load_table_counts(pipeline, "zero_part")["zero_part"] == 1 # one partition column - @dlt.resource(table_format="delta", columns={"c1": {"partition": True}}) + @dlt.resource(table_format=destination_config.table_format, columns={"c1": {"partition": True}}) def one_part(): yield [ {"c1": "foo", "c2": 1}, @@ -547,13 +606,13 @@ def one_part(): info = pipeline.run(one_part()) assert_load_info(info) - dt = get_delta_tables(pipeline, "one_part")["one_part"] - assert dt.metadata().partition_columns == ["c1"] + assert_partition_columns("one_part", destination_config.table_format, ["c1"]) assert load_table_counts(pipeline, "one_part")["one_part"] == 4 # two partition columns @dlt.resource( - table_format="delta", columns={"c1": {"partition": True}, "c2": {"partition": True}} + table_format=destination_config.table_format, + columns={"c1": {"partition": True}, "c2": {"partition": True}}, ) def two_part(): yield [ @@ -565,29 +624,31 @@ def two_part(): info = pipeline.run(two_part()) assert_load_info(info) - dt = get_delta_tables(pipeline, "two_part")["two_part"] - assert dt.metadata().partition_columns == ["c1", "c2"] + assert_partition_columns("two_part", destination_config.table_format, ["c1", "c2"]) assert load_table_counts(pipeline, "two_part")["two_part"] == 4 # test partitioning with empty source users_materialize_table_schema.apply_hints( - table_format="delta", + table_format=destination_config.table_format, columns={"id": {"partition": True}}, ) info = pipeline.run(users_materialize_table_schema()) assert_load_info(info) - dt = get_delta_tables(pipeline, "users")["users"] - assert dt.metadata().partition_columns == ["id"] + assert_partition_columns("users", destination_config.table_format, ["id"]) assert load_table_counts(pipeline, "users")["users"] == 0 # changing partitioning after initial table creation is not supported zero_part.apply_hints(columns={"foo": {"partition": True}}) - with pytest.raises(PipelineStepFailed) as pip_ex: + if destination_config.table_format == "delta": + # Delta raises error when trying to change partitioning + with pytest.raises(PipelineStepFailed) as pip_ex: + pipeline.run(zero_part()) + assert isinstance(pip_ex.value.__context__, LoadClientJobRetry) + assert "partitioning" in pip_ex.value.__context__.retry_message + elif destination_config.table_format == "iceberg": + # while Iceberg supports partition evolution, we don't apply it pipeline.run(zero_part()) - assert isinstance(pip_ex.value.__context__, LoadClientJobRetry) - assert "partitioning" in pip_ex.value.__context__.retry_message - dt = get_delta_tables(pipeline, "zero_part")["zero_part"] - assert dt.metadata().partition_columns == [] + assert_partition_columns("zero_part", destination_config.table_format, []) @pytest.mark.parametrize( @@ -646,7 +707,7 @@ def test_delta_table_partitioning_arrow_load_id( "destination_config", destinations_configs( table_format_filesystem_configs=True, - with_table_format="delta", + with_table_format=("delta", "iceberg"), bucket_subset=(FILE_BUCKET), ), ids=lambda x: x.name, @@ -659,20 +720,25 @@ def test_delta_table_partitioning_arrow_load_id( pytest.param({"disposition": "merge", "strategy": "upsert"}, id="upsert"), ), ) -def test_delta_table_schema_evolution( +def test_table_format_schema_evolution( destination_config: DestinationTestConfiguration, write_disposition: TWriteDisposition, ) -> None: - """Tests schema evolution (adding new columns) for `delta` table format.""" - from dlt.common.libs.deltalake import get_delta_tables, ensure_delta_compatible_arrow_data + """Tests schema evolution (adding new columns) for `delta` and `iceberg` table formats.""" + if destination_config.table_format == "iceberg" and write_disposition == { + "disposition": "merge", + "strategy": "upsert", + }: + pytest.skip("`upsert` currently not implemented for `iceberg`") + from dlt.common.libs.pyarrow import pyarrow @dlt.resource( write_disposition=write_disposition, primary_key="pk", - table_format="delta", + table_format=destination_config.table_format, ) - def delta_table(data): + def evolving_table(data): yield data pipeline = destination_config.setup_pipeline("fs_pipe", dev_mode=True) @@ -684,11 +750,11 @@ def delta_table(data): assert arrow_table.shape == (1, 1) # initial load - info = pipeline.run(delta_table(arrow_table)) + info = pipeline.run(evolving_table(arrow_table)) assert_load_info(info) - dt = get_delta_tables(pipeline, "delta_table")["delta_table"] - expected = ensure_delta_compatible_arrow_data(arrow_table) - actual = dt.to_pyarrow_table() + expected, actual = get_expected_actual( + pipeline, "evolving_table", destination_config.table_format, arrow_table + ) assert actual.equals(expected) # create Arrow table with many columns, two rows @@ -703,11 +769,11 @@ def delta_table(data): arrow_table = arrow_table.add_column(0, pk_field, [[1, 2]]) # second load — this should evolve the schema (i.e. add the new columns) - info = pipeline.run(delta_table(arrow_table)) + info = pipeline.run(evolving_table(arrow_table)) assert_load_info(info) - dt = get_delta_tables(pipeline, "delta_table")["delta_table"] - actual = dt.to_pyarrow_table() - expected = ensure_delta_compatible_arrow_data(arrow_table) + expected, actual = get_expected_actual( + pipeline, "evolving_table", destination_config.table_format, arrow_table + ) if write_disposition == "append": # just check shape and schema for `append`, because table comparison is # more involved than with the other dispositions @@ -724,13 +790,21 @@ def delta_table(data): empty_arrow_table = arrow_table.schema.empty_table() # load 3 — this should evolve the schema without changing data - info = pipeline.run(delta_table(empty_arrow_table)) + info = pipeline.run(evolving_table(empty_arrow_table)) assert_load_info(info) - dt = get_delta_tables(pipeline, "delta_table")["delta_table"] - actual = dt.to_pyarrow_table() - expected_schema = ensure_delta_compatible_arrow_data(arrow_table).schema - assert actual.schema.equals(expected_schema) - expected_num_rows = 3 if write_disposition == "append" else 2 + expected, actual = get_expected_actual( + pipeline, "evolving_table", destination_config.table_format, arrow_table + ) + assert actual.schema.equals(expected.schema) + if write_disposition == "append": + expected_num_rows = 3 + elif write_disposition == "replace": + expected_num_rows = 0 + if destination_config.table_format == "delta": + # TODO: fix https://github.com/dlt-hub/dlt/issues/2092 and remove this if-clause + expected_num_rows = 2 + elif write_disposition == {"disposition": "merge", "strategy": "upsert"}: + expected_num_rows = 2 assert actual.num_rows == expected_num_rows # new column should have NULLs only assert ( @@ -743,23 +817,38 @@ def delta_table(data): "destination_config", destinations_configs( table_format_filesystem_configs=True, - with_table_format="delta", + with_table_format=("delta", "iceberg"), bucket_subset=(FILE_BUCKET, AZ_BUCKET), ), ids=lambda x: x.name, ) -def test_delta_table_empty_source( +def test_table_format_empty_source( destination_config: DestinationTestConfiguration, ) -> None: - """Tests empty source handling for `delta` table format. + """Tests empty source handling for `delta` and `iceberg` table formats. Tests both empty Arrow table and `dlt.mark.materialize_table_schema()`. """ - from dlt.common.libs.deltalake import ensure_delta_compatible_arrow_data, get_delta_tables from tests.pipeline.utils import users_materialize_table_schema - @dlt.resource(table_format="delta") - def delta_table(data): + def get_table_version( # type: ignore[return] + pipeline: dlt.Pipeline, + table_name: str, + table_format: TTableFormat, + ) -> int: + if table_format == "delta": + from dlt.common.libs.deltalake import get_delta_tables + + dt = get_delta_tables(pipeline, table_name)[table_name] + return dt.version() + elif table_format == "iceberg": + from dlt.common.libs.pyiceberg import get_iceberg_tables + + it = get_iceberg_tables(pipeline, table_name)[table_name] + return it.last_sequence_number - 1 # subtract 1 to match `delta` + + @dlt.resource(table_format=destination_config.table_format) + def a_table(data): yield data # create empty Arrow table with schema @@ -779,61 +868,62 @@ def delta_table(data): # run 1: empty Arrow table with schema # this should create empty Delta table with same schema as Arrow table - info = pipeline.run(delta_table(empty_arrow_table)) + info = pipeline.run(a_table(empty_arrow_table)) assert_load_info(info) - dt = get_delta_tables(pipeline, "delta_table")["delta_table"] - assert dt.version() == 0 - dt_arrow_table = dt.to_pyarrow_table() - assert dt_arrow_table.shape == (0, empty_arrow_table.num_columns) - assert dt_arrow_table.schema.equals( - ensure_delta_compatible_arrow_data(empty_arrow_table).schema + assert get_table_version(pipeline, "a_table", destination_config.table_format) == 0 + expected, actual = get_expected_actual( + pipeline, "a_table", destination_config.table_format, empty_arrow_table ) + assert actual.shape == (0, expected.num_columns) + assert actual.schema.equals(expected.schema) # run 2: non-empty Arrow table with same schema as run 1 # this should load records into Delta table - info = pipeline.run(delta_table(arrow_table)) + info = pipeline.run(a_table(arrow_table)) assert_load_info(info) - dt = get_delta_tables(pipeline, "delta_table")["delta_table"] - assert dt.version() == 1 - dt_arrow_table = dt.to_pyarrow_table() - assert dt_arrow_table.shape == (2, empty_arrow_table.num_columns) - assert dt_arrow_table.schema.equals( - ensure_delta_compatible_arrow_data(empty_arrow_table).schema + assert get_table_version(pipeline, "a_table", destination_config.table_format) == 1 + expected, actual = get_expected_actual( + pipeline, "a_table", destination_config.table_format, empty_arrow_table ) + assert actual.shape == (2, expected.num_columns) + assert actual.schema.equals(expected.schema) # now run the empty frame again - info = pipeline.run(delta_table(empty_arrow_table)) + info = pipeline.run(a_table(empty_arrow_table)) assert_load_info(info) - # use materialized list - # NOTE: this will create an empty parquet file with a schema takes from dlt schema. - # the original parquet file had a nested (struct) type in `json` field that is now - # in the delta table schema. the empty parquet file lost this information and had - # string type (converted from dlt `json`) - info = pipeline.run([dlt.mark.materialize_table_schema()], table_name="delta_table") - assert_load_info(info) + if destination_config.table_format == "delta": + # use materialized list + # NOTE: this will create an empty parquet file with a schema takes from dlt schema. + # the original parquet file had a nested (struct) type in `json` field that is now + # in the delta table schema. the empty parquet file lost this information and had + # string type (converted from dlt `json`) + info = pipeline.run([dlt.mark.materialize_table_schema()], table_name="a_table") + assert_load_info(info) # test `dlt.mark.materialize_table_schema()` - users_materialize_table_schema.apply_hints(table_format="delta") + users_materialize_table_schema.apply_hints(table_format=destination_config.table_format) info = pipeline.run(users_materialize_table_schema(), loader_file_format="parquet") assert_load_info(info) - dt = get_delta_tables(pipeline, "users")["users"] - assert dt.version() == 0 - dt_arrow_table = dt.to_pyarrow_table() - assert dt_arrow_table.num_rows == 0 - assert "id", "name" == dt_arrow_table.schema.names[:2] + assert get_table_version(pipeline, "users", destination_config.table_format) == 0 + _, actual = get_expected_actual( + pipeline, "users", destination_config.table_format, empty_arrow_table + ) + assert actual.num_rows == 0 + assert "id", "name" == actual.schema.names[:2] @pytest.mark.parametrize( "destination_config", destinations_configs( table_format_filesystem_configs=True, + # job orchestration is same across table formats—no need to test all formats with_table_format="delta", bucket_subset=(FILE_BUCKET), ), ids=lambda x: x.name, ) -def test_delta_table_mixed_source( +def test_table_format_mixed_source( destination_config: DestinationTestConfiguration, ) -> None: """Tests file format handling in mixed source. @@ -877,12 +967,13 @@ def s(): "destination_config", destinations_configs( table_format_filesystem_configs=True, + # job orchestration is same across table formats—no need to test all formats with_table_format="delta", bucket_subset=(FILE_BUCKET), ), ids=lambda x: x.name, ) -def test_delta_table_dynamic_dispatch( +def test_table_format_dynamic_dispatch( destination_config: DestinationTestConfiguration, ) -> None: @dlt.resource(primary_key="id", table_name=lambda i: i["type"], table_format="delta") @@ -905,80 +996,96 @@ def github_events(): "destination_config", destinations_configs( table_format_filesystem_configs=True, - with_table_format="delta", + with_table_format=("delta", "iceberg"), bucket_subset=(FILE_BUCKET, AZ_BUCKET), ), ids=lambda x: x.name, ) -def test_delta_table_get_delta_tables_helper( +def test_table_format_get_tables_helper( destination_config: DestinationTestConfiguration, ) -> None: - """Tests `get_delta_tables` helper function.""" - from dlt.common.libs.deltalake import DeltaTable, get_delta_tables + """Tests `get_delta_tables` / `get_iceberg_tables` helper functions.""" + get_tables: Any + if destination_config.table_format == "delta": + from dlt.common.libs.deltalake import DeltaTable, get_delta_tables - @dlt.resource(table_format="delta") - def foo_delta(): + get_tables = get_delta_tables + get_num_rows = lambda table: table.to_pyarrow_table().num_rows + elif destination_config.table_format == "iceberg": + from dlt.common.libs.pyiceberg import IcebergTable, get_iceberg_tables + + get_tables = get_iceberg_tables + get_num_rows = lambda table: table.scan().to_arrow().num_rows + + @dlt.resource(table_format=destination_config.table_format) + def foo_table_format(): yield [{"foo": 1}, {"foo": 2}] - @dlt.resource(table_format="delta") - def bar_delta(): + @dlt.resource(table_format=destination_config.table_format) + def bar_table_format(): yield [{"bar": 1}] @dlt.resource - def baz_not_delta(): + def baz_not_table_format(): yield [{"baz": 1}] pipeline = destination_config.setup_pipeline("fs_pipe", dev_mode=True) - info = pipeline.run(foo_delta()) + info = pipeline.run(foo_table_format()) assert_load_info(info) - delta_tables = get_delta_tables(pipeline) - assert delta_tables.keys() == {"foo_delta"} - assert isinstance(delta_tables["foo_delta"], DeltaTable) - assert delta_tables["foo_delta"].to_pyarrow_table().num_rows == 2 - - info = pipeline.run([foo_delta(), bar_delta(), baz_not_delta()]) + tables = get_tables(pipeline) + assert tables.keys() == {"foo_table_format"} + if destination_config.table_format == "delta": + assert isinstance(tables["foo_table_format"], DeltaTable) + elif destination_config.table_format == "iceberg": + assert isinstance(tables["foo_table_format"], IcebergTable) + assert get_num_rows(tables["foo_table_format"]) == 2 + + info = pipeline.run([foo_table_format(), bar_table_format(), baz_not_table_format()]) assert_load_info(info) - delta_tables = get_delta_tables(pipeline) - assert delta_tables.keys() == {"foo_delta", "bar_delta"} - assert delta_tables["bar_delta"].to_pyarrow_table().num_rows == 1 - assert get_delta_tables(pipeline, "foo_delta").keys() == {"foo_delta"} - assert get_delta_tables(pipeline, "bar_delta").keys() == {"bar_delta"} - assert get_delta_tables(pipeline, "foo_delta", "bar_delta").keys() == {"foo_delta", "bar_delta"} + tables = get_tables(pipeline) + assert tables.keys() == {"foo_table_format", "bar_table_format"} + assert get_num_rows(tables["bar_table_format"]) == 1 + assert get_tables(pipeline, "foo_table_format").keys() == {"foo_table_format"} + assert get_tables(pipeline, "bar_table_format").keys() == {"bar_table_format"} + assert get_tables(pipeline, "foo_table_format", "bar_table_format").keys() == { + "foo_table_format", + "bar_table_format", + } # test with child table - @dlt.resource(table_format="delta") - def parent_delta(): + @dlt.resource(table_format=destination_config.table_format) + def parent_table_format(): yield [{"foo": 1, "child": [1, 2, 3]}] - info = pipeline.run(parent_delta()) + info = pipeline.run(parent_table_format()) assert_load_info(info) - delta_tables = get_delta_tables(pipeline) - assert "parent_delta__child" in delta_tables.keys() - assert delta_tables["parent_delta__child"].to_pyarrow_table().num_rows == 3 + tables = get_tables(pipeline) + assert "parent_table_format__child" in tables.keys() + assert get_num_rows(tables["parent_table_format__child"]) == 3 # test invalid input with pytest.raises(ValueError): - get_delta_tables(pipeline, "baz_not_delta") + get_tables(pipeline, "baz_not_table_format") with pytest.raises(ValueError): - get_delta_tables(pipeline, "non_existing_table") + get_tables(pipeline, "non_existing_table") # test unknown schema with pytest.raises(FileNotFoundError): - get_delta_tables(pipeline, "non_existing_table", schema_name="aux_2") + get_tables(pipeline, "non_existing_table", schema_name="aux_2") # load to a new schema and under new name aux_schema = dlt.Schema("aux_2") # NOTE: you cannot have a file with name - info = pipeline.run(parent_delta().with_name("aux_delta"), schema=aux_schema) + info = pipeline.run(parent_table_format().with_name("aux_table"), schema=aux_schema) # also state in seprate package assert_load_info(info, expected_load_packages=2) - delta_tables = get_delta_tables(pipeline, schema_name="aux_2") - assert "aux_delta__child" in delta_tables.keys() - get_delta_tables(pipeline, "aux_delta", schema_name="aux_2") + tables = get_tables(pipeline, schema_name="aux_2") + assert "aux_table__child" in tables.keys() + get_tables(pipeline, "aux_table", schema_name="aux_2") with pytest.raises(ValueError): - get_delta_tables(pipeline, "aux_delta") + get_tables(pipeline, "aux_table") @pytest.mark.parametrize( diff --git a/tests/load/sources/sql_database/test_sql_database_source.py b/tests/load/sources/sql_database/test_sql_database_source.py index 00257471e0..2de923fe38 100644 --- a/tests/load/sources/sql_database/test_sql_database_source.py +++ b/tests/load/sources/sql_database/test_sql_database_source.py @@ -1286,10 +1286,7 @@ def assert_no_precision_columns( ) -> None: actual = list(columns.values()) # we always infer and emit nullability - expected = cast( - List[TColumnSchema], - deepcopy(NULL_NO_PRECISION_COLUMNS if nullable else NOT_NULL_NO_PRECISION_COLUMNS), - ) + expected = deepcopy(NULL_NO_PRECISION_COLUMNS if nullable else NOT_NULL_NO_PRECISION_COLUMNS) if backend == "pyarrow": expected = cast( List[TColumnSchema], diff --git a/tests/load/utils.py b/tests/load/utils.py index 5c24b2d1dc..5660202ec3 100644 --- a/tests/load/utils.py +++ b/tests/load/utils.py @@ -26,7 +26,10 @@ from dlt.common.configuration import resolve_configuration from dlt.common.configuration.container import Container from dlt.common.configuration.specs.config_section_context import ConfigSectionContext -from dlt.common.configuration.specs import CredentialsConfiguration +from dlt.common.configuration.specs import ( + CredentialsConfiguration, + GcpOAuthCredentialsWithoutDefaults, +) from dlt.common.destination.reference import ( DestinationClientDwhConfiguration, JobClientBase, @@ -57,6 +60,7 @@ from dlt.pipeline.exceptions import SqlClientNotAvailable from tests.utils import ( ACTIVE_DESTINATIONS, + ACTIVE_TABLE_FORMATS, IMPLEMENTED_DESTINATIONS, SQL_DESTINATIONS, EXCLUDED_DESTINATION_CONFIGURATIONS, @@ -171,7 +175,9 @@ def destination_factory(self, **kwargs) -> Destination[Any, Any]: dest_type = kwargs.pop("destination", self.destination_type) dest_name = kwargs.pop("destination_name", self.destination_name) self.setup() - return Destination.from_reference(dest_type, destination_name=dest_name, **kwargs) + return Destination.from_reference( + dest_type, self.credentials, destination_name=dest_name, **kwargs + ) def raw_capabilities(self) -> DestinationCapabilitiesContext: dest = Destination.from_reference(self.destination_type) @@ -604,7 +610,7 @@ def destinations_configs( DestinationTestConfiguration( destination_type="filesystem", bucket_url=bucket, - extra_info=bucket + "-delta", + extra_info=bucket, table_format="delta", supports_merge=True, file_format="parquet", @@ -619,12 +625,33 @@ def destinations_configs( ), ) ] + if bucket == AZ_BUCKET: + # `pyiceberg` does not support `az` scheme + continue + destination_configs += [ + DestinationTestConfiguration( + destination_type="filesystem", + bucket_url=bucket, + extra_info=bucket, + table_format="iceberg", + supports_merge=False, + file_format="parquet", + destination_name="fsgcpoauth" if bucket == GCS_BUCKET else None, + ) + ] # filter out non active destinations destination_configs = [ conf for conf in destination_configs if conf.destination_type in ACTIVE_DESTINATIONS ] + # filter out non active table formats + destination_configs = [ + conf + for conf in destination_configs + if conf.table_format is None or conf.table_format in ACTIVE_TABLE_FORMATS + ] + # filter out destinations not in subset if subset: destination_configs = [ diff --git a/tests/pipeline/utils.py b/tests/pipeline/utils.py index 0ae734f72e..e72a27c827 100644 --- a/tests/pipeline/utils.py +++ b/tests/pipeline/utils.py @@ -197,10 +197,23 @@ def _load_tables_to_dicts_fs( delta_tables = get_delta_tables(p, *table_names, schema_name=schema_name) + iceberg_table_names = [ + table_name + for table_name in table_names + if get_table_format(client.schema.tables, table_name) == "iceberg" + ] + if len(iceberg_table_names) > 0: + from dlt.common.libs.pyiceberg import get_iceberg_tables + + iceberg_tables = get_iceberg_tables(p, *table_names, schema_name=schema_name) + for table_name in table_names: if table_name in client.schema.data_table_names() and table_name in delta_table_names: dt = delta_tables[table_name] result[table_name] = dt.to_pyarrow_table().to_pylist() + elif table_name in client.schema.data_table_names() and table_name in iceberg_table_names: + it = iceberg_tables[table_name] + result[table_name] = it.scan().to_arrow().to_pylist() else: table_files = client.list_table_files(table_name) for file in table_files: diff --git a/tests/sources/helpers/rest_client/test_client.py b/tests/sources/helpers/rest_client/test_client.py index 36fe009b93..e67ff9c70a 100644 --- a/tests/sources/helpers/rest_client/test_client.py +++ b/tests/sources/helpers/rest_client/test_client.py @@ -401,7 +401,7 @@ def test_paginate_json_body_without_params(self, rest_client) -> None: posts_skip = (DEFAULT_TOTAL_PAGES - 3) * DEFAULT_PAGE_SIZE class JSONBodyPageCursorPaginator(BaseReferencePaginator): - def update_state(self, response, data): + def update_state(self, response, data): # type: ignore[override] self._next_reference = response.json().get("next_page") def update_request(self, request): diff --git a/tests/utils.py b/tests/utils.py index 1aafa4bfe4..82d742ac65 100644 --- a/tests/utils.py +++ b/tests/utils.py @@ -32,6 +32,7 @@ from dlt.common.runtime.run_context import DOT_DLT, RunContext from dlt.common.runtime.telemetry import start_telemetry, stop_telemetry from dlt.common.schema import Schema +from dlt.common.schema.typing import TTableFormat from dlt.common.storages import FileStorage from dlt.common.storages.versioned_storage import VersionedStorage from dlt.common.typing import DictStrAny, StrAny, TDataItem @@ -88,6 +89,12 @@ ACTIVE_SQL_DESTINATIONS = SQL_DESTINATIONS.intersection(ACTIVE_DESTINATIONS) ACTIVE_NON_SQL_DESTINATIONS = NON_SQL_DESTINATIONS.intersection(ACTIVE_DESTINATIONS) +# filter out active table formats for current tests +IMPLEMENTED_TABLE_FORMATS = set(get_args(TTableFormat)) +ACTIVE_TABLE_FORMATS = set( + dlt.config.get("ACTIVE_TABLE_FORMATS", list) or IMPLEMENTED_TABLE_FORMATS +) + # sanity checks assert len(ACTIVE_DESTINATIONS) >= 0, "No active destinations selected"