diff --git a/.github/workflows/test_dbt_runner.yml b/.github/workflows/test_dbt_runner.yml index 13810fbc0d..ad29909d9a 100644 --- a/.github/workflows/test_dbt_runner.yml +++ b/.github/workflows/test_dbt_runner.yml @@ -60,7 +60,7 @@ jobs: - name: Install dependencies # install dlt with postgres support - run: poetry install --no-interaction -E postgres --with sentry-sdk,dbt + run: poetry install --no-interaction -E postgres -E postgis --with sentry-sdk,dbt - name: create secrets.toml run: pwd && echo "$DLT_SECRETS_TOML" > tests/.dlt/secrets.toml diff --git a/.github/workflows/test_destinations.yml b/.github/workflows/test_destinations.yml index df398e13ad..933248d994 100644 --- a/.github/workflows/test_destinations.yml +++ b/.github/workflows/test_destinations.yml @@ -78,7 +78,7 @@ jobs: - name: Install dependencies # if: steps.cached-poetry-dependencies.outputs.cache-hit != 'true' - run: poetry install --no-interaction -E redshift -E gs -E s3 -E az -E parquet -E duckdb -E cli -E filesystem --with sentry-sdk --with pipeline -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 -E deltalake - 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 61bfe1551a..4947a46a3b 100644 --- a/.github/workflows/test_local_destinations.yml +++ b/.github/workflows/test_local_destinations.yml @@ -48,7 +48,7 @@ jobs: # Label used to access the service container postgres: # Docker Hub image - image: postgres + image: postgis/postgis # Provide the password for postgres env: POSTGRES_DB: dlt_data @@ -95,7 +95,7 @@ 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 duckdb -E parquet -E filesystem -E cli -E weaviate -E qdrant -E sftp --with sentry-sdk --with pipeline -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 -E deltalake - name: Start SFTP server run: docker compose -f "tests/load/filesystem_sftp/docker-compose.yml" up -d diff --git a/.github/workflows/test_local_sources.yml b/.github/workflows/test_local_sources.yml index 8a3ba2a670..39689f5c85 100644 --- a/.github/workflows/test_local_sources.yml +++ b/.github/workflows/test_local_sources.yml @@ -43,7 +43,7 @@ jobs: # Label used to access the service container postgres: # Docker Hub image - image: postgres + image: postgis/postgis # Provide the password for postgres env: POSTGRES_DB: dlt_data @@ -83,7 +83,7 @@ jobs: # TODO: which deps should we enable? - name: Install dependencies - run: poetry install --no-interaction -E postgres -E duckdb -E parquet -E filesystem -E cli -E sql_database --with sentry-sdk,pipeline,sources + run: poetry install --no-interaction -E postgres -E postgis -E duckdb -E parquet -E filesystem -E cli -E sql_database --with sentry-sdk,pipeline,sources # run sources tests in load against configured destinations - run: poetry run pytest tests/load/sources diff --git a/dlt/cli/deploy_command_helpers.py b/dlt/cli/deploy_command_helpers.py index b508b32226..e3719fbe38 100644 --- a/dlt/cli/deploy_command_helpers.py +++ b/dlt/cli/deploy_command_helpers.py @@ -5,7 +5,6 @@ from yaml import Dumper from itertools import chain from typing import List, Optional, Sequence, Tuple, Any, Dict -from astunparse import unparse # optional dependencies import pipdeptree @@ -23,7 +22,7 @@ from dlt.common.git import get_origin, get_repo, Repo from dlt.common.configuration.specs.runtime_configuration import get_default_pipeline_name from dlt.common.typing import StrAny -from dlt.common.reflection.utils import evaluate_node_literal +from dlt.common.reflection.utils import evaluate_node_literal, ast_unparse from dlt.common.pipeline import LoadInfo, TPipelineState, get_dlt_repos_dir from dlt.common.storages import FileStorage from dlt.common.utils import set_working_dir @@ -313,7 +312,7 @@ def parse_pipeline_info(visitor: PipelineScriptVisitor) -> List[Tuple[str, Optio if f_r_value is None: fmt.warning( "The value of `dev_mode` in call to `dlt.pipeline` cannot be" - f" determined from {unparse(f_r_node).strip()}. We assume that you know" + f" determined from {ast_unparse(f_r_node).strip()}. We assume that you know" " what you are doing :)" ) if f_r_value is True: @@ -331,8 +330,8 @@ def parse_pipeline_info(visitor: PipelineScriptVisitor) -> List[Tuple[str, Optio raise CliCommandInnerException( "deploy", "The value of 'pipelines_dir' argument in call to `dlt_pipeline` cannot be" - f" determined from {unparse(p_d_node).strip()}. Pipeline working dir will" - " be found. Pass it directly with --pipelines-dir option.", + f" determined from {ast_unparse(p_d_node).strip()}. Pipeline working dir" + " will be found. Pass it directly with --pipelines-dir option.", ) p_n_node = call_args.arguments.get("pipeline_name") @@ -342,8 +341,8 @@ def parse_pipeline_info(visitor: PipelineScriptVisitor) -> List[Tuple[str, Optio raise CliCommandInnerException( "deploy", "The value of 'pipeline_name' argument in call to `dlt_pipeline` cannot be" - f" determined from {unparse(p_d_node).strip()}. Pipeline working dir will" - " be found. Pass it directly with --pipeline-name option.", + f" determined from {ast_unparse(p_d_node).strip()}. Pipeline working dir" + " will be found. Pass it directly with --pipeline-name option.", ) pipelines.append((pipeline_name, pipelines_dir)) diff --git a/dlt/cli/source_detection.py b/dlt/cli/source_detection.py index f4e9b3e050..7067f8b896 100644 --- a/dlt/cli/source_detection.py +++ b/dlt/cli/source_detection.py @@ -1,11 +1,10 @@ import ast import inspect -from astunparse import unparse from typing import Dict, Tuple, Set, List from dlt.common.configuration import is_secret_hint from dlt.common.configuration.specs import BaseConfiguration -from dlt.common.reflection.utils import creates_func_def_name_node +from dlt.common.reflection.utils import creates_func_def_name_node, ast_unparse from dlt.common.typing import is_optional_type from dlt.sources import SourceReference @@ -65,7 +64,7 @@ def find_source_calls_to_replace( for calls in visitor.known_sources_resources_calls.values(): for call in calls: transformed_nodes.append( - (call.func, ast.Name(id=pipeline_name + "_" + unparse(call.func))) + (call.func, ast.Name(id=pipeline_name + "_" + ast_unparse(call.func))) ) return transformed_nodes diff --git a/dlt/common/configuration/specs/azure_credentials.py b/dlt/common/configuration/specs/azure_credentials.py index c2071e2188..cf6ec493de 100644 --- a/dlt/common/configuration/specs/azure_credentials.py +++ b/dlt/common/configuration/specs/azure_credentials.py @@ -9,15 +9,32 @@ configspec, ) from dlt import version +from dlt.common.utils import without_none _AZURE_STORAGE_EXTRA = f"{version.DLT_PKG_NAME}[az]" @configspec -class AzureCredentialsWithoutDefaults(CredentialsConfiguration): +class AzureCredentialsBase(CredentialsConfiguration): + 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""" + + def to_adlfs_credentials(self) -> Dict[str, Any]: + pass + + def to_object_store_rs_credentials(self) -> Dict[str, str]: + # https://docs.rs/object_store/latest/object_store/azure + creds: Dict[str, Any] = without_none(self.to_adlfs_credentials()) # type: ignore[assignment] + # only string options accepted + creds.pop("anon", None) + return creds + + +@configspec +class AzureCredentialsWithoutDefaults(AzureCredentialsBase): """Credentials for Azure Blob Storage, compatible with adlfs""" - azure_storage_account_name: str = None azure_storage_account_key: Optional[TSecretStrValue] = None azure_storage_sas_token: TSecretStrValue = None azure_sas_token_permissions: str = "racwdl" @@ -29,17 +46,9 @@ def to_adlfs_credentials(self) -> Dict[str, Any]: account_name=self.azure_storage_account_name, account_key=self.azure_storage_account_key, sas_token=self.azure_storage_sas_token, + account_host=self.azure_account_host, ) - def to_object_store_rs_credentials(self) -> Dict[str, str]: - # https://docs.rs/object_store/latest/object_store/azure - creds = self.to_adlfs_credentials() - if creds["sas_token"] is None: - creds.pop("sas_token") - if creds["account_key"] is None: - creds.pop("account_key") - return creds - def create_sas_token(self) -> None: try: from azure.storage.blob import generate_account_sas, ResourceTypes @@ -63,8 +72,7 @@ def on_partial(self) -> None: @configspec -class AzureServicePrincipalCredentialsWithoutDefaults(CredentialsConfiguration): - azure_storage_account_name: str = None +class AzureServicePrincipalCredentialsWithoutDefaults(AzureCredentialsBase): azure_tenant_id: str = None azure_client_id: str = None azure_client_secret: TSecretStrValue = None @@ -72,15 +80,12 @@ class AzureServicePrincipalCredentialsWithoutDefaults(CredentialsConfiguration): def to_adlfs_credentials(self) -> Dict[str, Any]: return dict( account_name=self.azure_storage_account_name, + account_host=self.azure_account_host, tenant_id=self.azure_tenant_id, client_id=self.azure_client_id, client_secret=self.azure_client_secret, ) - def to_object_store_rs_credentials(self) -> Dict[str, str]: - # https://docs.rs/object_store/latest/object_store/azure - return self.to_adlfs_credentials() - @configspec class AzureCredentials(AzureCredentialsWithoutDefaults, CredentialsWithDefault): diff --git a/dlt/common/destination/reference.py b/dlt/common/destination/reference.py index 6c198dd468..e27f99cde7 100644 --- a/dlt/common/destination/reference.py +++ b/dlt/common/destination/reference.py @@ -76,12 +76,15 @@ try: from dlt.common.libs.pandas import DataFrame from dlt.common.libs.pyarrow import Table as ArrowTable + from dlt.common.libs.ibis import BaseBackend as IbisBackend except MissingDependencyException: DataFrame = Any ArrowTable = Any + IbisBackend = Any else: DataFrame = Any ArrowTable = Any + IbisBackend = Any class StorageSchemaInfo(NamedTuple): @@ -291,7 +294,6 @@ def _make_dataset_name(self, schema_name: str) -> str: # if default schema is None then suffix is not added if self.default_schema_name is not None and schema_name != self.default_schema_name: return (self.dataset_name or "") + "_" + schema_name - return self.dataset_name @@ -443,8 +445,9 @@ def run_managed( self._finished_at = pendulum.now() # sanity check assert self._state in ("completed", "retry", "failed") - # wake up waiting threads - signals.wake_all() + if self._state != "retry": + # wake up waiting threads + signals.wake_all() @abstractmethod def run(self) -> None: @@ -574,12 +577,17 @@ def close(self) -> None: ... class SupportsReadableDataset(Protocol): """A readable dataset retrieved from a destination, has support for creating readable relations for a query or table""" + @property + def schema(self) -> Schema: ... + def __call__(self, query: Any) -> SupportsReadableRelation: ... def __getitem__(self, table: str) -> SupportsReadableRelation: ... def __getattr__(self, table: str) -> SupportsReadableRelation: ... + def ibis(self) -> IbisBackend: ... + class JobClientBase(ABC): def __init__( diff --git a/dlt/common/destination/typing.py b/dlt/common/destination/typing.py index 8cc08756cd..c79a2b0adc 100644 --- a/dlt/common/destination/typing.py +++ b/dlt/common/destination/typing.py @@ -1,6 +1,10 @@ from typing import Optional -from dlt.common.schema.typing import _TTableSchemaBase, TWriteDisposition, TTableReferenceParam +from dlt.common.schema.typing import ( + _TTableSchemaBase, + TWriteDisposition, + TTableReferenceParam, +) class PreparedTableSchema(_TTableSchemaBase, total=False): diff --git a/dlt/common/incremental/__init__.py b/dlt/common/incremental/__init__.py new file mode 100644 index 0000000000..e69de29bb2 diff --git a/dlt/extract/incremental/typing.py b/dlt/common/incremental/typing.py similarity index 66% rename from dlt/extract/incremental/typing.py rename to dlt/common/incremental/typing.py index 7b7786b529..460e2f234b 100644 --- a/dlt/extract/incremental/typing.py +++ b/dlt/common/incremental/typing.py @@ -2,9 +2,7 @@ from typing import Any, Callable, List, Literal, Optional, Sequence, TypeVar, Union -from dlt.common.schema.typing import TColumnNames -from dlt.common.typing import TSortOrder -from dlt.extract.items import TTableHintTemplate +from dlt.common.typing import TSortOrder, TTableHintTemplate, TColumnNames TCursorValue = TypeVar("TCursorValue", bound=Any) LastValueFunc = Callable[[Sequence[TCursorValue]], Any] @@ -19,10 +17,12 @@ class IncrementalColumnState(TypedDict): class IncrementalArgs(TypedDict, total=False): cursor_path: str - initial_value: Optional[str] - last_value_func: Optional[LastValueFunc[str]] + initial_value: Optional[Any] + last_value_func: Optional[Union[LastValueFunc[str], Literal["min", "max"]]] + """Last value callable or name of built in function""" primary_key: Optional[TTableHintTemplate[TColumnNames]] - end_value: Optional[str] + end_value: Optional[Any] row_order: Optional[TSortOrder] allow_external_schedulers: Optional[bool] lag: Optional[Union[float, int]] + on_cursor_value_missing: Optional[OnCursorValueMissing] diff --git a/dlt/common/jsonpath.py b/dlt/common/jsonpath.py index 46004a07d6..13f63d336c 100644 --- a/dlt/common/jsonpath.py +++ b/dlt/common/jsonpath.py @@ -1,4 +1,4 @@ -from typing import Iterable, Union, List, Any +from typing import Iterable, Union, List, Any, Optional, cast from itertools import chain from dlt.common.typing import DictStrAny @@ -46,3 +46,48 @@ def resolve_paths(paths: TAnyJsonPath, data: DictStrAny) -> List[str]: paths = compile_paths(paths) p: JSONPath return list(chain.from_iterable((str(r.full_path) for r in p.find(data)) for p in paths)) + + +def is_simple_field_path(path: JSONPath) -> bool: + """Checks if the given path represents a simple single field name. + + Example: + >>> is_simple_field_path(compile_path('id')) + True + >>> is_simple_field_path(compile_path('$.id')) + False + """ + return isinstance(path, JSONPathFields) and len(path.fields) == 1 and path.fields[0] != "*" + + +def extract_simple_field_name(path: Union[str, JSONPath]) -> Optional[str]: + """ + Extracts a simple field name from a JSONPath if it represents a single field access. + Returns None if the path is complex (contains wildcards, array indices, or multiple fields). + + Args: + path: A JSONPath object or string + + Returns: + Optional[str]: The field name if path represents a simple field access, None otherwise + + Example: + >>> extract_simple_field_name('name') + 'name' + >>> extract_simple_field_name('"name"') + 'name' + >>> extract_simple_field_name('"na$me"') # Escaped characters are preserved + 'na$me' + >>> extract_simple_field_name('"na.me"') # Escaped characters are preserved + 'na.me' + >>> extract_simple_field_name('$.name') # Returns None + >>> extract_simple_field_name('$.items[*].name') # Returns None + >>> extract_simple_field_name('*') # Returns None + """ + if isinstance(path, str): + path = compile_path(path) + + if is_simple_field_path(path): + return cast(str, path.fields[0]) + + return None diff --git a/dlt/common/libs/ibis.py b/dlt/common/libs/ibis.py new file mode 100644 index 0000000000..ba6f363e66 --- /dev/null +++ b/dlt/common/libs/ibis.py @@ -0,0 +1,121 @@ +from typing import cast + +from dlt.common.exceptions import MissingDependencyException + +from dlt.common.destination.reference import TDestinationReferenceArg, Destination, JobClientBase + +try: + import ibis # type: ignore + from ibis import BaseBackend +except ModuleNotFoundError: + raise MissingDependencyException("dlt ibis Helpers", ["ibis"]) + + +SUPPORTED_DESTINATIONS = [ + "dlt.destinations.postgres", + "dlt.destinations.duckdb", + "dlt.destinations.motherduck", + "dlt.destinations.filesystem", + "dlt.destinations.bigquery", + "dlt.destinations.snowflake", + "dlt.destinations.redshift", + "dlt.destinations.mssql", + "dlt.destinations.synapse", + "dlt.destinations.clickhouse", + # NOTE: Athena could theoretically work with trino backend, but according to + # https://github.com/ibis-project/ibis/issues/7682 connecting with aws credentials + # does not work yet. + # "dlt.destinations.athena", +] + + +def create_ibis_backend( + destination: TDestinationReferenceArg, client: JobClientBase +) -> BaseBackend: + """Create a given ibis backend for a destination client and dataset""" + + # check if destination is supported + destination_type = Destination.from_reference(destination).destination_type + if destination_type not in SUPPORTED_DESTINATIONS: + raise NotImplementedError(f"Destination of type {destination_type} not supported by ibis.") + + if destination_type in ["dlt.destinations.motherduck", "dlt.destinations.duckdb"]: + import duckdb + from dlt.destinations.impl.duckdb.duck import DuckDbClient + + duck_client = cast(DuckDbClient, client) + duck = duckdb.connect( + database=duck_client.config.credentials._conn_str(), + read_only=duck_client.config.credentials.read_only, + config=duck_client.config.credentials._get_conn_config(), + ) + con = ibis.duckdb.from_connection(duck) + elif destination_type in [ + "dlt.destinations.postgres", + "dlt.destinations.redshift", + ]: + credentials = client.config.credentials.to_native_representation() + con = ibis.connect(credentials) + elif destination_type == "dlt.destinations.snowflake": + from dlt.destinations.impl.snowflake.snowflake import SnowflakeClient + + sf_client = cast(SnowflakeClient, client) + credentials = sf_client.config.credentials.to_connector_params() + con = ibis.snowflake.connect(**credentials) + elif destination_type in ["dlt.destinations.mssql", "dlt.destinations.synapse"]: + from dlt.destinations.impl.mssql.mssql import MsSqlJobClient + + mssql_client = cast(MsSqlJobClient, client) + con = ibis.mssql.connect( + host=mssql_client.config.credentials.host, + port=mssql_client.config.credentials.port, + database=mssql_client.config.credentials.database, + user=mssql_client.config.credentials.username, + password=mssql_client.config.credentials.password, + driver=mssql_client.config.credentials.driver, + ) + elif destination_type == "dlt.destinations.bigquery": + from dlt.destinations.impl.bigquery.bigquery import BigQueryClient + + bq_client = cast(BigQueryClient, client) + credentials = bq_client.config.credentials.to_native_credentials() + con = ibis.bigquery.connect( + credentials=credentials, + project_id=bq_client.sql_client.project_id, + location=bq_client.sql_client.location, + ) + elif destination_type == "dlt.destinations.clickhouse": + from dlt.destinations.impl.clickhouse.clickhouse import ClickHouseClient + + ch_client = cast(ClickHouseClient, client) + con = ibis.clickhouse.connect( + host=ch_client.config.credentials.host, + port=ch_client.config.credentials.http_port, + database=ch_client.config.credentials.database, + user=ch_client.config.credentials.username, + password=ch_client.config.credentials.password, + secure=bool(ch_client.config.credentials.secure), + # compression=True, + ) + elif destination_type == "dlt.destinations.filesystem": + import duckdb + from dlt.destinations.impl.filesystem.sql_client import ( + FilesystemClient, + FilesystemSqlClient, + ) + from dlt.destinations.impl.duckdb.factory import DuckDbCredentials + + # we create an in memory duckdb and create all tables on there + duck = duckdb.connect(":memory:") + fs_client = cast(FilesystemClient, client) + creds = DuckDbCredentials(duck) + sql_client = FilesystemSqlClient( + fs_client, dataset_name=fs_client.dataset_name, credentials=creds + ) + + # NOTE: we should probably have the option for the user to only select a subset of tables here + with sql_client as _: + sql_client.create_views_for_all_tables() + con = ibis.duckdb.from_connection(duck) + + return con diff --git a/dlt/common/libs/pandas.py b/dlt/common/libs/pandas.py index a165ea8747..35cfe623bb 100644 --- a/dlt/common/libs/pandas.py +++ b/dlt/common/libs/pandas.py @@ -8,8 +8,9 @@ raise MissingDependencyException("dlt Pandas Helpers", ["pandas"]) -def pandas_to_arrow(df: pandas.DataFrame) -> Any: +def pandas_to_arrow(df: pandas.DataFrame, preserve_index: bool = False) -> Any: """Converts pandas to arrow or raises an exception if pyarrow is not installed""" from dlt.common.libs.pyarrow import pyarrow as pa - return pa.Table.from_pandas(df) + # NOTE: None preserves named indexes but ignores unnamed + return pa.Table.from_pandas(df, preserve_index=preserve_index) diff --git a/dlt/common/libs/pyarrow.py b/dlt/common/libs/pyarrow.py index 14e31b4603..029cd75399 100644 --- a/dlt/common/libs/pyarrow.py +++ b/dlt/common/libs/pyarrow.py @@ -593,7 +593,7 @@ def row_tuples_to_arrow( pivoted_rows = np.asarray(rows, dtype="object", order="k").T # type: ignore[call-overload] columnar = { - col: dat.ravel() for col, dat in zip(columns, np.vsplit(pivoted_rows, len(columns))) + col: dat.ravel() for col, dat in zip(columns, np.vsplit(pivoted_rows, len(pivoted_rows))) } columnar_known_types = { col["name"]: columnar[col["name"]] @@ -620,7 +620,7 @@ def row_tuples_to_arrow( ) float_array = pa.array(columnar_known_types[field.name], type=pa.float64()) columnar_known_types[field.name] = float_array.cast(field.type, safe=False) - if issubclass(py_type, (dict, list)): + if issubclass(py_type, (dict, list, set)): logger.warning( f"Field {field.name} was reflected as JSON type and needs to be serialized back to" " string to be placed in arrow table. This will slow data extraction down. You" @@ -628,7 +628,14 @@ def row_tuples_to_arrow( " extracting an SQL VIEW that selects with cast." ) json_str_array = pa.array( - [None if s is None else json.dumps(s) for s in columnar_known_types[field.name]] + [ + ( + None + if s is None + else json.dumps(s) if not issubclass(type(s), set) else json.dumps(list(s)) + ) + for s in columnar_known_types[field.name] + ] ) columnar_known_types[field.name] = json_str_array @@ -669,7 +676,7 @@ def row_tuples_to_arrow( pa.field( key, arrow_col.type, - nullable=columns[key]["nullable"], + nullable=columns[key].get("nullable", True), ) ) diff --git a/dlt/common/libs/sql_alchemy.py b/dlt/common/libs/sql_alchemy.py index 19ebbbc78a..edc71c01d0 100644 --- a/dlt/common/libs/sql_alchemy.py +++ b/dlt/common/libs/sql_alchemy.py @@ -4,7 +4,8 @@ try: from sqlalchemy import MetaData, Table, Column, create_engine from sqlalchemy.engine import Engine, URL, make_url, Row - from sqlalchemy.sql import sqltypes, Select + from sqlalchemy.sql import sqltypes, Select, Executable + from sqlalchemy.sql.elements import TextClause from sqlalchemy.sql.sqltypes import TypeEngine from sqlalchemy.exc import CompileError import sqlalchemy as sa @@ -18,3 +19,22 @@ # TODO: maybe use sa.__version__? IS_SQL_ALCHEMY_20 = hasattr(sa, "Double") + +__all__ = [ + "IS_SQL_ALCHEMY_20", + "MetaData", + "Table", + "Column", + "create_engine", + "Engine", + "URL", + "make_url", + "Row", + "sqltypes", + "Select", + "Executable", + "TextClause", + "TypeEngine", + "CompileError", + "sa", +] diff --git a/dlt/common/normalizers/json/helpers.py b/dlt/common/normalizers/json/helpers.py new file mode 100644 index 0000000000..96c9ab4954 --- /dev/null +++ b/dlt/common/normalizers/json/helpers.py @@ -0,0 +1,141 @@ +""" +Cached helper methods for all operations that are called often +""" +from functools import lru_cache +from typing import Any, Dict, List, Optional, Tuple, cast + +from dlt.common.json import json +from dlt.common.destination.utils import resolve_merge_strategy +from dlt.common.normalizers.naming import NamingConvention +from dlt.common.normalizers.typing import TRowIdType +from dlt.common.normalizers.utils import DLT_ID_LENGTH_BYTES +from dlt.common.schema import Schema +from dlt.common.schema.typing import TColumnSchema, C_DLT_ID, DLT_NAME_PREFIX +from dlt.common.schema.utils import ( + get_columns_names_with_prop, + get_first_column_name_with_prop, + is_nested_table, +) +from dlt.common.utils import digest128 + + +@lru_cache(maxsize=None) +def shorten_fragments(naming: NamingConvention, *idents: str) -> str: + return naming.shorten_fragments(*idents) + + +@lru_cache(maxsize=None) +def normalize_table_identifier(schema: Schema, naming: NamingConvention, table_name: str) -> str: + if schema._normalizers_config.get("use_break_path_on_normalize", True): + return naming.normalize_tables_path(table_name) + else: + return naming.normalize_table_identifier(table_name) + + +@lru_cache(maxsize=None) +def normalize_identifier(schema: Schema, naming: NamingConvention, identifier: str) -> str: + if schema._normalizers_config.get("use_break_path_on_normalize", True): + return naming.normalize_path(identifier) + else: + return naming.normalize_identifier(identifier) + + +@lru_cache(maxsize=None) +def get_table_nesting_level( + schema: Schema, table_name: str, default_nesting: int = 1000 +) -> Optional[int]: + """gets table nesting level, will inherit from parent if not set""" + + table = schema.tables.get(table_name) + if ( + table + and (max_nesting := cast(int, table.get("x-normalizer", {}).get("max_nesting"))) is not None + ): + return max_nesting + return default_nesting + + +@lru_cache(maxsize=None) +def get_primary_key(schema: Schema, table_name: str) -> List[str]: + if table_name not in schema.tables: + return [] + table = schema.get_table(table_name) + return get_columns_names_with_prop(table, "primary_key", include_incomplete=True) + + +@lru_cache(maxsize=None) +def is_nested_type( + schema: Schema, + table_name: str, + field_name: str, + _r_lvl: int, +) -> bool: + """For those paths the nested objects should be left in place. + Cache perf: max_nesting < _r_lvl: ~2x faster, full check 10x faster + """ + + # nesting level is counted backwards + # is we have traversed to or beyond the calculated nesting level, we detect a nested type + if _r_lvl <= 0: + return True + + column: TColumnSchema = None + table = schema.tables.get(table_name) + if table: + column = table["columns"].get(field_name) + if column is None or "data_type" not in column: + data_type = schema.get_preferred_type(field_name) + else: + data_type = column["data_type"] + + return data_type == "json" + + +@lru_cache(maxsize=None) +def get_nested_row_id_type(schema: Schema, table_name: str) -> Tuple[TRowIdType, bool]: + """Gets type of row id to be added to nested table and if linking information should be added""" + if table := schema.tables.get(table_name): + merge_strategy = resolve_merge_strategy(schema.tables, table) + if merge_strategy not in ("upsert", "scd2") and not is_nested_table(table): + return "random", False + else: + # table will be created, use standard linking + pass + return "row_hash", True + + +@lru_cache(maxsize=None) +def get_root_row_id_type(schema: Schema, table_name: str) -> TRowIdType: + if table := schema.tables.get(table_name): + merge_strategy = resolve_merge_strategy(schema.tables, table) + if merge_strategy == "upsert": + return "key_hash" + elif merge_strategy == "scd2": + x_row_version_col = get_first_column_name_with_prop( + schema.get_table(table_name), + "x-row-version", + include_incomplete=True, + ) + if x_row_version_col == schema.naming.normalize_identifier(C_DLT_ID): + return "row_hash" + return "random" + + +def get_row_hash(row: Dict[str, Any], subset: Optional[List[str]] = None) -> str: + """Returns hash of row. + + Hash includes column names and values and is ordered by column name. + Excludes dlt system columns. + Can be used as deterministic row identifier. + """ + row_filtered = {k: v for k, v in row.items() if not k.startswith(DLT_NAME_PREFIX)} + if subset is not None: + row_filtered = {k: v for k, v in row.items() if k in subset} + row_str = json.dumps(row_filtered, sort_keys=True) + return digest128(row_str, DLT_ID_LENGTH_BYTES) + + +def get_nested_row_hash(parent_row_id: str, nested_table: str, list_idx: int) -> str: + # create deterministic unique id of the nested row taking into account that all lists are ordered + # and all nested tables must be lists + return digest128(f"{parent_row_id}_{nested_table}_{list_idx}", DLT_ID_LENGTH_BYTES) diff --git a/dlt/common/normalizers/json/relational.py b/dlt/common/normalizers/json/relational.py index c5338192a0..e365017125 100644 --- a/dlt/common/normalizers/json/relational.py +++ b/dlt/common/normalizers/json/relational.py @@ -1,34 +1,27 @@ -from functools import lru_cache from typing import Dict, List, Mapping, Optional, Sequence, Tuple, cast, TypedDict, Any -from dlt.common.destination.utils import resolve_merge_strategy -from dlt.common.json import json -from dlt.common.normalizers.exceptions import InvalidJsonNormalizer -from dlt.common.normalizers.typing import TJSONNormalizer, TRowIdType -from dlt.common.normalizers.utils import generate_dlt_id, DLT_ID_LENGTH_BYTES +from dlt.common.normalizers.exceptions import InvalidJsonNormalizer +from dlt.common.normalizers.typing import TJSONNormalizer +from dlt.common.normalizers.utils import generate_dlt_id from dlt.common.typing import DictStrAny, TDataItem, StrAny from dlt.common.schema import Schema from dlt.common.schema.typing import ( C_DLT_ID, C_DLT_LOAD_ID, - TColumnSchema, TColumnName, TSimpleRegex, - DLT_NAME_PREFIX, ) from dlt.common.schema.utils import ( column_name_validator, - get_columns_names_with_prop, - get_first_column_name_with_prop, - has_column_with_prop, is_nested_table, ) -from dlt.common.utils import digest128, update_dict_nested +from dlt.common.utils import update_dict_nested from dlt.common.normalizers.json import ( TNormalizedRowIterator, wrap_in_dict, DataItemNormalizer as DataItemNormalizerBase, ) +from dlt.common.normalizers.json import helpers from dlt.common.validation import validate_dict @@ -103,18 +96,18 @@ def _flatten( def norm_row_dicts(dict_row: StrAny, __r_lvl: int, path: Tuple[str, ...] = ()) -> None: for k, v in dict_row.items(): if k.strip(): - norm_k = self._normalize_identifier(self.schema, k) + norm_k = helpers.normalize_identifier(self.schema, self.naming, k) else: # for empty keys in the data use _ norm_k = self.EMPTY_KEY_IDENTIFIER # if norm_k != k: # print(f"{k} -> {norm_k}") nested_name = ( - norm_k if path == () else self._shorten_fragments(self.schema, *path, norm_k) + norm_k if path == () else helpers.shorten_fragments(self.naming, *path, norm_k) ) # for lists and dicts we must check if type is possibly nested if isinstance(v, (dict, list)): - if not self._is_nested_type(self.schema, table, nested_name, __r_lvl): + if not helpers.is_nested_type(self.schema, table, nested_name, __r_lvl): # TODO: if schema contains table {table}__{nested_name} then convert v into single element list if isinstance(v, dict): # flatten the dict more @@ -122,7 +115,8 @@ def norm_row_dicts(dict_row: StrAny, __r_lvl: int, path: Tuple[str, ...] = ()) - else: # pass the list to out_rec_list out_rec_list[ - path + (self._normalize_table_identifier(self.schema, k),) + path + + (helpers.normalize_table_identifier(self.schema, self.naming, k),) ] = v continue else: @@ -134,26 +128,6 @@ def norm_row_dicts(dict_row: StrAny, __r_lvl: int, path: Tuple[str, ...] = ()) - norm_row_dicts(dict_row, _r_lvl) return out_rec_row, out_rec_list - @staticmethod - def get_row_hash(row: Dict[str, Any], subset: Optional[List[str]] = None) -> str: - """Returns hash of row. - - Hash includes column names and values and is ordered by column name. - Excludes dlt system columns. - Can be used as deterministic row identifier. - """ - row_filtered = {k: v for k, v in row.items() if not k.startswith(DLT_NAME_PREFIX)} - if subset is not None: - row_filtered = {k: v for k, v in row.items() if k in subset} - row_str = json.dumps(row_filtered, sort_keys=True) - return digest128(row_str, DLT_ID_LENGTH_BYTES) - - @staticmethod - def _get_nested_row_hash(parent_row_id: str, nested_table: str, list_idx: int) -> str: - # create deterministic unique id of the nested row taking into account that all lists are ordered - # and all nested tables must be lists - return digest128(f"{parent_row_id}_{nested_table}_{list_idx}", DLT_ID_LENGTH_BYTES) - def _link_row(self, row: DictStrAny, parent_row_id: str, list_idx: int) -> DictStrAny: assert parent_row_id row[self.c_dlt_parent_id] = parent_row_id @@ -175,20 +149,20 @@ def _add_row_id( is_root: bool = False, ) -> str: if is_root: # root table - row_id_type = self._get_root_row_id_type(self.schema, table) + row_id_type = helpers.get_root_row_id_type(self.schema, table) if row_id_type in ("key_hash", "row_hash"): subset = None if row_id_type == "key_hash": - subset = self._get_primary_key(self.schema, table) + subset = helpers.get_primary_key(self.schema, table) # base hash on `dict_row` instead of `flattened_row` # so changes in nested tables lead to new row id - row_id = self.get_row_hash(dict_row, subset=subset) + row_id = helpers.get_row_hash(dict_row, subset=subset) else: row_id = generate_dlt_id() else: # nested table - row_id_type, is_nested = self._get_nested_row_id_type(self.schema, table) + row_id_type, is_nested = helpers.get_nested_row_id_type(self.schema, table) if row_id_type == "row_hash": - row_id = DataItemNormalizer._get_nested_row_hash(parent_row_id, table, pos) + row_id = helpers.get_nested_row_hash(parent_row_id, table, pos) # link to parent table if is_nested: self._link_row(flattened_row, parent_row_id, pos) @@ -227,7 +201,7 @@ def _normalize_list( parent_row_id: Optional[str] = None, _r_lvl: int = 0, ) -> TNormalizedRowIterator: - table = self._shorten_fragments(self.schema, *parent_path, *ident_path) + table = helpers.shorten_fragments(self.naming, *parent_path, *ident_path) for idx, v in enumerate(seq): if isinstance(v, dict): @@ -251,7 +225,7 @@ def _normalize_list( wrap_v = wrap_in_dict(self.c_value, v) DataItemNormalizer._extend_row(extend, wrap_v) self._add_row_id(table, wrap_v, wrap_v, parent_row_id, idx) - yield (table, self._shorten_fragments(self.schema, *parent_path)), wrap_v + yield (table, helpers.shorten_fragments(self.naming, *parent_path)), wrap_v def _normalize_row( self, @@ -264,8 +238,8 @@ def _normalize_row( _r_lvl: int = 0, is_root: bool = False, ) -> TNormalizedRowIterator: - schema = self.schema - table = self._shorten_fragments(schema, *parent_path, *ident_path) + naming = self.naming + table = helpers.shorten_fragments(naming, *parent_path, *ident_path) # flatten current row and extract all lists to recur into flattened_row, lists = self._flatten(table, dict_row, _r_lvl) # always extend row @@ -280,7 +254,7 @@ def _normalize_row( # yield parent table first should_descend = yield ( - (table, self._shorten_fragments(schema, *parent_path)), + (table, helpers.shorten_fragments(naming, *parent_path)), flattened_row, ) if should_descend is False: @@ -361,8 +335,10 @@ def normalize_data_item( # identify load id if loaded data must be processed after loading incrementally row[self.c_dlt_load_id] = load_id # get table name and nesting level - root_table_name = self._normalize_table_identifier(self.schema, table_name) - max_nesting = self._get_table_nesting_level(self.schema, root_table_name, self.max_nesting) + root_table_name = helpers.normalize_table_identifier(self.schema, self.naming, table_name) + max_nesting = helpers.get_table_nesting_level( + self.schema, root_table_name, self.max_nesting + ) yield from self._normalize_row( row, @@ -426,103 +402,3 @@ def _normalize_prop( "./normalizers/json/config", validator_f=column_name_validator(schema.naming), ) - - # - # Cached helper methods for all operations that are called often - # - @staticmethod - @lru_cache(maxsize=None) - def _shorten_fragments(schema: Schema, *idents: str) -> str: - return schema.naming.shorten_fragments(*idents) - - @staticmethod - @lru_cache(maxsize=None) - def _normalize_table_identifier(schema: Schema, table_name: str) -> str: - return schema.naming.normalize_table_identifier(table_name) - - @staticmethod - @lru_cache(maxsize=None) - def _normalize_identifier(schema: Schema, identifier: str) -> str: - return schema.naming.normalize_path(identifier) - - @staticmethod - @lru_cache(maxsize=None) - def _get_table_nesting_level( - schema: Schema, table_name: str, default_nesting: int = 1000 - ) -> Optional[int]: - """gets table nesting level, will inherit from parent if not set""" - - table = schema.tables.get(table_name) - if ( - table - and (max_nesting := cast(int, table.get("x-normalizer", {}).get("max_nesting"))) - is not None - ): - return max_nesting - return default_nesting - - @staticmethod - @lru_cache(maxsize=None) - def _get_primary_key(schema: Schema, table_name: str) -> List[str]: - if table_name not in schema.tables: - return [] - table = schema.get_table(table_name) - return get_columns_names_with_prop(table, "primary_key", include_incomplete=True) - - @staticmethod - @lru_cache(maxsize=None) - def _is_nested_type( - schema: Schema, - table_name: str, - field_name: str, - _r_lvl: int, - ) -> bool: - """For those paths the nested objects should be left in place. - Cache perf: max_nesting < _r_lvl: ~2x faster, full check 10x faster - """ - - # nesting level is counted backwards - # is we have traversed to or beyond the calculated nesting level, we detect a nested type - if _r_lvl <= 0: - return True - - column: TColumnSchema = None - table = schema.tables.get(table_name) - if table: - column = table["columns"].get(field_name) - if column is None or "data_type" not in column: - data_type = schema.get_preferred_type(field_name) - else: - data_type = column["data_type"] - - return data_type == "json" - - @staticmethod - @lru_cache(maxsize=None) - def _get_nested_row_id_type(schema: Schema, table_name: str) -> Tuple[TRowIdType, bool]: - """Gets type of row id to be added to nested table and if linking information should be added""" - if table := schema.tables.get(table_name): - merge_strategy = resolve_merge_strategy(schema.tables, table) - if merge_strategy not in ("upsert", "scd2") and not is_nested_table(table): - return "random", False - else: - # table will be created, use standard linking - pass - return "row_hash", True - - @staticmethod - @lru_cache(maxsize=None) - def _get_root_row_id_type(schema: Schema, table_name: str) -> TRowIdType: - if table := schema.tables.get(table_name): - merge_strategy = resolve_merge_strategy(schema.tables, table) - if merge_strategy == "upsert": - return "key_hash" - elif merge_strategy == "scd2": - x_row_version_col = get_first_column_name_with_prop( - schema.get_table(table_name), - "x-row-version", - include_incomplete=True, - ) - if x_row_version_col == schema.naming.normalize_identifier(C_DLT_ID): - return "row_hash" - return "random" diff --git a/dlt/common/normalizers/naming/naming.py b/dlt/common/normalizers/naming/naming.py index 5ae5847963..9953d25913 100644 --- a/dlt/common/normalizers/naming/naming.py +++ b/dlt/common/normalizers/naming/naming.py @@ -45,6 +45,8 @@ def make_path(self, *identifiers: str) -> str: def break_path(self, path: str) -> Sequence[str]: """Breaks path into sequence of identifiers""" + # TODO: this is no longer needed if we modify all naming convention to do not contract + # underscores then also normalize_path will not be needed return [ident for ident in path.split(self.PATH_SEPARATOR) if ident.strip()] def normalize_path(self, path: str) -> str: diff --git a/dlt/common/normalizers/typing.py b/dlt/common/normalizers/typing.py index 9840f3a4d2..16ad097fde 100644 --- a/dlt/common/normalizers/typing.py +++ b/dlt/common/normalizers/typing.py @@ -18,5 +18,7 @@ class TJSONNormalizer(TypedDict, total=False): class TNormalizersConfig(TypedDict, total=False): names: str allow_identifier_change_on_table_with_data: Optional[bool] + use_break_path_on_normalize: Optional[bool] + """Post 1.4.0 to allow table and column names that contain table separators""" detections: Optional[List[str]] json: TJSONNormalizer diff --git a/dlt/common/pipeline.py b/dlt/common/pipeline.py index dba1036f85..9d3d5792ea 100644 --- a/dlt/common/pipeline.py +++ b/dlt/common/pipeline.py @@ -48,7 +48,6 @@ ) from dlt.common.schema import Schema from dlt.common.schema.typing import ( - TColumnNames, TColumnSchema, TWriteDispositionConfig, TSchemaContract, @@ -56,7 +55,7 @@ from dlt.common.storages.load_package import ParsedLoadJobFileName from dlt.common.storages.load_storage import LoadPackageInfo from dlt.common.time import ensure_pendulum_datetime, precise_time -from dlt.common.typing import DictStrAny, REPattern, StrAny, SupportsHumanize +from dlt.common.typing import DictStrAny, REPattern, StrAny, SupportsHumanize, TColumnNames from dlt.common.jsonpath import delete_matches, TAnyJsonPath from dlt.common.data_writers.writers import TLoaderFileFormat from dlt.common.utils import RowCounts, merge_row_counts diff --git a/dlt/common/reflection/utils.py b/dlt/common/reflection/utils.py index cbf38a7327..c612c5a4f1 100644 --- a/dlt/common/reflection/utils.py +++ b/dlt/common/reflection/utils.py @@ -1,7 +1,13 @@ import ast import inspect -import astunparse -from typing import Any, Dict, List, Optional, Sequence, Tuple, Union +from typing import Any, Dict, List, Optional, Sequence, Tuple, Union, Callable + +try: + import astunparse + + ast_unparse: Callable[[ast.AST], str] = astunparse.unparse +except ImportError: + ast_unparse = ast.unparse # type: ignore[attr-defined, unused-ignore] from dlt.common.typing import AnyFun @@ -25,7 +31,7 @@ def get_literal_defaults(node: Union[ast.FunctionDef, ast.AsyncFunctionDef]) -> literal_defaults: Dict[str, str] = {} for arg, default in zip(reversed(args), reversed(defaults)): if default: - literal_defaults[str(arg.arg)] = astunparse.unparse(default).strip() + literal_defaults[str(arg.arg)] = ast_unparse(default).strip() return literal_defaults @@ -99,7 +105,7 @@ def rewrite_python_script( script_lines.append(source_script_lines[last_line][last_offset : node.col_offset]) # replace node value - script_lines.append(astunparse.unparse(t_value).strip()) + script_lines.append(ast_unparse(t_value).strip()) last_line = node.end_lineno - 1 last_offset = node.end_col_offset diff --git a/dlt/common/runtime/collector.py b/dlt/common/runtime/collector.py index be5453cdd3..8504334281 100644 --- a/dlt/common/runtime/collector.py +++ b/dlt/common/runtime/collector.py @@ -37,7 +37,13 @@ class Collector(ABC): @abstractmethod def update( - self, name: str, inc: int = 1, total: int = None, message: str = None, label: str = None + self, + name: str, + inc: int = 1, + total: int = None, + inc_total: int = None, + message: str = None, + label: str = None, ) -> None: """Creates or updates a counter @@ -48,6 +54,7 @@ def update( name (str): An unique name of a counter, displayable. inc (int, optional): Increase amount. Defaults to 1. total (int, optional): Maximum value of a counter. Defaults to None which means unbound counter. + icn_total (int, optional): Increase the maximum value of the counter, does nothing if counter does not exit yet message (str, optional): Additional message attached to a counter. Defaults to None. label (str, optional): Creates nested counter for counter `name`. Defaults to None. """ @@ -80,7 +87,13 @@ class NullCollector(Collector): """A default counter that does not count anything.""" def update( - self, name: str, inc: int = 1, total: int = None, message: str = None, label: str = None + self, + name: str, + inc: int = 1, + total: int = None, + inc_total: int = None, + message: str = None, + label: str = None, ) -> None: pass @@ -98,7 +111,13 @@ def __init__(self) -> None: self.counters: DefaultDict[str, int] = None def update( - self, name: str, inc: int = 1, total: int = None, message: str = None, label: str = None + self, + name: str, + inc: int = 1, + total: int = None, + inc_total: int = None, + message: str = None, + label: str = None, ) -> None: assert not label, "labels not supported in dict collector" self.counters[name] += inc @@ -158,7 +177,13 @@ def __init__( self.last_log_time: float = None def update( - self, name: str, inc: int = 1, total: int = None, message: str = None, label: str = None + self, + name: str, + inc: int = 1, + total: int = None, + inc_total: int = None, + message: str = None, + label: str = None, ) -> None: counter_key = f"{name}_{label}" if label else name @@ -171,6 +196,14 @@ def update( ) self.messages[counter_key] = None self.last_log_time = None + else: + counter_info = self.counter_info[counter_key] + if inc_total: + self.counter_info[counter_key] = LogCollector.CounterInfo( + description=counter_info.description, + start_time=counter_info.start_time, + total=counter_info.total + inc_total, + ) self.counters[counter_key] += inc if message is not None: @@ -264,7 +297,13 @@ def __init__(self, single_bar: bool = False, **tqdm_kwargs: Any) -> None: self.tqdm_kwargs = tqdm_kwargs or {} def update( - self, name: str, inc: int = 1, total: int = None, message: str = None, label: str = "" + self, + name: str, + inc: int = 1, + total: int = None, + inc_total: int = None, + message: str = None, + label: str = "", ) -> None: key = f"{name}_{label}" bar = self._bars.get(key) @@ -281,6 +320,10 @@ def update( bar = tqdm(desc=desc, total=total, leave=False, **self.tqdm_kwargs) bar.refresh() self._bars[key] = bar + else: + if inc_total: + bar.total += inc_total + bar.refresh() if message: bar.set_postfix_str(message) bar.update(inc) @@ -312,11 +355,18 @@ def __init__(self, single_bar: bool = True, **alive_kwargs: Any) -> None: ) self.single_bar = single_bar self._bars: Dict[str, Any] = {} + self._bars_counts: Dict[str, int] = {} self._bars_contexts: Dict[str, ContextManager[Any]] = {} self.alive_kwargs = alive_kwargs or {} def update( - self, name: str, inc: int = 1, total: int = None, message: str = None, label: str = "" + self, + name: str, + inc: int = 1, + total: int = None, + inc_total: int = None, + message: str = None, + label: str = "", ) -> None: key = f"{name}_{label}" bar = self._bars.get(key) @@ -333,19 +383,28 @@ def update( bar = alive_bar(total=total, title=desc, **self.alive_kwargs) self._bars_contexts[key] = bar bar = self._bars[key] = bar.__enter__() + self._bars_counts[key] = 0 + else: + # TODO: implement once total change is supported + pass + # if message: # bar.set_postfix_str(message) - bar(inc) + if inc > 0: + bar(inc) + self._bars_counts[key] += inc def _start(self, step: str) -> None: self._bars = {} self._bars_contexts = {} + self def _stop(self) -> None: for bar in self._bars_contexts.values(): bar.__exit__(None, None, None) self._bars.clear() self._bars_contexts.clear() + self._bars_counts.clear() class EnlightenCollector(Collector): @@ -376,7 +435,13 @@ def __init__(self, single_bar: bool = False, **enlighten_kwargs: Any) -> None: self.enlighten_kwargs = enlighten_kwargs def update( - self, name: str, inc: int = 1, total: int = None, message: str = None, label: str = "" + self, + name: str, + inc: int = 1, + total: int = None, + inc_total: int = None, + message: str = None, + label: str = "", ) -> None: key = f"{name}_{label}" bar = self._bars.get(key) @@ -391,6 +456,9 @@ def update( ) bar.refresh() self._bars[key] = bar + else: + if inc_total: + bar.total = bar.total + inc_total bar.update(inc) def _start(self, step: str) -> None: diff --git a/dlt/common/schema/configuration.py b/dlt/common/schema/configuration.py index e64dd57494..72f79026da 100644 --- a/dlt/common/schema/configuration.py +++ b/dlt/common/schema/configuration.py @@ -14,3 +14,5 @@ class SchemaConfiguration(BaseConfiguration): naming: Optional[TNamingConventionReferenceArg] = None # Union[str, NamingConvention] json_normalizer: Optional[DictStrAny] = None allow_identifier_change_on_table_with_data: Optional[bool] = None + use_break_path_on_normalize: Optional[bool] = None + """Post 1.4.0 to allow table and column names that contain table separators""" diff --git a/dlt/common/schema/migrations.py b/dlt/common/schema/migrations.py index d9e758f204..06eb35c0f6 100644 --- a/dlt/common/schema/migrations.py +++ b/dlt/common/schema/migrations.py @@ -29,13 +29,13 @@ def migrate_schema(schema_dict: DictStrAny, from_engine: int, to_engine: int) -> schema_dict["excludes"] = [] from_engine = 2 if from_engine == 2 and to_engine > 2: - from dlt.common.schema.normalizers import import_normalizers, explicit_normalizers + from dlt.common.schema.normalizers import import_normalizers, configured_normalizers # current version of the schema current = cast(TStoredSchema, schema_dict) # add default normalizers and root hash propagation # use explicit None to get default settings. ignore any naming conventions - normalizers = explicit_normalizers(naming=None, json_normalizer=None) + normalizers = configured_normalizers(naming=None, json_normalizer=None) current["normalizers"], _, _ = import_normalizers(normalizers, normalizers) current["normalizers"]["json"]["config"] = { "propagation": {"root": {"_dlt_id": "_dlt_root_id"}} @@ -169,6 +169,9 @@ def migrate_filters(group: str, filters: List[str]) -> None: json_config.pop("generate_dlt_id", None) from_engine = 10 + if from_engine == 10 and to_engine > 10: + schema_dict["normalizers"]["use_break_path_on_normalize"] = False + from_engine = 11 schema_dict["engine_version"] = from_engine if from_engine != to_engine: diff --git a/dlt/common/schema/normalizers.py b/dlt/common/schema/normalizers.py index 9b2a37e708..8f42e90596 100644 --- a/dlt/common/schema/normalizers.py +++ b/dlt/common/schema/normalizers.py @@ -40,13 +40,14 @@ def _section_for_schema(kwargs: Dict[str, Any]) -> Tuple[str, ...]: @with_config(spec=SchemaConfiguration, sections=_section_for_schema) # type: ignore[call-overload] -def explicit_normalizers( +def configured_normalizers( naming: TNamingConventionReferenceArg = dlt.config.value, json_normalizer: TJSONNormalizer = dlt.config.value, allow_identifier_change_on_table_with_data: bool = None, + use_break_path_on_normalize: Optional[bool] = None, schema_name: Optional[str] = None, ) -> TNormalizersConfig: - """Gets explicitly configured normalizers without any defaults or capabilities injection. If `naming` + """Gets explicitly onfigured normalizers without any defaults or capabilities injection. If `naming` is a module or a type it will get converted into string form via import. If `schema_name` is present, a section ("sources", schema_name, "schema") is used to inject the config @@ -57,6 +58,8 @@ def explicit_normalizers( norm_conf["allow_identifier_change_on_table_with_data"] = ( allow_identifier_change_on_table_with_data ) + if use_break_path_on_normalize is not None: + norm_conf["use_break_path_on_normalize"] = use_break_path_on_normalize return norm_conf diff --git a/dlt/common/schema/schema.py b/dlt/common/schema/schema.py index 0dbeda93cf..d6031a08fa 100644 --- a/dlt/common/schema/schema.py +++ b/dlt/common/schema/schema.py @@ -57,7 +57,7 @@ SchemaCorruptedException, TableIdentifiersFrozen, ) -from dlt.common.schema.normalizers import import_normalizers, explicit_normalizers +from dlt.common.schema.normalizers import import_normalizers, configured_normalizers from dlt.common.schema.exceptions import DataValidationError from dlt.common.validation import validate_dict @@ -439,7 +439,8 @@ def update_schema(self, schema: "Schema") -> None: """Updates this schema from an incoming schema. Normalizes identifiers after updating normalizers.""" # pass normalizer config self._settings = deepcopy(schema.settings) - self._configure_normalizers(schema._normalizers_config) + # make shallow copy of normalizer settings + self._configure_normalizers(copy(schema._normalizers_config)) self._compile_settings() # update all tables for table in schema.tables.values(): @@ -753,7 +754,7 @@ def update_normalizers(self) -> None: Default hints, preferred data types and normalize configs (ie. column propagation) are normalized as well. Regexes are included as long as textual parts can be extracted from an expression. """ - self._configure_normalizers(explicit_normalizers(schema_name=self._schema_name)) + self._configure_normalizers(configured_normalizers(schema_name=self._schema_name)) self._compile_settings() def will_update_normalizers(self) -> bool: @@ -761,7 +762,7 @@ def will_update_normalizers(self) -> bool: # import desired modules _, to_naming, _ = import_normalizers( - explicit_normalizers(schema_name=self._schema_name), self._normalizers_config + configured_normalizers(schema_name=self._schema_name), self._normalizers_config ) return type(to_naming) is not type(self.naming) # noqa @@ -1106,13 +1107,13 @@ def _verify_identifiers(table: TTableSchema, norm_table: TTableSchema) -> None: else: return self._schema_tables - def _renormalize_schema_identifiers( + def _replace_and_apply_naming( self, normalizers_config: TNormalizersConfig, to_naming: NamingConvention, from_naming: NamingConvention, ) -> None: - """Normalizes all identifiers in the schema in place""" + """Normalizes all identifiers in the schema in place according to `to_naming`""" self._schema_tables = self._verify_update_normalizers( normalizers_config, to_naming, from_naming ) @@ -1140,10 +1141,19 @@ def _renormalize_schema_identifiers( def _configure_normalizers(self, explicit_normalizers: TNormalizersConfig) -> None: """Gets naming and item normalizer from schema yaml, config providers and destination capabilities and applies them to schema.""" + # preserve current schema settings if not explicitly set in `explicit_normalizers` + if explicit_normalizers and self._normalizers_config: + for prop_ in [ + "use_break_path_on_normalize", + "allow_identifier_change_on_table_with_data", + ]: + if prop_ in self._normalizers_config and prop_ not in explicit_normalizers: + explicit_normalizers[prop_] = self._normalizers_config[prop_] # type: ignore[literal-required] + normalizers_config, to_naming, item_normalizer_class = import_normalizers( explicit_normalizers, self._normalizers_config ) - self._renormalize_schema_identifiers(normalizers_config, to_naming, self.naming) + self._replace_and_apply_naming(normalizers_config, to_naming, self.naming) # data item normalization function self.data_item_normalizer = item_normalizer_class(self) self.data_item_normalizer.extend_schema() @@ -1174,7 +1184,7 @@ def _reset_schema(self, name: str, normalizers: TNormalizersConfig = None) -> No self._add_standard_hints() # configure normalizers, including custom config if present if not normalizers: - normalizers = explicit_normalizers(schema_name=self._schema_name) + normalizers = configured_normalizers(schema_name=self._schema_name) self._configure_normalizers(normalizers) # add version tables self._add_standard_tables() diff --git a/dlt/common/schema/typing.py b/dlt/common/schema/typing.py index ed6c1c6d78..6f5d6213c9 100644 --- a/dlt/common/schema/typing.py +++ b/dlt/common/schema/typing.py @@ -19,7 +19,7 @@ from dlt.common.data_types import TDataType from dlt.common.normalizers.typing import TNormalizersConfig -from dlt.common.typing import TSortOrder, TAnyDateTime, TLoaderFileFormat +from dlt.common.typing import TSortOrder, TAnyDateTime, TLoaderFileFormat, TColumnNames try: from pydantic import BaseModel as _PydanticBaseModel @@ -28,7 +28,7 @@ # current version of schema engine -SCHEMA_ENGINE_VERSION = 10 +SCHEMA_ENGINE_VERSION = 11 # dlt tables VERSION_TABLE_NAME = "_dlt_version" @@ -132,8 +132,6 @@ class TColumnPropInfo(NamedTuple): "timestamp", "iso_timestamp", "iso_date", "large_integer", "hexbytes_to_text", "wei_to_double" ] TTypeDetectionFunc = Callable[[Type[Any], Any], Optional[TDataType]] -TColumnNames = Union[str, Sequence[str]] -"""A string representing a column name or a list of""" class TColumnType(TypedDict, total=False): @@ -166,6 +164,7 @@ class TColumnSchema(TColumnSchemaBase, total=False): variant: Optional[bool] hard_delete: Optional[bool] dedup_sort: Optional[TSortOrder] + incremental: Optional[bool] TTableSchemaColumns = Dict[str, TColumnSchema] diff --git a/dlt/common/schema/utils.py b/dlt/common/schema/utils.py index f116291e1b..3af024fff5 100644 --- a/dlt/common/schema/utils.py +++ b/dlt/common/schema/utils.py @@ -547,6 +547,17 @@ def merge_diff(table: TTableSchema, table_diff: TPartialTableSchema) -> TPartial * table hints are added or replaced from diff * nothing gets deleted """ + + incremental_a_col = get_first_column_name_with_prop( + table, "incremental", include_incomplete=True + ) + if incremental_a_col: + incremental_b_col = get_first_column_name_with_prop( + table_diff, "incremental", include_incomplete=True + ) + if incremental_b_col: + table["columns"][incremental_a_col].pop("incremental") + # add new columns when all checks passed updated_columns = merge_columns(table["columns"], table_diff["columns"]) table.update(table_diff) diff --git a/dlt/common/storages/configuration.py b/dlt/common/storages/configuration.py index 777b51a488..806e84b73d 100644 --- a/dlt/common/storages/configuration.py +++ b/dlt/common/storages/configuration.py @@ -14,6 +14,7 @@ BaseConfiguration, SFTPCredentials, ) +from dlt.common.exceptions import TerminalValueError from dlt.common.typing import DictStrAny from dlt.common.utils import digest128 @@ -57,6 +58,43 @@ class LoadStorageConfiguration(BaseConfiguration): ] +def ensure_canonical_az_url( + bucket_url: str, target_scheme: str, storage_account_name: str = None, account_host: str = None +) -> str: + """Converts any of the forms of azure blob storage into canonical form of {target_scheme}://@.{account_host}/path + + `azure_storage_account_name` is optional only if not present in bucket_url, `account_host` assumes ".dfs.core.windows.net" by default + """ + parsed_bucket_url = urlparse(bucket_url) + # Converts an az:/// to abfss://@.dfs.core.windows.net/ + if parsed_bucket_url.username: + # has the right form, ensure abfss schema + return urlunparse(parsed_bucket_url._replace(scheme=target_scheme)) + + if not storage_account_name and not account_host: + raise TerminalValueError( + f"Could not convert azure blob storage url {bucket_url} into canonical form " + f" ({target_scheme}://@.dfs.core.windows.net/)" + f" because storage account name is not known. Please use {target_scheme}:// canonical" + " url as bucket_url in filesystem credentials" + ) + + account_host = account_host or f"{storage_account_name}.dfs.core.windows.net" + netloc = ( + f"{parsed_bucket_url.netloc}@{account_host}" if parsed_bucket_url.netloc else account_host + ) + + # as required by databricks + _path = parsed_bucket_url.path + return urlunparse( + parsed_bucket_url._replace( + scheme=target_scheme, + netloc=netloc, + path=_path, + ) + ) + + def _make_sftp_url(scheme: str, fs_path: str, bucket_url: str) -> str: parsed_bucket_url = urlparse(bucket_url) return f"{scheme}://{parsed_bucket_url.hostname}{fs_path}" diff --git a/dlt/common/storages/fsspec_filesystem.py b/dlt/common/storages/fsspec_filesystem.py index 6ac5f31007..fb929031a1 100644 --- a/dlt/common/storages/fsspec_filesystem.py +++ b/dlt/common/storages/fsspec_filesystem.py @@ -40,6 +40,7 @@ ) from dlt.common.time import ensure_pendulum_datetime from dlt.common.typing import DictStrAny +from dlt.common.utils import without_none class FileItem(TypedDict, total=False): @@ -97,6 +98,10 @@ class FileItem(TypedDict, total=False): DEFAULT_KWARGS["azure"] = DEFAULT_KWARGS["az"] DEFAULT_KWARGS["abfss"] = DEFAULT_KWARGS["az"] +AZURE_BLOB_STORAGE_PROTOCOLS = ["az", "azure", "adl", "abfss", "abfs"] +S3_PROTOCOLS = ["s3", "s3a"] +GCS_PROTOCOLS = ["gs", "gcs"] + def fsspec_filesystem( protocol: str, @@ -130,7 +135,11 @@ def prepare_fsspec_args(config: FilesystemConfiguration) -> DictStrAny: """ protocol = config.protocol # never use listing caches - fs_kwargs: DictStrAny = {"use_listings_cache": False, "listings_expiry_time": 60.0} + fs_kwargs: DictStrAny = { + "use_listings_cache": False, + "listings_expiry_time": 60.0, + "skip_instance_cache": True, + } credentials = CREDENTIALS_DISPATCH.get(protocol, lambda _: {})(config) if protocol == "gdrive": @@ -151,7 +160,7 @@ def prepare_fsspec_args(config: FilesystemConfiguration) -> DictStrAny: if "client_kwargs" in fs_kwargs and "client_kwargs" in credentials: fs_kwargs["client_kwargs"].update(credentials.pop("client_kwargs")) - fs_kwargs.update(credentials) + fs_kwargs.update(without_none(credentials)) return fs_kwargs @@ -174,8 +183,13 @@ def fsspec_from_config(config: FilesystemConfiguration) -> Tuple[AbstractFileSys # first get the class to check the protocol fs_cls = get_filesystem_class(config.protocol) if fs_cls.protocol == "abfs": + url = urlparse(config.bucket_url) # if storage account is present in bucket_url and in credentials, az fsspec will fail - if urlparse(config.bucket_url).username: + # account name is detected only for blob.core.windows.net host + if url.username and ( + url.hostname.endswith("blob.core.windows.net") + or url.hostname.endswith("dfs.core.windows.net") + ): fs_kwargs.pop("account_name") return url_to_fs(config.bucket_url, **fs_kwargs) # type: ignore except ImportError as e: diff --git a/dlt/common/typing.py b/dlt/common/typing.py index 94edb57194..a3364d1b07 100644 --- a/dlt/common/typing.py +++ b/dlt/common/typing.py @@ -29,6 +29,7 @@ Iterator, Generator, NamedTuple, + Sequence, ) from typing_extensions import ( @@ -112,6 +113,8 @@ class SecretSentinel: TSecretStrValue = Annotated[str, SecretSentinel] +TColumnNames = Union[str, Sequence[str]] +"""A string representing a column name or a list of""" TDataItem: TypeAlias = Any """A single data item as extracted from data source""" TDataItems: TypeAlias = Union[TDataItem, List[TDataItem]] @@ -126,6 +129,10 @@ class SecretSentinel: TLoaderFileFormat = Literal["jsonl", "typed-jsonl", "insert_values", "parquet", "csv", "reference"] """known loader file formats""" +TDynHintType = TypeVar("TDynHintType") +TFunHintTemplate = Callable[[TDataItem], TDynHintType] +TTableHintTemplate = Union[TDynHintType, TFunHintTemplate[TDynHintType]] + class ConfigValueSentinel(NamedTuple): """Class to create singleton sentinel for config and secret injected value""" diff --git a/dlt/common/utils.py b/dlt/common/utils.py index 170baf6f9f..58e1dbd824 100644 --- a/dlt/common/utils.py +++ b/dlt/common/utils.py @@ -635,3 +635,20 @@ def __getattribute__(self, name: str) -> Any: raise RuntimeError("This instance has been dropped and cannot be used anymore.") return DefunctClass + + +def is_typeerror_due_to_wrong_call(exc: Exception, func: AnyFun) -> bool: + """ + Determine if a TypeError is due to a wrong call to the function (incorrect arguments) + by inspecting the exception message. + """ + if not isinstance(exc, TypeError): + return False + func_name = func.__name__ + message = str(exc) + return message.__contains__(f"{func_name}()") + + +removeprefix = getattr( + str, "removeprefix", lambda s_, p_: s_[len(p_) :] if s_.startswith(p_) else s_ +) diff --git a/dlt/destinations/dataset.py b/dlt/destinations/dataset.py index cffdc0f059..27a7f5a7af 100644 --- a/dlt/destinations/dataset.py +++ b/dlt/destinations/dataset.py @@ -1,10 +1,11 @@ -from typing import Any, Generator, Optional, Sequence, Union, List -from dlt.common.json import json -from copy import deepcopy - -from dlt.common.normalizers.naming.naming import NamingConvention +from typing import Any, Generator, Sequence, Union, TYPE_CHECKING, Tuple from contextlib import contextmanager + +from dlt import version +from dlt.common.json import json +from dlt.common.exceptions import MissingDependencyException +from dlt.common.destination import AnyDestination from dlt.common.destination.reference import ( SupportsReadableRelation, SupportsReadableDataset, @@ -14,6 +15,9 @@ JobClientBase, WithStateSync, DestinationClientDwhConfiguration, + DestinationClientStagingConfiguration, + DestinationClientConfiguration, + DestinationClientDwhWithStagingConfiguration, ) from dlt.common.schema.typing import TTableSchemaColumns @@ -21,6 +25,14 @@ from dlt.common.schema import Schema from dlt.common.exceptions import DltException +if TYPE_CHECKING: + try: + from dlt.common.libs.ibis import BaseBackend as IbisBackend + except MissingDependencyException: + IbisBackend = Any +else: + IbisBackend = Any + class DatasetException(DltException): pass @@ -93,7 +105,7 @@ def query(self) -> Any: return self._provided_query table_name = self.sql_client.make_qualified_table_name( - self.schema.naming.normalize_path(self._table_name) + self.schema.naming.normalize_tables_path(self._table_name) ) maybe_limit_clause_1 = "" @@ -228,6 +240,16 @@ def __init__( self._sql_client: SqlClientBase[Any] = None self._schema: Schema = None + def ibis(self) -> IbisBackend: + """return a connected ibis backend""" + from dlt.common.libs.ibis import create_ibis_backend + + self._ensure_client_and_schema() + return create_ibis_backend( + self._destination, + self._destination_client(self.schema), + ) + @property def schema(self) -> Schema: self._ensure_client_and_schema() @@ -239,15 +261,13 @@ def sql_client(self) -> SqlClientBase[Any]: return self._sql_client def _destination_client(self, schema: Schema) -> JobClientBase: - client_spec = self._destination.spec() - if isinstance(client_spec, DestinationClientDwhConfiguration): - client_spec._bind_dataset_name( - dataset_name=self._dataset_name, default_schema_name=schema.name - ) - return self._destination.client(schema, client_spec) + return get_destination_clients( + schema, destination=self._destination, destination_dataset_name=self._dataset_name + )[0] def _ensure_client_and_schema(self) -> None: """Lazy load schema and client""" + # full schema given, nothing to do if not self._schema and isinstance(self._provided_schema, Schema): self._schema = self._provided_schema @@ -259,6 +279,8 @@ def _ensure_client_and_schema(self) -> None: stored_schema = client.get_stored_schema(self._provided_schema) if stored_schema: self._schema = Schema.from_stored_schema(json.loads(stored_schema.schema)) + else: + self._schema = Schema(self._provided_schema) # no schema name given, load newest schema from destination elif not self._schema: @@ -268,7 +290,7 @@ def _ensure_client_and_schema(self) -> None: if stored_schema: self._schema = Schema.from_stored_schema(json.loads(stored_schema.schema)) - # default to empty schema with dataset name if nothing found + # default to empty schema with dataset name if not self._schema: self._schema = Schema(self._dataset_name) @@ -310,3 +332,81 @@ def dataset( if dataset_type == "dbapi": return ReadableDBAPIDataset(destination, dataset_name, schema) raise NotImplementedError(f"Dataset of type {dataset_type} not implemented") + + +# helpers +def get_destination_client_initial_config( + destination: AnyDestination, + default_schema_name: str, + dataset_name: str, + as_staging: bool = False, +) -> DestinationClientConfiguration: + client_spec = destination.spec + + # this client supports many schemas and datasets + if issubclass(client_spec, DestinationClientDwhConfiguration): + if issubclass(client_spec, DestinationClientStagingConfiguration): + spec: DestinationClientDwhConfiguration = client_spec(as_staging_destination=as_staging) + else: + spec = client_spec() + + spec._bind_dataset_name(dataset_name, default_schema_name) + return spec + + return client_spec() + + +def get_destination_clients( + schema: Schema, + destination: AnyDestination = None, + destination_dataset_name: str = None, + destination_initial_config: DestinationClientConfiguration = None, + staging: AnyDestination = None, + staging_dataset_name: str = None, + staging_initial_config: DestinationClientConfiguration = None, + # pipeline specific settings + default_schema_name: str = None, +) -> Tuple[JobClientBase, JobClientBase]: + destination = Destination.from_reference(destination) if destination else None + staging = Destination.from_reference(staging) if staging else None + + try: + # resolve staging config in order to pass it to destination client config + staging_client = None + if staging: + if not staging_initial_config: + # this is just initial config - without user configuration injected + staging_initial_config = get_destination_client_initial_config( + staging, + dataset_name=staging_dataset_name, + default_schema_name=default_schema_name, + as_staging=True, + ) + # create the client - that will also resolve the config + staging_client = staging.client(schema, staging_initial_config) + + if not destination_initial_config: + # config is not provided then get it with injected credentials + initial_config = get_destination_client_initial_config( + destination, + dataset_name=destination_dataset_name, + default_schema_name=default_schema_name, + ) + + # attach the staging client config to destination client config - if its type supports it + if ( + staging_client + and isinstance(initial_config, DestinationClientDwhWithStagingConfiguration) + and isinstance(staging_client.config, DestinationClientStagingConfiguration) + ): + initial_config.staging_config = staging_client.config + # create instance with initial_config properly set + client = destination.client(schema, initial_config) + return client, staging_client + except ModuleNotFoundError: + client_spec = destination.spec() + raise MissingDependencyException( + f"{client_spec.destination_type} destination", + [f"{version.DLT_PKG_NAME}[{client_spec.destination_type}]"], + "Dependencies for specific destinations are available as extras of dlt", + ) diff --git a/dlt/destinations/impl/bigquery/bigquery_adapter.py b/dlt/destinations/impl/bigquery/bigquery_adapter.py index 5f6a1fab85..05b26530d9 100644 --- a/dlt/destinations/impl/bigquery/bigquery_adapter.py +++ b/dlt/destinations/impl/bigquery/bigquery_adapter.py @@ -4,10 +4,8 @@ from dlt.common.destination import PreparedTableSchema from dlt.common.pendulum import timezone -from dlt.common.schema.typing import ( - TColumnNames, - TTableSchemaColumns, -) +from dlt.common.schema.typing import TTableSchemaColumns +from dlt.common.typing import TColumnNames from dlt.destinations.utils import get_resource_for_adapter from dlt.extract import DltResource from dlt.extract.items import TTableHintTemplate diff --git a/dlt/destinations/impl/clickhouse/clickhouse.py b/dlt/destinations/impl/clickhouse/clickhouse.py index dba0a8667d..3a5f5c3e28 100644 --- a/dlt/destinations/impl/clickhouse/clickhouse.py +++ b/dlt/destinations/impl/clickhouse/clickhouse.py @@ -29,7 +29,8 @@ ) from dlt.common.schema.utils import is_nullable_column from dlt.common.storages import FileStorage -from dlt.common.storages.configuration import FilesystemConfiguration +from dlt.common.storages.configuration import FilesystemConfiguration, ensure_canonical_az_url +from dlt.common.storages.fsspec_filesystem import AZURE_BLOB_STORAGE_PROTOCOLS from dlt.destinations.exceptions import LoadJobTerminalException from dlt.destinations.impl.clickhouse.configuration import ( ClickHouseClientConfiguration, @@ -140,7 +141,7 @@ def run(self) -> None: f"s3('{bucket_http_url}',{auth},'{clickhouse_format}','auto','{compression}')" ) - elif bucket_scheme in ("az", "abfs"): + elif bucket_scheme in AZURE_BLOB_STORAGE_PROTOCOLS: if not isinstance(self._staging_credentials, AzureCredentialsWithoutDefaults): raise LoadJobTerminalException( self._file_path, @@ -149,7 +150,11 @@ def run(self) -> None: # Authenticated access. account_name = self._staging_credentials.azure_storage_account_name - storage_account_url = f"https://{self._staging_credentials.azure_storage_account_name}.blob.core.windows.net" + account_host = ( + self._staging_credentials.azure_account_host + or f"{account_name}.blob.core.windows.net" + ) + storage_account_url = ensure_canonical_az_url("", "https", account_name, account_host) account_key = self._staging_credentials.azure_storage_account_key # build table func diff --git a/dlt/destinations/impl/clickhouse/sql_client.py b/dlt/destinations/impl/clickhouse/sql_client.py index 00f35da082..a6c4ee0458 100644 --- a/dlt/destinations/impl/clickhouse/sql_client.py +++ b/dlt/destinations/impl/clickhouse/sql_client.py @@ -28,6 +28,7 @@ from dlt.common import logger from dlt.common.destination import DestinationCapabilitiesContext from dlt.common.typing import DictStrAny +from dlt.common.utils import removeprefix from dlt.destinations.exceptions import ( DatabaseUndefinedRelation, @@ -88,9 +89,8 @@ def has_dataset(self) -> bool: sentinel_table = self.config.dataset_sentinel_table_name all_ds_tables = self._list_tables() if self.dataset_name: - return sentinel_table in [ - t.split(self.config.dataset_table_separator)[1] for t in all_ds_tables - ] + prefix = self.dataset_name + self.config.dataset_table_separator + return sentinel_table in [removeprefix(t, prefix) for t in all_ds_tables] else: # if no dataset specified we look for sentinel table return sentinel_table in all_ds_tables diff --git a/dlt/destinations/impl/databricks/databricks.py b/dlt/destinations/impl/databricks/databricks.py index 718427af87..2bb68a607e 100644 --- a/dlt/destinations/impl/databricks/databricks.py +++ b/dlt/destinations/impl/databricks/databricks.py @@ -18,12 +18,17 @@ AzureCredentialsWithoutDefaults, ) from dlt.common.exceptions import TerminalValueError +from dlt.common.storages.configuration import ensure_canonical_az_url from dlt.common.storages.file_storage import FileStorage +from dlt.common.storages.fsspec_filesystem import ( + AZURE_BLOB_STORAGE_PROTOCOLS, + S3_PROTOCOLS, + GCS_PROTOCOLS, +) from dlt.common.schema import TColumnSchema, Schema from dlt.common.schema.typing import TColumnType from dlt.common.storages import FilesystemConfiguration, fsspec_from_config - from dlt.destinations.insert_job_client import InsertValuesJobClient from dlt.destinations.exceptions import LoadJobTerminalException from dlt.destinations.impl.databricks.configuration import DatabricksClientConfiguration @@ -32,8 +37,8 @@ from dlt.destinations.job_impl import ReferenceFollowupJobRequest from dlt.destinations.utils import is_compression_disabled -AZURE_BLOB_STORAGE_PROTOCOLS = ["az", "abfss", "abfs"] -SUPPORTED_BLOB_STORAGE_PROTOCOLS = AZURE_BLOB_STORAGE_PROTOCOLS + ["s3", "gs", "gcs"] + +SUPPORTED_BLOB_STORAGE_PROTOCOLS = AZURE_BLOB_STORAGE_PROTOCOLS + S3_PROTOCOLS + GCS_PROTOCOLS class DatabricksLoadJob(RunnableLoadJob, HasFollowupJobs): @@ -106,7 +111,9 @@ def run(self) -> None: # Explicit azure credentials are needed to load from bucket without a named stage credentials_clause = f"""WITH(CREDENTIAL(AZURE_SAS_TOKEN='{staging_credentials.azure_storage_sas_token}'))""" bucket_path = self.ensure_databricks_abfss_url( - bucket_path, staging_credentials.azure_storage_account_name + bucket_path, + staging_credentials.azure_storage_account_name, + staging_credentials.azure_account_host, ) else: raise LoadJobTerminalException( @@ -124,7 +131,9 @@ def run(self) -> None: ), ) bucket_path = self.ensure_databricks_abfss_url( - bucket_path, staging_credentials.azure_storage_account_name + bucket_path, + staging_credentials.azure_storage_account_name, + staging_credentials.azure_account_host, ) # always add FROM clause @@ -165,30 +174,10 @@ def run(self) -> None: @staticmethod def ensure_databricks_abfss_url( - bucket_path: str, azure_storage_account_name: str = None + bucket_path: str, azure_storage_account_name: str = None, account_host: str = None ) -> str: - bucket_url = urlparse(bucket_path) - # Converts an az:/// to abfss://@.dfs.core.windows.net/ - if bucket_url.username: - # has the right form, ensure abfss schema - return urlunparse(bucket_url._replace(scheme="abfss")) - - if not azure_storage_account_name: - raise TerminalValueError( - f"Could not convert azure blob storage url {bucket_path} into form required by" - " Databricks" - " (abfss://@.dfs.core.windows.net/)" - " because storage account name is not known. Please use Databricks abfss://" - " canonical url as bucket_url in staging credentials" - ) - # as required by databricks - _path = bucket_url.path - return urlunparse( - bucket_url._replace( - scheme="abfss", - netloc=f"{bucket_url.netloc}@{azure_storage_account_name}.dfs.core.windows.net", - path=_path, - ) + return ensure_canonical_az_url( + bucket_path, "abfss", azure_storage_account_name, account_host ) diff --git a/dlt/destinations/impl/duckdb/sql_client.py b/dlt/destinations/impl/duckdb/sql_client.py index 89a522c8f7..ee73965df6 100644 --- a/dlt/destinations/impl/duckdb/sql_client.py +++ b/dlt/destinations/impl/duckdb/sql_client.py @@ -52,10 +52,8 @@ def iter_arrow(self, chunk_size: int) -> Generator[ArrowTable, None, None]: yield self.native_cursor.fetch_arrow_table() return # iterate - try: - yield from self.native_cursor.fetch_record_batch(chunk_size) - except StopIteration: - pass + for item in self.native_cursor.fetch_record_batch(chunk_size): + yield ArrowTable.from_batches([item]) class DuckDbSqlClient(SqlClientBase[duckdb.DuckDBPyConnection], DBTransaction): diff --git a/dlt/destinations/impl/filesystem/filesystem.py b/dlt/destinations/impl/filesystem/filesystem.py index 0cf63b3ac9..1739c87fb3 100644 --- a/dlt/destinations/impl/filesystem/filesystem.py +++ b/dlt/destinations/impl/filesystem/filesystem.py @@ -37,7 +37,7 @@ TPipelineStateDoc, load_package as current_load_package, ) -from dlt.destinations.sql_client import DBApiCursor, WithSqlClient, SqlClientBase +from dlt.destinations.sql_client import WithSqlClient, SqlClientBase from dlt.common.destination import DestinationCapabilitiesContext from dlt.common.destination.reference import ( FollowupJobRequest, @@ -63,7 +63,6 @@ from dlt.destinations.impl.filesystem.configuration import FilesystemDestinationClientConfiguration from dlt.destinations import path_utils from dlt.destinations.fs_client import FSClientBase -from dlt.destinations.dataset import ReadableDBAPIDataset from dlt.destinations.utils import verify_schema_merge_disposition INIT_FILE_NAME = "init" diff --git a/dlt/destinations/impl/filesystem/sql_client.py b/dlt/destinations/impl/filesystem/sql_client.py index fec761ff36..d03a00b418 100644 --- a/dlt/destinations/impl/filesystem/sql_client.py +++ b/dlt/destinations/impl/filesystem/sql_client.py @@ -178,11 +178,6 @@ def create_authentication(self, persistent: bool = False, secret_name: str = Non elif self.fs_client.config.protocol == "memory": self._conn.register_filesystem(self.fs_client.fs_client) - # the line below solves problems with certificate path lookup on linux - # see duckdb docs - if self.fs_client.config.protocol in ["az", "abfss"]: - self._conn.sql("SET azure_transport_option_type = 'curl';") - def open_connection(self) -> duckdb.DuckDBPyConnection: # we keep the in memory instance around, so if this prop is set, return it first_connection = self.credentials.has_open_connection @@ -195,8 +190,16 @@ def open_connection(self) -> duckdb.DuckDBPyConnection: self._conn.sql(f"USE {self.fully_qualified_dataset_name()}") self.create_authentication() + # the line below solves problems with certificate path lookup on linux + # see duckdb docs + if self.fs_client.config.protocol in ["az", "abfss"]: + self._conn.sql("SET azure_transport_option_type = 'curl';") + return self._conn + def create_views_for_all_tables(self) -> None: + self.create_views_for_tables({v: v for v in self.fs_client.schema.tables.keys()}) + @raise_database_error def create_views_for_tables(self, tables: Dict[str, str]) -> None: """Add the required tables as views to the duckdb in memory instance""" diff --git a/dlt/destinations/impl/lancedb/configuration.py b/dlt/destinations/impl/lancedb/configuration.py index 8f6a192bb0..33642268c1 100644 --- a/dlt/destinations/impl/lancedb/configuration.py +++ b/dlt/destinations/impl/lancedb/configuration.py @@ -82,6 +82,8 @@ class LanceDBClientConfiguration(DestinationClientDwhConfiguration): """Embedding provider used for generating embeddings. Default is "cohere". You can find the full list of providers at https://github.com/lancedb/lancedb/tree/main/python/python/lancedb/embeddings as well as https://lancedb.github.io/lancedb/embeddings/default_embedding_functions/.""" + embedding_model_provider_host: Optional[str] = None + """Full host URL with protocol and port (e.g. 'http://localhost:11434'). Uses LanceDB's default if not specified, assuming the provider accepts this parameter.""" embedding_model: str = "embed-english-v3.0" """The model used by the embedding provider for generating embeddings. Check with the embedding provider which options are available. diff --git a/dlt/destinations/impl/lancedb/lancedb_adapter.py b/dlt/destinations/impl/lancedb/lancedb_adapter.py index 4314dd703f..d192168d0a 100644 --- a/dlt/destinations/impl/lancedb/lancedb_adapter.py +++ b/dlt/destinations/impl/lancedb/lancedb_adapter.py @@ -1,6 +1,7 @@ from typing import Any, Dict -from dlt.common.schema.typing import TColumnNames, TTableSchemaColumns +from dlt.common.schema.typing import TTableSchemaColumns +from dlt.common.typing import TColumnNames from dlt.destinations.utils import get_resource_for_adapter from dlt.extract import DltResource from dlt.extract.items import TTableHintTemplate diff --git a/dlt/destinations/impl/lancedb/lancedb_client.py b/dlt/destinations/impl/lancedb/lancedb_client.py index 1a3e1a7d34..bb0e12f8ec 100644 --- a/dlt/destinations/impl/lancedb/lancedb_client.py +++ b/dlt/destinations/impl/lancedb/lancedb_client.py @@ -251,6 +251,7 @@ def __init__( self.dataset_name = self.config.normalize_dataset_name(self.schema) embedding_model_provider = self.config.embedding_model_provider + embedding_model_host = self.config.embedding_model_provider_host # LanceDB doesn't provide a standardized way to set API keys across providers. # Some use ENV variables and others allow passing api key as an argument. @@ -259,12 +260,13 @@ def __init__( embedding_model_provider, self.config.credentials.embedding_model_provider_api_key, ) + self.model_func = self.registry.get(embedding_model_provider).create( name=self.config.embedding_model, max_retries=self.config.options.max_retries, api_key=self.config.credentials.api_key, + **({"host": embedding_model_host} if embedding_model_host else {}), ) - self.vector_field_name = self.config.vector_field_name @property diff --git a/dlt/destinations/impl/postgres/factory.py b/dlt/destinations/impl/postgres/factory.py index bde0e35f3d..e0dc2836eb 100644 --- a/dlt/destinations/impl/postgres/factory.py +++ b/dlt/destinations/impl/postgres/factory.py @@ -1,19 +1,19 @@ import typing as t +from dlt.common.arithmetics import DEFAULT_NUMERIC_PRECISION, DEFAULT_NUMERIC_SCALE from dlt.common.data_writers.configuration import CsvFormatConfiguration -from dlt.common.destination import Destination, DestinationCapabilitiesContext from dlt.common.data_writers.escape import escape_postgres_identifier, escape_postgres_literal -from dlt.common.arithmetics import DEFAULT_NUMERIC_PRECISION, DEFAULT_NUMERIC_SCALE +from dlt.common.destination import Destination, DestinationCapabilitiesContext from dlt.common.destination.typing import PreparedTableSchema from dlt.common.exceptions import TerminalValueError from dlt.common.schema.typing import TColumnSchema, TColumnType from dlt.common.wei import EVM_DECIMAL_PRECISION - -from dlt.destinations.type_mapping import TypeMapperImpl from dlt.destinations.impl.postgres.configuration import ( PostgresCredentials, PostgresClientConfiguration, ) +from dlt.destinations.impl.postgres.postgres_adapter import GEOMETRY_HINT, SRID_HINT +from dlt.destinations.type_mapping import TypeMapperImpl if t.TYPE_CHECKING: from dlt.destinations.impl.postgres.postgres import PostgresClient @@ -55,6 +55,7 @@ class PostgresTypeMapper(TypeMapperImpl): "character varying": "text", "smallint": "bigint", "integer": "bigint", + "geometry": "text", } def to_db_integer_type(self, column: TColumnSchema, table: PreparedTableSchema = None) -> str: @@ -108,11 +109,18 @@ def to_db_datetime_type( def from_destination_type( self, db_type: str, precision: t.Optional[int] = None, scale: t.Optional[int] = None ) -> TColumnType: - if db_type == "numeric": - if (precision, scale) == self.capabilities.wei_precision: - return dict(data_type="wei") + if db_type == "numeric" and (precision, scale) == self.capabilities.wei_precision: + return dict(data_type="wei") + if db_type.startswith("geometry"): + return dict(data_type="text") return super().from_destination_type(db_type, precision, scale) + def to_destination_type(self, column: TColumnSchema, table: PreparedTableSchema) -> str: + if column.get(GEOMETRY_HINT): + srid = column.get(SRID_HINT, 4326) + return f"geometry(Geometry, {srid})" + return super().to_destination_type(column, table) + class postgres(Destination[PostgresClientConfiguration, "PostgresClient"]): spec = PostgresClientConfiguration diff --git a/dlt/destinations/impl/postgres/postgres.py b/dlt/destinations/impl/postgres/postgres.py index 682f70da04..2459ee1dbe 100644 --- a/dlt/destinations/impl/postgres/postgres.py +++ b/dlt/destinations/impl/postgres/postgres.py @@ -2,9 +2,9 @@ from dlt.common import logger from dlt.common.data_writers.configuration import CsvFormatConfiguration +from dlt.common.destination import DestinationCapabilitiesContext from dlt.common.destination.exceptions import ( DestinationInvalidFileFormat, - DestinationTerminalException, ) from dlt.common.destination.reference import ( HasFollowupJobs, @@ -12,20 +12,16 @@ RunnableLoadJob, FollowupJobRequest, LoadJob, - TLoadJobState, ) -from dlt.common.destination import DestinationCapabilitiesContext -from dlt.common.exceptions import TerminalValueError from dlt.common.schema import TColumnSchema, TColumnHint, Schema -from dlt.common.schema.typing import TColumnType, TTableFormat +from dlt.common.schema.typing import TColumnType from dlt.common.schema.utils import is_nullable_column from dlt.common.storages.file_storage import FileStorage - -from dlt.destinations.sql_jobs import SqlStagingCopyFollowupJob, SqlJobParams -from dlt.destinations.insert_job_client import InsertValuesJobClient -from dlt.destinations.impl.postgres.sql_client import Psycopg2SqlClient from dlt.destinations.impl.postgres.configuration import PostgresClientConfiguration +from dlt.destinations.impl.postgres.sql_client import Psycopg2SqlClient +from dlt.destinations.insert_job_client import InsertValuesJobClient from dlt.destinations.sql_client import SqlClientBase +from dlt.destinations.sql_jobs import SqlStagingCopyFollowupJob, SqlJobParams HINT_TO_POSTGRES_ATTR: Dict[TColumnHint, str] = {"unique": "UNIQUE"} @@ -43,15 +39,16 @@ def generate_sql( with sql_client.with_staging_dataset(): staging_table_name = sql_client.make_qualified_table_name(table["name"]) table_name = sql_client.make_qualified_table_name(table["name"]) - # drop destination table - sql.append(f"DROP TABLE IF EXISTS {table_name};") - # moving staging table to destination schema - sql.append( - f"ALTER TABLE {staging_table_name} SET SCHEMA" - f" {sql_client.fully_qualified_dataset_name()};" + sql.extend( + ( + f"DROP TABLE IF EXISTS {table_name};", + ( + f"ALTER TABLE {staging_table_name} SET SCHEMA" + f" {sql_client.fully_qualified_dataset_name()};" + ), + f"CREATE TABLE {staging_table_name} (like {table_name} including all);", + ) ) - # recreate staging table - sql.append(f"CREATE TABLE {staging_table_name} (like {table_name} including all);") return sql @@ -111,8 +108,7 @@ def run(self) -> None: split_columns.append(norm_col) if norm_col in split_headers and is_nullable_column(col): split_null_headers.append(norm_col) - split_unknown_headers = set(split_headers).difference(split_columns) - if split_unknown_headers: + if split_unknown_headers := set(split_headers).difference(split_columns): raise DestinationInvalidFileFormat( "postgres", "csv", @@ -130,15 +126,8 @@ def run(self) -> None: qualified_table_name = sql_client.make_qualified_table_name(table_name) copy_sql = ( - "COPY %s (%s) FROM STDIN WITH (FORMAT CSV, DELIMITER '%s', NULL ''," - " %s ENCODING '%s')" - % ( - qualified_table_name, - headers, - sep, - null_headers, - csv_format.encoding, - ) + f"COPY {qualified_table_name} ({headers}) FROM STDIN WITH (FORMAT CSV, DELIMITER" + f" '{sep}', NULL '', {null_headers} ENCODING '{csv_format.encoding}')" ) with sql_client.begin_transaction(): with sql_client.native_connection.cursor() as cursor: @@ -173,15 +162,16 @@ def create_load_job( return job def _get_column_def_sql(self, c: TColumnSchema, table: PreparedTableSchema = None) -> str: - hints_str = " ".join( + hints_ = " ".join( self.active_hints.get(h, "") for h in self.active_hints.keys() if c.get(h, False) is True ) column_name = self.sql_client.escape_column_name(c["name"]) - return ( - f"{column_name} {self.type_mapper.to_destination_type(c,table)} {hints_str} {self._gen_not_null(c.get('nullable', True))}" - ) + nullability = self._gen_not_null(c.get("nullable", True)) + column_type = self.type_mapper.to_destination_type(c, table) + + return f"{column_name} {column_type} {hints_} {nullability}" def _create_replace_followup_jobs( self, table_chain: Sequence[PreparedTableSchema] diff --git a/dlt/destinations/impl/postgres/postgres_adapter.py b/dlt/destinations/impl/postgres/postgres_adapter.py new file mode 100644 index 0000000000..11e86ec525 --- /dev/null +++ b/dlt/destinations/impl/postgres/postgres_adapter.py @@ -0,0 +1,63 @@ +from typing import Any, Optional + +from dlt.common.schema.typing import TColumnNames, TTableSchemaColumns +from dlt.destinations.utils import get_resource_for_adapter +from dlt.extract import DltResource + +GEOMETRY_HINT = "x-postgres-geometry" +SRID_HINT = "x-postgres-srid" + + +def postgres_adapter( + data: Any, + geometry: TColumnNames = None, + srid: Optional[int] = 4326, +) -> DltResource: + """Prepares data for the postgres destination by specifying which columns should + be cast to PostGIS geometry types. + + Args: + data (Any): The data to be transformed. It can be raw data or an instance + of DltResource. If raw data, the function wraps it into a DltResource + object. + geometry (TColumnNames, optional): Specify columns to cast to geometries. + It can be a single column name as a string, or a list of column names. + srid (int, optional): The Spatial Reference System Identifier (SRID) to be + used for the geometry columns. If not provided, SRID 4326 will be used. + + Returns: + DltResource: A resource with applied postgres-specific hints. + + Raises: + ValueError: If input for `geometry` is invalid, or if no geometry columns are specified. + + Examples: + >>> data = [{"town": "Null Island", "loc": "POINT(0 0)"}] + >>> postgres_adapter(data, geometry="loc", srid=4326) + [DltResource with hints applied] + """ + resource = get_resource_for_adapter(data) + + column_hints: TTableSchemaColumns = {} + + if geometry: + if isinstance(geometry, str): + geometry = [geometry] + if not isinstance(geometry, list): + raise ValueError( + "'geometry' must be a list of column names or a single column name as a string." + ) + + for column_name in geometry: + column_hints[column_name] = { + "name": column_name, + GEOMETRY_HINT: True, # type: ignore[misc] + } + if srid is not None: + column_hints[column_name][SRID_HINT] = srid # type: ignore + + if not column_hints: + raise ValueError("A value for 'geometry' must be specified.") + else: + resource.apply_hints(columns=column_hints) + return resource diff --git a/dlt/destinations/impl/qdrant/qdrant_adapter.py b/dlt/destinations/impl/qdrant/qdrant_adapter.py index bbc2d719a8..5a5a44965c 100644 --- a/dlt/destinations/impl/qdrant/qdrant_adapter.py +++ b/dlt/destinations/impl/qdrant/qdrant_adapter.py @@ -1,6 +1,7 @@ from typing import Any -from dlt.common.schema.typing import TColumnNames, TTableSchemaColumns +from dlt.common.schema.typing import TTableSchemaColumns +from dlt.common.typing import TColumnNames from dlt.extract import DltResource from dlt.destinations.utils import get_resource_for_adapter diff --git a/dlt/destinations/impl/snowflake/snowflake.py b/dlt/destinations/impl/snowflake/snowflake.py index 41a8384754..e5146139f2 100644 --- a/dlt/destinations/impl/snowflake/snowflake.py +++ b/dlt/destinations/impl/snowflake/snowflake.py @@ -15,19 +15,18 @@ AwsCredentialsWithoutDefaults, AzureCredentialsWithoutDefaults, ) -from dlt.common.storages.configuration import FilesystemConfiguration +from dlt.common.storages.configuration import FilesystemConfiguration, ensure_canonical_az_url from dlt.common.storages.file_storage import FileStorage from dlt.common.schema import TColumnSchema, Schema from dlt.common.schema.typing import TColumnType -from dlt.common.exceptions import TerminalValueError +from dlt.common.storages.fsspec_filesystem import AZURE_BLOB_STORAGE_PROTOCOLS, S3_PROTOCOLS from dlt.common.typing import TLoaderFileFormat from dlt.destinations.job_client_impl import SqlJobClientWithStagingDataset from dlt.destinations.exceptions import LoadJobTerminalException from dlt.destinations.impl.snowflake.configuration import SnowflakeClientConfiguration from dlt.destinations.impl.snowflake.sql_client import SnowflakeSqlClient -from dlt.destinations.impl.snowflake.sql_client import SnowflakeSqlClient from dlt.destinations.job_impl import ReferenceFollowupJobRequest @@ -124,33 +123,27 @@ def gen_copy_sql( if not is_local: bucket_scheme = parsed_file_url.scheme # referencing an external s3/azure stage does not require explicit AWS credentials - if bucket_scheme in ["s3", "az", "abfs"] and stage_name: + if bucket_scheme in AZURE_BLOB_STORAGE_PROTOCOLS + S3_PROTOCOLS and stage_name: from_clause = f"FROM '@{stage_name}'" files_clause = f"FILES = ('{parsed_file_url.path.lstrip('/')}')" # referencing an staged files via a bucket URL requires explicit AWS credentials elif ( - bucket_scheme == "s3" + bucket_scheme in S3_PROTOCOLS and staging_credentials and isinstance(staging_credentials, AwsCredentialsWithoutDefaults) ): credentials_clause = f"""CREDENTIALS=(AWS_KEY_ID='{staging_credentials.aws_access_key_id}' AWS_SECRET_KEY='{staging_credentials.aws_secret_access_key}')""" from_clause = f"FROM '{file_url}'" elif ( - bucket_scheme in ["az", "abfs"] + bucket_scheme in AZURE_BLOB_STORAGE_PROTOCOLS and staging_credentials and isinstance(staging_credentials, AzureCredentialsWithoutDefaults) ): - # Explicit azure credentials are needed to load from bucket without a named stage credentials_clause = f"CREDENTIALS=(AZURE_SAS_TOKEN='?{staging_credentials.azure_storage_sas_token}')" - # Converts an az:/// to azure://.blob.core.windows.net// - # as required by snowflake - _path = "/" + parsed_file_url.netloc + parsed_file_url.path - file_url = urlunparse( - parsed_file_url._replace( - scheme="azure", - netloc=f"{staging_credentials.azure_storage_account_name}.blob.core.windows.net", - path=_path, - ) + file_url = cls.ensure_snowflake_azure_url( + file_url, + staging_credentials.azure_storage_account_name, + staging_credentials.azure_account_host, ) from_clause = f"FROM '{file_url}'" else: @@ -204,6 +197,28 @@ def gen_copy_sql( {on_error_clause} """ + @staticmethod + def ensure_snowflake_azure_url( + file_url: str, account_name: str = None, account_host: str = None + ) -> str: + # Explicit azure credentials are needed to load from bucket without a named stage + if not account_host and account_name: + account_host = f"{account_name}.blob.core.windows.net" + # get canonical url first to convert it into snowflake form + canonical_url = ensure_canonical_az_url( + file_url, + "azure", + account_name, + account_host, + ) + parsed_file_url = urlparse(canonical_url) + return urlunparse( + parsed_file_url._replace( + path=f"/{parsed_file_url.username}{parsed_file_url.path}", + netloc=parsed_file_url.hostname, + ) + ) + class SnowflakeClient(SqlJobClientWithStagingDataset, SupportsStagingDestination): def __init__( diff --git a/dlt/destinations/impl/snowflake/sql_client.py b/dlt/destinations/impl/snowflake/sql_client.py index e52c5424d3..22e27ea48b 100644 --- a/dlt/destinations/impl/snowflake/sql_client.py +++ b/dlt/destinations/impl/snowflake/sql_client.py @@ -1,5 +1,5 @@ from contextlib import contextmanager, suppress -from typing import Any, AnyStr, ClassVar, Dict, Iterator, Optional, Sequence, List +from typing import Any, AnyStr, ClassVar, Iterator, Optional, Sequence import snowflake.connector as snowflake_lib diff --git a/dlt/destinations/impl/weaviate/weaviate_adapter.py b/dlt/destinations/impl/weaviate/weaviate_adapter.py index 0ca9047528..329d13c493 100644 --- a/dlt/destinations/impl/weaviate/weaviate_adapter.py +++ b/dlt/destinations/impl/weaviate/weaviate_adapter.py @@ -1,6 +1,7 @@ from typing import Dict, Any, Literal, Set, get_args -from dlt.common.schema.typing import TColumnNames, TTableSchemaColumns +from dlt.common.schema.typing import TTableSchemaColumns +from dlt.common.typing import TColumnNames from dlt.extract import DltResource, resource as make_resource from dlt.destinations.utils import get_resource_for_adapter diff --git a/dlt/destinations/job_client_impl.py b/dlt/destinations/job_client_impl.py index e02cdbaa09..d1f211b1e9 100644 --- a/dlt/destinations/job_client_impl.py +++ b/dlt/destinations/job_client_impl.py @@ -597,7 +597,10 @@ def _check_table_update_hints( for c in new_columns if c.get(hint, False) ] - if hint == "null": + if hint in ["hard_delete", "dedup_sort", "merge_key"]: + # you may add those + pass + elif hint == "null": logger.warning( f"Column(s) {hint_columns} with NOT NULL are being added to existing" f" table {table_name}. If there's data in the table the operation" diff --git a/dlt/destinations/sql_client.py b/dlt/destinations/sql_client.py index afd4f82f6c..7d1728b43d 100644 --- a/dlt/destinations/sql_client.py +++ b/dlt/destinations/sql_client.py @@ -399,6 +399,7 @@ def iter_arrow(self, chunk_size: int) -> Generator[ArrowTable, None, None]: or DestinationCapabilitiesContext.generic_capabilities() ) + # note that columns_schema is inferred from the cursor and is not normalized if not chunk_size: result = self.fetchall() yield row_tuples_to_arrow(result, caps, self.columns_schema, tz="UTC") diff --git a/dlt/extract/decorators.py b/dlt/extract/decorators.py index 63140e8f78..f8703e1452 100644 --- a/dlt/extract/decorators.py +++ b/dlt/extract/decorators.py @@ -32,7 +32,6 @@ from dlt.common.schema.utils import DEFAULT_WRITE_DISPOSITION from dlt.common.schema.schema import Schema from dlt.common.schema.typing import ( - TColumnNames, TFileFormat, TWriteDisposition, TWriteDispositionConfig, @@ -43,7 +42,8 @@ ) from dlt.common.storages.exceptions import SchemaNotFoundError from dlt.common.storages.schema_storage import SchemaStorage -from dlt.common.typing import AnyFun, ParamSpec, Concatenate, TDataItem, TDataItems +from dlt.common.typing import AnyFun, ParamSpec, Concatenate, TDataItem, TDataItems, TColumnNames + from dlt.common.utils import get_callable_name, get_module_name, is_inner_callable from dlt.extract.hints import make_hints @@ -70,6 +70,7 @@ TSourceFunParams, ) from dlt.extract.resource import DltResource, TUnboundDltResource, TDltResourceImpl +from dlt.extract.incremental import TIncrementalConfig @configspec @@ -446,6 +447,7 @@ def resource( selected: bool = True, spec: Type[BaseConfiguration] = None, parallelized: bool = False, + incremental: Optional[TIncrementalConfig] = None, _impl_cls: Type[TDltResourceImpl] = DltResource, # type: ignore[assignment] ) -> TDltResourceImpl: ... @@ -468,6 +470,7 @@ def resource( selected: bool = True, spec: Type[BaseConfiguration] = None, parallelized: bool = False, + incremental: Optional[TIncrementalConfig] = None, _impl_cls: Type[TDltResourceImpl] = DltResource, # type: ignore[assignment] ) -> Callable[[Callable[TResourceFunParams, Any]], TDltResourceImpl]: ... @@ -490,6 +493,7 @@ def resource( selected: bool = True, spec: Type[BaseConfiguration] = None, parallelized: bool = False, + incremental: Optional[TIncrementalConfig] = None, _impl_cls: Type[TDltResourceImpl] = DltResource, # type: ignore[assignment] standalone: Literal[True] = True, ) -> Callable[ @@ -515,6 +519,7 @@ def resource( selected: bool = True, spec: Type[BaseConfiguration] = None, parallelized: bool = False, + incremental: Optional[TIncrementalConfig] = None, _impl_cls: Type[TDltResourceImpl] = DltResource, # type: ignore[assignment] ) -> TDltResourceImpl: ... @@ -536,6 +541,7 @@ def resource( selected: bool = True, spec: Type[BaseConfiguration] = None, parallelized: bool = False, + incremental: Optional[TIncrementalConfig] = None, _impl_cls: Type[TDltResourceImpl] = DltResource, # type: ignore[assignment] standalone: bool = False, data_from: TUnboundDltResource = None, @@ -632,6 +638,7 @@ def make_resource(_name: str, _section: str, _data: Any) -> TDltResourceImpl: table_format=table_format, file_format=file_format, references=references, + incremental=incremental, ) resource = _impl_cls.from_data( @@ -643,6 +650,10 @@ def make_resource(_name: str, _section: str, _data: Any) -> TDltResourceImpl: cast(DltResource, data_from), True, ) + + if incremental: + # Reset the flag to allow overriding by incremental argument + resource.incremental._from_hints = False # If custom nesting level was specified then # we need to add it to table hints so that # later in normalizer dlt/common/normalizers/json/relational.py @@ -681,7 +692,7 @@ def _wrap(*args: Any, **kwargs: Any) -> TDltResourceImpl: return _wrap def decorator( - f: Callable[TResourceFunParams, Any] + f: Callable[TResourceFunParams, Any], ) -> Callable[TResourceFunParams, TDltResourceImpl]: if not callable(f): if data_from: @@ -1023,7 +1034,7 @@ def get_source() -> DltSource: def defer( - f: Callable[TDeferredFunParams, TBoundItems] + f: Callable[TDeferredFunParams, TBoundItems], ) -> Callable[TDeferredFunParams, TDeferred[TBoundItems]]: @wraps(f) def _wrap(*args: Any, **kwargs: Any) -> TDeferred[TBoundItems]: diff --git a/dlt/extract/extract.py b/dlt/extract/extract.py index e65f6cf0d0..25c3a0dbae 100644 --- a/dlt/extract/extract.py +++ b/dlt/extract/extract.py @@ -2,7 +2,7 @@ from collections.abc import Sequence as C_Sequence from copy import copy import itertools -from typing import Iterator, List, Dict, Any, Optional +from typing import Iterator, List, Dict, Any, Optional, Mapping import yaml from dlt.common.configuration.container import Container @@ -17,13 +17,12 @@ WithStepInfo, reset_resource_state, ) -from dlt.common.typing import DictStrAny +from dlt.common.typing import DictStrAny, TColumnNames from dlt.common.runtime import signals from dlt.common.runtime.collector import Collector, NULL_COLLECTOR from dlt.common.schema import Schema, utils from dlt.common.schema.typing import ( TAnySchemaColumns, - TColumnNames, TSchemaContract, TTableFormat, TWriteDispositionConfig, @@ -39,7 +38,7 @@ from dlt.extract.decorators import SourceInjectableContext, SourceSchemaInjectableContext from dlt.extract.exceptions import DataItemRequiredForDynamicTableHints -from dlt.extract.incremental import IncrementalResourceWrapper +from dlt.extract.incremental import IncrementalResourceWrapper, Incremental from dlt.extract.pipe_iterator import PipeIterator from dlt.extract.source import DltSource from dlt.extract.resource import DltResource diff --git a/dlt/extract/extractors.py b/dlt/extract/extractors.py index 41d3035a9f..03f8a31462 100644 --- a/dlt/extract/extractors.py +++ b/dlt/extract/extractors.py @@ -18,6 +18,8 @@ TTableSchemaColumns, TPartialTableSchema, ) +from dlt.common.normalizers.json import helpers as normalize_helpers + from dlt.extract.hints import HintsMeta, TResourceHints from dlt.extract.resource import DltResource from dlt.extract.items import DataItemWithMeta, TableNameMeta @@ -141,7 +143,9 @@ def write_items(self, resource: DltResource, items: TDataItems, meta: Any) -> No self._write_to_dynamic_table(resource, items, meta) def write_empty_items_file(self, table_name: str) -> None: - table_name = self.naming.normalize_table_identifier(table_name) + table_name = normalize_helpers.normalize_table_identifier( + self.schema, self.naming, table_name + ) self.item_storage.write_empty_items_file(self.load_id, self.schema.name, table_name, None) def _get_static_table_name(self, resource: DltResource, meta: Any) -> Optional[str]: @@ -151,10 +155,12 @@ def _get_static_table_name(self, resource: DltResource, meta: Any) -> Optional[s table_name = meta.table_name else: table_name = resource.table_name # type: ignore[assignment] - return self.naming.normalize_table_identifier(table_name) + return normalize_helpers.normalize_table_identifier(self.schema, self.naming, table_name) def _get_dynamic_table_name(self, resource: DltResource, item: TDataItem) -> str: - return self.naming.normalize_table_identifier(resource._table_name_hint_fun(item)) + return normalize_helpers.normalize_table_identifier( + self.schema, self.naming, resource._table_name_hint_fun(item) + ) def _write_item( self, @@ -322,7 +328,7 @@ def write_items(self, resource: DltResource, items: TDataItems, meta: Any) -> No ) for tbl in ( ( - # 1. Convert pandas frame(s) to arrow Table + # 1. Convert pandas frame(s) to arrow Table, remove indexes because we store pandas_to_arrow(item) if (pandas and isinstance(item, pandas.DataFrame)) else item diff --git a/dlt/extract/hints.py b/dlt/extract/hints.py index 5daabd0c6a..000e5c4cdb 100644 --- a/dlt/extract/hints.py +++ b/dlt/extract/hints.py @@ -1,10 +1,9 @@ -from typing import TypedDict, cast, Any, Optional, Dict, Sequence, Mapping +from typing import TypedDict, cast, Any, Optional, Dict, Sequence, Mapping, Union from typing_extensions import Self from dlt.common import logger from dlt.common.schema.typing import ( C_DLT_ID, - TColumnNames, TColumnProp, TFileFormat, TPartialTableSchema, @@ -28,7 +27,7 @@ new_column, new_table, ) -from dlt.common.typing import TDataItem +from dlt.common.typing import TDataItem, TColumnNames from dlt.common.time import ensure_pendulum_datetime from dlt.common.utils import clone_dict_nested from dlt.common.normalizers.json.relational import DataItemNormalizer @@ -37,7 +36,7 @@ DataItemRequiredForDynamicTableHints, InconsistentTableTemplate, ) -from dlt.extract.incremental import Incremental +from dlt.extract.incremental import Incremental, TIncrementalConfig from dlt.extract.items import TFunHintTemplate, TTableHintTemplate, TableNameMeta, ValidateItem from dlt.extract.utils import ensure_table_schema_columns, ensure_table_schema_columns_hint from dlt.extract.validation import create_item_validator @@ -86,6 +85,7 @@ def make_hints( table_format: TTableHintTemplate[TTableFormat] = None, file_format: TTableHintTemplate[TFileFormat] = None, references: TTableHintTemplate[TTableReferenceParam] = None, + incremental: TIncrementalConfig = None, ) -> TResourceHints: """A convenience function to create resource hints. Accepts both static and dynamic hints based on data. @@ -119,6 +119,8 @@ def make_hints( if validator: new_template["validator"] = validator DltResourceHints.validate_dynamic_hints(new_template) + if incremental is not None: # TODO: Validate + new_template["incremental"] = Incremental.ensure_instance(incremental) return new_template @@ -204,6 +206,10 @@ def compute_table_schema(self, item: TDataItem = None, meta: Any = None) -> TTab for k, v in table_template.items() if k not in NATURAL_CALLABLES } # type: ignore + if "incremental" in table_template: + incremental = table_template["incremental"] + if isinstance(incremental, Incremental) and incremental is not Incremental.EMPTY: + resolved_template["incremental"] = incremental table_schema = self._create_table_schema(resolved_template, self.name) migrate_complex_types(table_schema, warn=True) validate_dict_ignoring_xkeys( @@ -221,7 +227,7 @@ def apply_hints( columns: TTableHintTemplate[TAnySchemaColumns] = None, primary_key: TTableHintTemplate[TColumnNames] = None, merge_key: TTableHintTemplate[TColumnNames] = None, - incremental: Incremental[Any] = None, + incremental: TIncrementalConfig = None, schema_contract: TTableHintTemplate[TSchemaContract] = None, additional_table_hints: Optional[Dict[str, TTableHintTemplate[Any]]] = None, table_format: TTableHintTemplate[TTableFormat] = None, @@ -360,7 +366,7 @@ def apply_hints( # set properties that can't be passed to make_hints if incremental is not None: - t["incremental"] = incremental + t["incremental"] = Incremental.ensure_instance(incremental) self._set_hints(t, create_table_variant) return self @@ -506,6 +512,22 @@ def _merge_merge_disposition_dict(dict_: Dict[str, Any]) -> None: "row_key": False, } + @staticmethod + def _merge_incremental_column_hint(dict_: Dict[str, Any]) -> None: + incremental = dict_.pop("incremental") + if incremental is None: + return + col_name = incremental.get_cursor_column_name() + if not col_name: + # cursor cannot resolve to a single column, no hint added + return + incremental_col = dict_["columns"].get(col_name) + if not incremental_col: + incremental_col = {"name": col_name} + + incremental_col["incremental"] = True + dict_["columns"][col_name] = incremental_col + @staticmethod def _create_table_schema(resource_hints: TResourceHints, resource_name: str) -> TTableSchema: """Creates table schema from resource hints and resource name. Resource hints are resolved @@ -518,6 +540,8 @@ def _create_table_schema(resource_hints: TResourceHints, resource_name: str) -> "disposition": resource_hints["write_disposition"] } # wrap in dict DltResourceHints._merge_write_disposition_dict(resource_hints) # type: ignore[arg-type] + if "incremental" in resource_hints: + DltResourceHints._merge_incremental_column_hint(resource_hints) # type: ignore[arg-type] dict_ = cast(TTableSchema, resource_hints) dict_["resource"] = resource_name return dict_ diff --git a/dlt/extract/incremental/__init__.py b/dlt/extract/incremental/__init__.py index 69af0d68a6..28d33bb71f 100644 --- a/dlt/extract/incremental/__init__.py +++ b/dlt/extract/incremental/__init__.py @@ -1,6 +1,6 @@ import os from datetime import datetime # noqa: I251 -from typing import Generic, ClassVar, Any, Optional, Type, Dict, Union +from typing import Generic, ClassVar, Any, Optional, Type, Dict, Union, Literal, Tuple from typing_extensions import get_args import inspect @@ -9,7 +9,7 @@ from dlt.common import logger from dlt.common.exceptions import MissingDependencyException from dlt.common.pendulum import pendulum -from dlt.common.jsonpath import compile_path +from dlt.common.jsonpath import compile_path, extract_simple_field_name from dlt.common.typing import ( TDataItem, TDataItems, @@ -19,8 +19,8 @@ get_generic_type_argument_from_instance, is_optional_type, is_subclass, + TColumnNames, ) -from dlt.common.schema.typing import TColumnNames from dlt.common.configuration import configspec, ConfigurationValueError from dlt.common.configuration.specs import BaseConfiguration from dlt.common.pipeline import resource_state @@ -29,17 +29,19 @@ coerce_value, py_type_to_sc_type, ) +from dlt.common.utils import without_none from dlt.extract.exceptions import IncrementalUnboundError from dlt.extract.incremental.exceptions import ( IncrementalCursorPathMissing, IncrementalPrimaryKeyMissing, ) -from dlt.extract.incremental.typing import ( +from dlt.common.incremental.typing import ( IncrementalColumnState, TCursorValue, LastValueFunc, OnCursorValueMissing, + IncrementalArgs, ) from dlt.extract.items import SupportsPipe, TTableHintTemplate, ItemTransform from dlt.extract.incremental.transform import ( @@ -123,7 +125,7 @@ def __init__( self, cursor_path: str = None, initial_value: Optional[TCursorValue] = None, - last_value_func: Optional[LastValueFunc[TCursorValue]] = max, + last_value_func: Optional[Union[LastValueFunc[TCursorValue], Literal["min", "max"]]] = max, primary_key: Optional[TTableHintTemplate[TColumnNames]] = None, end_value: Optional[TCursorValue] = None, row_order: Optional[TSortOrder] = None, @@ -135,6 +137,16 @@ def __init__( if cursor_path: compile_path(cursor_path) self.cursor_path = cursor_path + if isinstance(last_value_func, str): + if last_value_func == "min": + last_value_func = min + elif last_value_func == "max": + last_value_func = max + else: + raise ValueError( + f"Unknown last_value_func '{last_value_func}' passed as string. Provide a" + " callable to use a custom function." + ) self.last_value_func = last_value_func self.initial_value = initial_value """Initial value of last_value""" @@ -247,6 +259,10 @@ def copy(self) -> "Incremental[TCursorValue]": # merge creates a copy return self.merge(self) + def get_cursor_column_name(self) -> Optional[str]: + """Return the name of the cursor column if the cursor path resolves to a single column""" + return extract_simple_field_name(self.cursor_path) + def on_resolved(self) -> None: compile_path(self.cursor_path) if self.end_value is not None and self.initial_value is None: @@ -491,6 +507,12 @@ def can_close(self) -> bool: and self.start_out_of_range ) + @classmethod + def ensure_instance(cls, value: "TIncrementalConfig") -> "Incremental[TCursorValue]": + if isinstance(value, Incremental): + return value + return cls(**value) + def __str__(self) -> str: return ( f"Incremental at 0x{id(self):x} for resource {self.resource_name} with cursor path:" @@ -511,7 +533,6 @@ def _get_transformer(self, items: TDataItems) -> IncrementalTransform: def __call__(self, rows: TDataItems, meta: Any = None) -> Optional[TDataItems]: if rows is None: return rows - transformer = self._get_transformer(rows) if isinstance(rows, list): rows = [ @@ -556,6 +577,8 @@ def _check_duplicate_cursor_threshold( Incremental.EMPTY = Incremental[Any]() Incremental.EMPTY.__is_resolved__ = True +TIncrementalConfig = Union[Incremental[Any], IncrementalArgs] + class IncrementalResourceWrapper(ItemTransform[TDataItem]): placement_affinity: ClassVar[float] = 1 # stick to end @@ -595,6 +618,34 @@ def get_incremental_arg(sig: inspect.Signature) -> Optional[inspect.Parameter]: break return incremental_param + @staticmethod + def inject_implicit_incremental_arg( + incremental: Optional[Union[Incremental[Any], "IncrementalResourceWrapper"]], + sig: inspect.Signature, + func_args: Tuple[Any], + func_kwargs: Dict[str, Any], + fallback: Optional[Incremental[Any]] = None, + ) -> Tuple[Tuple[Any], Dict[str, Any], Optional[Incremental[Any]]]: + """Inject the incremental instance into function arguments + if the function has an incremental argument without default in its signature and it is not already set in the arguments. + + Returns: + Tuple of the new args, kwargs and the incremental instance that was injected (if any) + """ + if isinstance(incremental, IncrementalResourceWrapper): + incremental = incremental.incremental + if not incremental: + if not fallback: + return func_args, func_kwargs, None + incremental = fallback + incremental_param = IncrementalResourceWrapper.get_incremental_arg(sig) + if incremental_param: + bound_args = sig.bind_partial(*func_args, **func_kwargs) + if not bound_args.arguments.get(incremental_param.name): + bound_args.arguments[incremental_param.name] = incremental + return bound_args.args, bound_args.kwargs, incremental + return func_args, func_kwargs, None + def wrap(self, sig: inspect.Signature, func: TFun) -> TFun: """Wrap the callable to inject an `Incremental` object configured for the resource.""" incremental_param = self.get_incremental_arg(sig) @@ -666,12 +717,14 @@ def incremental(self) -> Optional[Incremental[Any]]: return self._incremental def set_incremental( - self, incremental: Optional[Incremental[Any]], from_hints: bool = False + self, incremental: Optional[TIncrementalConfig], from_hints: bool = False ) -> None: """Sets the incremental. If incremental was set from_hints, it can only be changed in the same manner""" if self._from_hints and not from_hints: # do not accept incremental if apply hints were used return + if incremental is not None: + incremental = Incremental.ensure_instance(incremental) self._from_hints = from_hints self._incremental = incremental @@ -710,6 +763,12 @@ def __call__(self, item: TDataItems, meta: Any = None) -> Optional[TDataItems]: return self._incremental(item, meta) +def incremental_config_to_instance(cfg: TIncrementalConfig) -> Incremental[Any]: + if isinstance(cfg, Incremental): + return cfg + return Incremental(**cfg) + + __all__ = [ "Incremental", "IncrementalResourceWrapper", @@ -717,6 +776,7 @@ def __call__(self, item: TDataItems, meta: Any = None) -> Optional[TDataItems]: "IncrementalCursorPathMissing", "IncrementalPrimaryKeyMissing", "IncrementalUnboundError", + "TIncrementalconfig", "LastValueFunc", "TCursorValue", ] diff --git a/dlt/extract/incremental/transform.py b/dlt/extract/incremental/transform.py index fe15571e41..22b1194b51 100644 --- a/dlt/extract/incremental/transform.py +++ b/dlt/extract/incremental/transform.py @@ -5,18 +5,17 @@ from dlt.common.utils import digest128 from dlt.common.json import json from dlt.common.pendulum import pendulum -from dlt.common.typing import TDataItem -from dlt.common.jsonpath import find_values, JSONPathFields, compile_path +from dlt.common.typing import TDataItem, TColumnNames +from dlt.common.jsonpath import find_values, compile_path, extract_simple_field_name from dlt.extract.incremental.exceptions import ( IncrementalCursorInvalidCoercion, IncrementalCursorPathMissing, IncrementalPrimaryKeyMissing, IncrementalCursorPathHasValueNone, ) -from dlt.extract.incremental.typing import TCursorValue, LastValueFunc, OnCursorValueMissing +from dlt.common.incremental.typing import TCursorValue, LastValueFunc, OnCursorValueMissing from dlt.extract.utils import resolve_column_value from dlt.extract.items import TTableHintTemplate -from dlt.common.schema.typing import TColumnNames try: from dlt.common.libs import pyarrow @@ -75,12 +74,8 @@ def __init__( # compile jsonpath self._compiled_cursor_path = compile_path(cursor_path) # for simple column name we'll fallback to search in dict - if ( - isinstance(self._compiled_cursor_path, JSONPathFields) - and len(self._compiled_cursor_path.fields) == 1 - and self._compiled_cursor_path.fields[0] != "*" - ): - self.cursor_path = self._compiled_cursor_path.fields[0] + if simple_field_name := extract_simple_field_name(self._compiled_cursor_path): + self.cursor_path = simple_field_name self._compiled_cursor_path = None def compute_unique_value( diff --git a/dlt/extract/items.py b/dlt/extract/items.py index d721e8094e..888787e6b7 100644 --- a/dlt/extract/items.py +++ b/dlt/extract/items.py @@ -19,7 +19,14 @@ ) from concurrent.futures import Future -from dlt.common.typing import TAny, TDataItem, TDataItems +from dlt.common.typing import ( + TAny, + TDataItem, + TDataItems, + TTableHintTemplate, + TFunHintTemplate, + TDynHintType, +) TDecompositionStrategy = Literal["none", "scc"] @@ -27,9 +34,6 @@ TAwaitableDataItems = Awaitable[TDataItems] TPipedDataItems = Union[TDataItems, TDeferredDataItems, TAwaitableDataItems] -TDynHintType = TypeVar("TDynHintType") -TFunHintTemplate = Callable[[TDataItem], TDynHintType] -TTableHintTemplate = Union[TDynHintType, TFunHintTemplate[TDynHintType]] if TYPE_CHECKING: TItemFuture = Future[TPipedDataItems] diff --git a/dlt/extract/resource.py b/dlt/extract/resource.py index c6ca1660f4..42e3905162 100644 --- a/dlt/extract/resource.py +++ b/dlt/extract/resource.py @@ -11,6 +11,7 @@ Union, Any, Optional, + Mapping, ) from typing_extensions import TypeVar, Self @@ -28,6 +29,7 @@ pipeline_state, ) from dlt.common.utils import flatten_list_or_items, get_callable_name, uniq_id +from dlt.common.schema.typing import TTableSchema from dlt.extract.utils import wrap_async_iterator, wrap_parallel_iterator from dlt.extract.items import ( @@ -42,7 +44,7 @@ ) from dlt.extract.pipe_iterator import ManagedPipeIterator from dlt.extract.pipe import Pipe, TPipeStep -from dlt.extract.hints import DltResourceHints, HintsMeta, TResourceHints +from dlt.extract.hints import DltResourceHints, HintsMeta, TResourceHints, make_hints from dlt.extract.incremental import Incremental, IncrementalResourceWrapper from dlt.extract.exceptions import ( InvalidTransformerDataTypeGeneratorFunctionRequired, @@ -442,35 +444,60 @@ def add_step( self._pipe.insert_step(item_transform, insert_at) return self + def _remove_incremental_step(self) -> None: + step_no = self._pipe.find(Incremental, IncrementalResourceWrapper) + if step_no >= 0: + self._pipe.remove_step(step_no) + + def set_incremental( + self, + new_incremental: Union[Incremental[Any], IncrementalResourceWrapper], + from_hints: bool = False, + ) -> Optional[Union[Incremental[Any], IncrementalResourceWrapper]]: + """Set/replace the incremental transform for the resource. + + Args: + new_incremental: The Incremental instance/hint to set or replace + from_hints: If the incremental is set from hints. Defaults to False. + """ + if new_incremental is Incremental.EMPTY: + new_incremental = None + incremental = self.incremental + if incremental is not None: + # if isinstance(new_incremental, Mapping): + # new_incremental = Incremental.ensure_instance(new_incremental) + + if isinstance(new_incremental, IncrementalResourceWrapper): + # Completely replace the wrapper + self._remove_incremental_step() + self.add_step(new_incremental) + elif isinstance(incremental, IncrementalResourceWrapper): + incremental.set_incremental(new_incremental, from_hints=from_hints) + else: + self._remove_incremental_step() + # re-add the step + incremental = None + if incremental is None: + # if there's no wrapper add incremental as a transform + if new_incremental: + if not isinstance(new_incremental, IncrementalResourceWrapper): + new_incremental = Incremental.ensure_instance(new_incremental) + self.add_step(new_incremental) + return new_incremental + def _set_hints( self, table_schema_template: TResourceHints, create_table_variant: bool = False ) -> None: super()._set_hints(table_schema_template, create_table_variant) # validators and incremental apply only to resource hints if not create_table_variant: - incremental = self.incremental # try to late assign incremental if table_schema_template.get("incremental") is not None: - new_incremental = table_schema_template["incremental"] - # remove incremental if empty - if new_incremental is Incremental.EMPTY: - new_incremental = None - - if incremental is not None: - if isinstance(incremental, IncrementalResourceWrapper): - # replace in wrapper - incremental.set_incremental(new_incremental, from_hints=True) - else: - step_no = self._pipe.find(Incremental) - self._pipe.remove_step(step_no) - # re-add the step - incremental = None - - if incremental is None: - # if there's no wrapper add incremental as a transform - incremental = new_incremental # type: ignore - if new_incremental: - self.add_step(new_incremental) + incremental = self.set_incremental( + table_schema_template["incremental"], from_hints=True + ) + else: + incremental = self.incremental if incremental: primary_key = table_schema_template.get("primary_key", incremental.primary_key) @@ -480,10 +507,25 @@ def _set_hints( if table_schema_template.get("validator") is not None: self.validator = table_schema_template["validator"] + def compute_table_schema(self, item: TDataItem = None, meta: Any = None) -> TTableSchema: + incremental: Optional[Union[Incremental[Any], IncrementalResourceWrapper]] = ( + self.incremental + ) + if incremental and "incremental" not in self._hints: + if isinstance(incremental, IncrementalResourceWrapper): + incremental = incremental.incremental + if incremental: + self._hints["incremental"] = incremental + + table_schema = super().compute_table_schema(item, meta) + + return table_schema + def bind(self: TDltResourceImpl, *args: Any, **kwargs: Any) -> TDltResourceImpl: """Binds the parametrized resource to passed arguments. Modifies resource pipe in place. Does not evaluate generators or iterators.""" if self._args_bound: raise TypeError(f"Parametrized resource {self.name} is not callable") + orig_gen = self._pipe.gen gen = self._pipe.bind_gen(*args, **kwargs) if isinstance(gen, DltResource): @@ -599,14 +641,14 @@ def _eject_config(self) -> bool: if not self._pipe.is_empty and not self._args_bound: orig_gen = getattr(self._pipe.gen, "__GEN__", None) if orig_gen: - step_no = self._pipe.find(IncrementalResourceWrapper) - if step_no >= 0: - self._pipe.remove_step(step_no) + self._remove_incremental_step() self._pipe.replace_gen(orig_gen) return True return False - def _inject_config(self) -> "DltResource": + def _inject_config( + self, incremental_from_hints_override: Optional[bool] = None + ) -> "DltResource": """Wraps the pipe generation step in incremental and config injection wrappers and adds pipe step with Incremental transform. """ @@ -618,8 +660,17 @@ def _inject_config(self) -> "DltResource": sig = inspect.signature(gen) if IncrementalResourceWrapper.should_wrap(sig): incremental = IncrementalResourceWrapper(self._hints.get("primary_key")) + if incr_hint := self._hints.get("incremental"): + incremental.set_incremental( + incr_hint, + from_hints=( + incremental_from_hints_override + if incremental_from_hints_override is not None + else True + ), + ) incr_f = incremental.wrap(sig, gen) - self.add_step(incremental) + self.set_incremental(incremental) else: incr_f = gen resource_sections = (known_sections.SOURCES, self.section, self.name) @@ -649,6 +700,12 @@ def _clone( if self._pipe and not self._pipe.is_empty: pipe = pipe._clone(new_name=new_name, with_parent=with_parent) # incremental and parent are already in the pipe (if any) + + incremental = self.incremental + if isinstance(incremental, IncrementalResourceWrapper): + incremental_from_hints: Optional[bool] = incremental._from_hints + else: + incremental_from_hints = None r_ = self.__class__( pipe, self._clone_hints(self._hints), @@ -661,7 +718,7 @@ def _clone( # this makes sure that a take config values from a right section and wrapper has a separated # instance in the pipeline if r_._eject_config(): - r_._inject_config() + r_._inject_config(incremental_from_hints_override=incremental_from_hints) return r_ def _get_config_section_context(self) -> ConfigSectionContext: diff --git a/dlt/extract/utils.py b/dlt/extract/utils.py index 55a8b0b8c4..68570d0995 100644 --- a/dlt/extract/utils.py +++ b/dlt/extract/utils.py @@ -22,8 +22,15 @@ from dlt.common.data_writers import TDataItemFormat from dlt.common.exceptions import MissingDependencyException from dlt.common.pipeline import reset_resource_state -from dlt.common.schema.typing import TColumnNames, TAnySchemaColumns, TTableSchemaColumns -from dlt.common.typing import AnyFun, DictStrAny, TDataItem, TDataItems, TAnyFunOrGenerator +from dlt.common.schema.typing import TAnySchemaColumns, TTableSchemaColumns +from dlt.common.typing import ( + AnyFun, + DictStrAny, + TDataItem, + TDataItems, + TAnyFunOrGenerator, + TColumnNames, +) from dlt.common.utils import get_callable_name from dlt.extract.exceptions import ( diff --git a/dlt/load/load.py b/dlt/load/load.py index 060b2c5d8e..ddbc7193ed 100644 --- a/dlt/load/load.py +++ b/dlt/load/load.py @@ -370,6 +370,7 @@ def create_followup_jobs( f"Job {starting_job.job_id()} CREATED a new FOLLOWUP JOB" f" {followup_job.new_file_path()} placed in new_jobs" ) + self.collector.update("Jobs", inc=0, inc_total=len(jobs)) def complete_jobs( self, load_id: str, jobs: Sequence[LoadJob], schema: Schema diff --git a/dlt/normalize/worker.py b/dlt/normalize/worker.py index 53a856f7d0..5eccdf5433 100644 --- a/dlt/normalize/worker.py +++ b/dlt/normalize/worker.py @@ -20,6 +20,7 @@ ParsedLoadJobFileName, ) from dlt.common.schema import TSchemaUpdate, Schema +from dlt.common.normalizers.json import helpers as normalize_helpers from dlt.normalize.configuration import NormalizeConfiguration from dlt.normalize.exceptions import NormalizeJobFailed @@ -218,8 +219,8 @@ def _gather_metrics_and_close( parsed_file_name = ParsedLoadJobFileName.parse(extracted_items_file) # normalize table name in case the normalization changed # NOTE: this is the best we can do, until a full lineage information is in the schema - root_table_name = schema.naming.normalize_table_identifier( - parsed_file_name.table_name + root_table_name = normalize_helpers.normalize_table_identifier( + schema, schema.naming, parsed_file_name.table_name ) root_tables.add(root_table_name) root_table = stored_schema["tables"].get(root_table_name, {"name": root_table_name}) diff --git a/dlt/pipeline/pipeline.py b/dlt/pipeline/pipeline.py index 037458f9c1..70d160ea67 100644 --- a/dlt/pipeline/pipeline.py +++ b/dlt/pipeline/pipeline.py @@ -15,6 +15,7 @@ cast, get_type_hints, ContextManager, + Union, ) from dlt import version @@ -37,7 +38,6 @@ from dlt.common.exceptions import MissingDependencyException from dlt.common.runtime import signals, apply_runtime_config from dlt.common.schema.typing import ( - TColumnNames, TSchemaTables, TTableFormat, TWriteDispositionConfig, @@ -46,7 +46,7 @@ ) from dlt.common.schema.utils import normalize_schema_name from dlt.common.storages.exceptions import LoadPackageNotFound -from dlt.common.typing import ConfigValue, TFun, TSecretStrValue, is_optional_type +from dlt.common.typing import ConfigValue, TFun, TSecretStrValue, is_optional_type, TColumnNames from dlt.common.runners import pool_runner as runner from dlt.common.storages import ( LiveSchemaStorage, @@ -110,7 +110,10 @@ from dlt.destinations.sql_client import SqlClientBase, WithSqlClient from dlt.destinations.fs_client import FSClientBase from dlt.destinations.job_client_impl import SqlJobClientBase -from dlt.destinations.dataset import dataset +from dlt.destinations.dataset import ( + dataset, + get_destination_clients, +) from dlt.load.configuration import LoaderConfiguration from dlt.load import Load @@ -583,7 +586,7 @@ def load( return None # make sure that destination is set and client is importable and can be instantiated - client, staging_client = self._get_destination_clients(self.default_schema) + client, staging_client = self._get_destination_clients() # create default loader config and the loader load_config = LoaderConfiguration( @@ -834,7 +837,7 @@ def _sync_destination( if self.default_schema_name is None: should_wipe = True else: - with self._get_destination_clients(self.default_schema)[0] as job_client: + with self._get_destination_clients()[0] as job_client: # and storage is not initialized should_wipe = not job_client.is_storage_initialized() if should_wipe: @@ -1034,8 +1037,7 @@ def sql_client(self, schema_name: str = None) -> SqlClientBase[Any]: # "Sql Client is not available in a pipeline without a default schema. Extract some data first or restore the pipeline from the destination using 'restore_from_destination' flag. There's also `_inject_schema` method for advanced users." # ) schema = self._get_schema_or_create(schema_name) - client_config = self._get_destination_client_initial_config() - client = self._get_destination_clients(schema, client_config)[0] + client = self._get_destination_clients(schema)[0] if isinstance(client, WithSqlClient): return client.sql_client else: @@ -1102,8 +1104,7 @@ def _get_schema_or_create(self, schema_name: str = None) -> Schema: return Schema(self.pipeline_name) def _sql_job_client(self, schema: Schema) -> SqlJobClientBase: - client_config = self._get_destination_client_initial_config() - client = self._get_destination_clients(schema, client_config)[0] + client = self._get_destination_clients(schema)[0] if isinstance(client, SqlJobClientBase): return client else: @@ -1249,11 +1250,13 @@ def _extract_source( return load_id - def _get_destination_client_initial_config( - self, destination: AnyDestination = None, as_staging: bool = False - ) -> DestinationClientConfiguration: - destination = destination or self._destination - if not destination: + def _get_destination_clients( + self, + schema: Schema = None, + initial_config: DestinationClientConfiguration = None, + initial_staging_config: DestinationClientConfiguration = None, + ) -> Tuple[JobClientBase, JobClientBase]: + if not self._destination: raise PipelineConfigMissing( self.pipeline_name, "destination", @@ -1261,75 +1264,32 @@ def _get_destination_client_initial_config( "Please provide `destination` argument to `pipeline`, `run` or `load` method" " directly or via .dlt config.toml file or environment variable.", ) - client_spec = destination.spec - # this client supports many schemas and datasets - if issubclass(client_spec, DestinationClientDwhConfiguration): - if not self.dataset_name and self.dev_mode: - logger.warning( - "Dev mode may not work if dataset name is not set. Please set the" - " dataset_name argument in dlt.pipeline or run method" - ) - # set default schema name to load all incoming data to a single dataset, no matter what is the current schema name - default_schema_name = ( - None if self.config.use_single_dataset else self.default_schema_name - ) - - if issubclass(client_spec, DestinationClientStagingConfiguration): - spec: DestinationClientDwhConfiguration = client_spec( - as_staging_destination=as_staging - ) - else: - spec = client_spec() + destination_client, staging_client = get_destination_clients( + schema=schema or self.default_schema, + default_schema_name=( + self.default_schema_name if not self.config.use_single_dataset else None + ), + destination=self._destination, + destination_dataset_name=self.dataset_name, + destination_initial_config=initial_config, + staging=self._staging, # in case of destination that does not need dataset name, we still must # provide one to staging # TODO: allow for separate staging_dataset_name, that will require to migrate pipeline state # to store it. - dataset_name = self.dataset_name - if not dataset_name and as_staging: - dataset_name = self._make_dataset_name(None, destination) - spec._bind_dataset_name(dataset_name, default_schema_name) - return spec + staging_dataset_name=self.dataset_name or self._make_dataset_name(None, self._staging), + staging_initial_config=initial_staging_config, + ) - return client_spec() + if isinstance(destination_client.config, DestinationClientStagingConfiguration): + if not self.dataset_name and self.dev_mode: + logger.warning( + "Dev mode may not work if dataset name is not set. Please set the" + " dataset_name argument in dlt.pipeline or run method" + ) - def _get_destination_clients( - self, - schema: Schema, - initial_config: DestinationClientConfiguration = None, - initial_staging_config: DestinationClientConfiguration = None, - ) -> Tuple[JobClientBase, JobClientBase]: - try: - # resolve staging config in order to pass it to destination client config - staging_client = None - if self._staging: - if not initial_staging_config: - # this is just initial config - without user configuration injected - initial_staging_config = self._get_destination_client_initial_config( - self._staging, as_staging=True - ) - # create the client - that will also resolve the config - staging_client = self._staging.client(schema, initial_staging_config) - if not initial_config: - # config is not provided then get it with injected credentials - initial_config = self._get_destination_client_initial_config(self._destination) - # attach the staging client config to destination client config - if its type supports it - if ( - self._staging - and isinstance(initial_config, DestinationClientDwhWithStagingConfiguration) - and isinstance(staging_client.config, DestinationClientStagingConfiguration) - ): - initial_config.staging_config = staging_client.config - # create instance with initial_config properly set - client = self._destination.client(schema, initial_config) - return client, staging_client - except ModuleNotFoundError: - client_spec = self._destination.spec() - raise MissingDependencyException( - f"{client_spec.destination_type} destination", - [f"{version.DLT_PKG_NAME}[{client_spec.destination_type}]"], - "Dependencies for specific destinations are available as extras of dlt", - ) + return destination_client, staging_client def _get_destination_capabilities(self) -> DestinationCapabilitiesContext: if not self._destination: @@ -1790,11 +1750,15 @@ def __getstate__(self) -> Any: # pickle only the SupportsPipeline protocol fields return {"pipeline_name": self.pipeline_name} - def _dataset(self, dataset_type: TDatasetType = "dbapi") -> SupportsReadableDataset: + def _dataset( + self, schema: Union[Schema, str, None] = None, dataset_type: TDatasetType = "dbapi" + ) -> SupportsReadableDataset: """Access helper to dataset""" + if schema is None: + schema = self.default_schema if self.default_schema_name else None return dataset( self._destination, self.dataset_name, - schema=(self.default_schema if self.default_schema_name else None), + schema=schema, dataset_type=dataset_type, ) diff --git a/dlt/reflection/script_visitor.py b/dlt/reflection/script_visitor.py index f4a5569ed0..c49fed20ab 100644 --- a/dlt/reflection/script_visitor.py +++ b/dlt/reflection/script_visitor.py @@ -1,10 +1,9 @@ import inspect import ast -import astunparse from ast import NodeVisitor from typing import Any, Dict, List -from dlt.common.reflection.utils import find_outer_func_def +from dlt.common.reflection.utils import find_outer_func_def, ast_unparse import dlt.reflection.names as n @@ -68,9 +67,9 @@ def visit_FunctionDef(self, node: ast.FunctionDef) -> Any: for deco in node.decorator_list: # decorators can be function calls, attributes or names if isinstance(deco, (ast.Name, ast.Attribute)): - alias_name = astunparse.unparse(deco).strip() + alias_name = ast_unparse(deco).strip() elif isinstance(deco, ast.Call): - alias_name = astunparse.unparse(deco.func).strip() + alias_name = ast_unparse(deco.func).strip() else: raise ValueError( self.source_segment(deco), type(deco), "Unknown decorator form" @@ -87,7 +86,7 @@ def visit_FunctionDef(self, node: ast.FunctionDef) -> Any: def visit_Call(self, node: ast.Call) -> Any: if self._curr_pass == 2: # check if this is a call to any of known functions - alias_name = astunparse.unparse(node.func).strip() + alias_name = ast_unparse(node.func).strip() fn = self.func_aliases.get(alias_name) if not fn: # try a fallback to "run" function that may be called on pipeline or source diff --git a/dlt/sources/_core_source_templates/sql_database_pipeline.py b/dlt/sources/_core_source_templates/sql_database_pipeline.py index 4b82997fd7..c33bb046ab 100644 --- a/dlt/sources/_core_source_templates/sql_database_pipeline.py +++ b/dlt/sources/_core_source_templates/sql_database_pipeline.py @@ -121,12 +121,13 @@ def select_columns() -> None: full_refresh=True, ) - def table_adapter(table: Table) -> None: + def table_adapter(table: Table) -> Table: print(table.name) if table.name == "family": # this is SqlAlchemy table. _columns are writable # let's drop updated column table._columns.remove(table.columns["updated"]) # type: ignore + return table family = sql_table( credentials="mysql+pymysql://rfamro@mysql-rfam-public.ebi.ac.uk:4497/Rfam", @@ -184,11 +185,12 @@ def my_sql_via_pyarrow() -> None: dataset_name="rfam_data_arrow_4", ) - def _double_as_decimal_adapter(table: sa.Table) -> None: + def _double_as_decimal_adapter(table: sa.Table) -> sa.Table: """Return double as double, not decimals, only works if you are using sqlalchemy 2.0""" for column in table.columns.values(): if hasattr(sa, "Double") and isinstance(column.type, sa.Double): column.type.asdecimal = False + return table sql_alchemy_source = sql_database( "mysql+pymysql://rfamro@mysql-rfam-public.ebi.ac.uk:4497/Rfam?&binary_prefix=true", @@ -277,11 +279,12 @@ def test_pandas_backend_verbatim_decimals() -> None: dataset_name="rfam_data_pandas_2", ) - def _double_as_decimal_adapter(table: sa.Table) -> None: + def _double_as_decimal_adapter(table: sa.Table) -> sa.Table: """Emits decimals instead of floats.""" for column in table.columns.values(): if isinstance(column.type, sa.Float): column.type.asdecimal = True + return table sql_alchemy_source = sql_database( "mysql+pymysql://rfamro@mysql-rfam-public.ebi.ac.uk:4497/Rfam?&binary_prefix=true", diff --git a/dlt/sources/helpers/rest_client/client.py b/dlt/sources/helpers/rest_client/client.py index 6d04373d8d..a619a05a00 100644 --- a/dlt/sources/helpers/rest_client/client.py +++ b/dlt/sources/helpers/rest_client/client.py @@ -58,6 +58,7 @@ class RESTClient: auth (Optional[AuthBase]): Authentication configuration for all requests. paginator (Optional[BasePaginator]): Default paginator for handling paginated responses. data_selector (Optional[jsonpath.TJsonPath]): JSONPath selector for extracting data from responses. + Only used in `paginate`. session (BaseSession): HTTP session for making requests. paginator_factory (Optional[PaginatorFactory]): Factory for creating paginator instances, used for detecting paginators. @@ -96,18 +97,18 @@ def __init__( def _create_request( self, - path: str, + path_or_url: str, method: HTTPMethod, params: Optional[Dict[str, Any]] = None, json: Optional[Dict[str, Any]] = None, auth: Optional[AuthBase] = None, hooks: Optional[Hooks] = None, ) -> Request: - parsed_url = urlparse(path) + parsed_url = urlparse(path_or_url) if parsed_url.scheme in ("http", "https"): - url = path + url = path_or_url else: - url = join_url(self.base_url, path) + url = join_url(self.base_url, path_or_url) return Request( method=method, @@ -140,7 +141,7 @@ def _send_request(self, request: Request, **kwargs: Any) -> Response: def request(self, path: str = "", method: HTTPMethod = "GET", **kwargs: Any) -> Response: prepared_request = self._create_request( - path=path, + path_or_url=path, method=method, params=kwargs.pop("params", None), json=kwargs.pop("json", None), @@ -171,6 +172,8 @@ def paginate( Args: path (str): Endpoint path for the request, relative to `base_url`. + Can also be a fully qualified URL; if starting with http(s) it will + be used instead of the base_url + path. method (HTTPMethodBasic): HTTP method for the request, defaults to 'get'. params (Optional[Dict[str, Any]]): URL parameters for the request. json (Optional[Dict[str, Any]]): JSON payload for the request. @@ -210,7 +213,7 @@ def paginate( hooks["response"] = [raise_for_status] request = self._create_request( - path=path, method=method, params=params, json=json, auth=auth, hooks=hooks + path_or_url=path, method=method, params=params, json=json, auth=auth, hooks=hooks ) if paginator: diff --git a/dlt/sources/rest_api/__init__.py b/dlt/sources/rest_api/__init__.py index ed55f71e10..966d9e8b6c 100644 --- a/dlt/sources/rest_api/__init__.py +++ b/dlt/sources/rest_api/__init__.py @@ -266,7 +266,7 @@ def create_resources( client = RESTClient( base_url=client_config["base_url"], headers=client_config.get("headers"), - auth=create_auth(client_config.get("auth")), + auth=create_auth(endpoint_config.get("auth", client_config.get("auth"))), paginator=create_paginator(client_config.get("paginator")), session=client_config.get("session"), ) @@ -409,7 +409,16 @@ def _validate_config(config: RESTAPIConfig) -> None: if client_config: auth = client_config.get("auth") if auth: - auth = _mask_secrets(auth) + _mask_secrets(auth) + resources = c.get("resources", []) + for resource in resources: + if isinstance(resource, (str, DltResource)): + continue + if endpoint := resource.get("endpoint"): + if not isinstance(endpoint, str): + auth = endpoint.get("auth") + if auth: + _mask_secrets(auth) validate_dict(RESTAPIConfig, c, path=".") diff --git a/dlt/sources/rest_api/typing.py b/dlt/sources/rest_api/typing.py index d4cea892a3..c48e54de4a 100644 --- a/dlt/sources/rest_api/typing.py +++ b/dlt/sources/rest_api/typing.py @@ -15,7 +15,7 @@ from dlt.common.schema.typing import ( TAnySchemaColumns, ) -from dlt.extract.incremental.typing import IncrementalArgs +from dlt.common.incremental.typing import IncrementalArgs from dlt.extract.items import TTableHintTemplate from dlt.extract.hints import TResourceHintsBase from dlt.sources.helpers.rest_client.auth import AuthConfigBase, TApiKeyLocation @@ -23,9 +23,8 @@ from dataclasses import dataclass, field from dlt.common import jsonpath -from dlt.common.typing import TSortOrder +from dlt.common.typing import TSortOrder, TColumnNames from dlt.common.schema.typing import ( - TColumnNames, TTableFormat, TAnySchemaColumns, TWriteDispositionConfig, @@ -33,7 +32,7 @@ ) from dlt.extract.items import TTableHintTemplate -from dlt.extract.incremental.typing import LastValueFunc +from dlt.common.incremental.typing import LastValueFunc from dlt.extract.resource import DltResource from requests import Session @@ -263,6 +262,7 @@ class Endpoint(TypedDict, total=False): data_selector: Optional[jsonpath.TJsonPath] response_actions: Optional[List[ResponseAction]] incremental: Optional[IncrementalConfig] + auth: Optional[AuthConfig] class ProcessingSteps(TypedDict): diff --git a/dlt/sources/sql_database/__init__.py b/dlt/sources/sql_database/__init__.py index 4f65b26f04..f7d238641e 100644 --- a/dlt/sources/sql_database/__init__.py +++ b/dlt/sources/sql_database/__init__.py @@ -4,22 +4,24 @@ import dlt from dlt.common.configuration.specs import ConnectionStringCredentials +from dlt.common.schema.typing import TWriteDispositionConfig from dlt.common.libs.sql_alchemy import MetaData, Table, Engine from dlt.extract import DltResource, Incremental, decorators from .helpers import ( + _execute_table_adapter, table_rows, engine_from_credentials, + remove_nullability_adapter, TableBackend, SqlTableResourceConfiguration, _detect_precision_hints_deprecated, TQueryAdapter, + TTableAdapter, ) from .schema_types import ( - default_table_adapter, table_to_resource_hints, - get_primary_key, ReflectionLevel, TTypeAdapter, ) @@ -36,12 +38,13 @@ def sql_database( detect_precision_hints: Optional[bool] = False, reflection_level: Optional[ReflectionLevel] = "full", defer_table_reflect: Optional[bool] = None, - table_adapter_callback: Callable[[Table], None] = None, + table_adapter_callback: Optional[TTableAdapter] = None, backend_kwargs: Dict[str, Any] = None, include_views: bool = False, type_adapter_callback: Optional[TTypeAdapter] = None, query_adapter_callback: Optional[TQueryAdapter] = None, resolve_foreign_keys: bool = False, + engine_adapter_callback: Callable[[Engine], Engine] = None, ) -> Iterable[DltResource]: """ A dlt source which loads data from an SQL database using SQLAlchemy. @@ -60,8 +63,8 @@ def sql_database( detect_precision_hints (bool): Deprecated. Use `reflection_level`. Set column precision and scale hints for supported data types in the target schema based on the columns in the source tables. This is disabled by default. reflection_level: (ReflectionLevel): Specifies how much information should be reflected from the source database schema. - "minimal": Only table names, nullability and primary keys are reflected. Data types are inferred from the data. - "full": Data types will be reflected on top of "minimal". `dlt` will coerce the data into reflected types if necessary. This is the default option. + "minimal": Only table names, nullability and primary keys are reflected. Data types are inferred from the data. This is the default option. + "full": Data types will be reflected on top of "minimal". `dlt` will coerce the data into reflected types if necessary. "full_with_precision": Sets precision and scale on supported data types (ie. decimal, text, binary). Creates big and regular integer types. defer_table_reflect (bool): Will connect and reflect table schema only when yielding data. Requires table_names to be explicitly passed. Enable this option when running on Airflow. Available on dlt 0.4.4 and later. @@ -71,9 +74,11 @@ def sql_database( type_adapter_callback(Optional[Callable]): Callable to override type inference when reflecting columns. Argument is a single sqlalchemy data type (`TypeEngine` instance) and it should return another sqlalchemy data type, or `None` (type will be inferred from data) query_adapter_callback(Optional[Callable[Select, Table], Select]): Callable to override the SELECT query used to fetch data from the table. - The callback receives the sqlalchemy `Select` and corresponding `Table` objects and should return the modified `Select`. + The callback receives the sqlalchemy `Select` and corresponding `Table`, 'Incremental` and `Engine` objects and should return the modified `Select` or `Text`. resolve_foreign_keys (bool): Translate foreign keys in the same schema to `references` table hints. May incur additional database calls as all referenced tables are reflected. + engine_adapter_callback (Callable[[Engine], Engine]): Callback to configure, modify and Engine instance that will be used to open a connection ie. to + set transaction isolation level. Returns: Iterable[DltResource]: A list of DLT resources for each table to be loaded. @@ -89,6 +94,8 @@ def sql_database( # set up alchemy engine engine = engine_from_credentials(credentials) engine.execution_options(stream_results=True, max_row_buffer=2 * chunk_size) + if engine_adapter_callback: + engine = engine_adapter_callback(engine) metadata = metadata or MetaData(schema=schema) if defer_table_reflect: @@ -126,6 +133,7 @@ def sql_database( type_adapter_callback=type_adapter_callback, query_adapter_callback=query_adapter_callback, resolve_foreign_keys=resolve_foreign_keys, + engine_adapter_callback=engine_adapter_callback, ) @@ -143,12 +151,14 @@ def sql_table( detect_precision_hints: Optional[bool] = None, reflection_level: Optional[ReflectionLevel] = "full", defer_table_reflect: Optional[bool] = None, - table_adapter_callback: Callable[[Table], None] = None, + table_adapter_callback: Optional[TTableAdapter] = None, backend_kwargs: Dict[str, Any] = None, type_adapter_callback: Optional[TTypeAdapter] = None, included_columns: Optional[List[str]] = None, query_adapter_callback: Optional[TQueryAdapter] = None, resolve_foreign_keys: bool = False, + engine_adapter_callback: Callable[[Engine], Engine] = None, + write_disposition: TWriteDispositionConfig = "append", ) -> DltResource: """ A dlt resource which loads data from an SQL database table using SQLAlchemy. @@ -166,8 +176,8 @@ def sql_table( "sqlalchemy" is the default and does not require additional dependencies, "pyarrow" creates stable destination schemas with correct data types, "connectorx" is typically the fastest but ignores the "chunk_size" so you must deal with large tables yourself. reflection_level: (ReflectionLevel): Specifies how much information should be reflected from the source database schema. - "minimal": Only table names, nullability and primary keys are reflected. Data types are inferred from the data. - "full": Data types will be reflected on top of "minimal". `dlt` will coerce the data into reflected types if necessary. This is the default option. + "minimal": Only table names, nullability and primary keys are reflected. Data types are inferred from the data. This is the default option. + "full": Data types will be reflected on top of "minimal". `dlt` will coerce the data into reflected types if necessary. "full_with_precision": Sets precision and scale on supported data types (ie. decimal, text, binary). Creates big and regular integer types. detect_precision_hints (bool): Deprecated. Use `reflection_level`. Set column precision and scale hints for supported data types in the target schema based on the columns in the source tables. This is disabled by default. @@ -179,9 +189,12 @@ def sql_table( Argument is a single sqlalchemy data type (`TypeEngine` instance) and it should return another sqlalchemy data type, or `None` (type will be inferred from data) included_columns (Optional[List[str]): List of column names to select from the table. If not provided, all columns are loaded. query_adapter_callback(Optional[Callable[Select, Table], Select]): Callable to override the SELECT query used to fetch data from the table. - The callback receives the sqlalchemy `Select` and corresponding `Table` objects and should return the modified `Select`. + The callback receives the sqlalchemy `Select` and corresponding `Table`, 'Incremental` and `Engine` objects and should return the modified `Select` or `Text`. resolve_foreign_keys (bool): Translate foreign keys in the same schema to `references` table hints. May incur additional database calls as all referenced tables are reflected. + engine_adapter_callback (Callable[[Engine], Engine]): Callback to configure, modify and Engine instance that will be used to open a connection ie. to + set transaction isolation level. + write_disposition (TWriteDispositionConfig): write disposition of the table resource, defaults to `append`. Returns: DltResource: The dlt resource for loading data from the SQL database table. @@ -195,6 +208,8 @@ def sql_table( engine = engine_from_credentials(credentials, may_dispose_after_use=True) engine.execution_options(stream_results=True, max_row_buffer=2 * chunk_size) + if engine_adapter_callback: + engine = engine_adapter_callback(engine) metadata = metadata or MetaData(schema=schema) skip_nested_on_minimal = backend == "sqlalchemy" @@ -206,9 +221,7 @@ def sql_table( if table_obj is not None: if not defer_table_reflect: - default_table_adapter(table_obj, included_columns) - if table_adapter_callback: - table_adapter_callback(table_obj) + table_obj = _execute_table_adapter(table_obj, table_adapter_callback, included_columns) hints = table_to_resource_hints( table_obj, reflection_level, @@ -219,7 +232,9 @@ def sql_table( else: hints = {} - return decorators.resource(table_rows, name=table, **hints)( + return decorators.resource( + table_rows, name=table, write_disposition=write_disposition, **hints + )( engine, table_obj if table_obj is not None else table, # Pass table name if reflection deferred metadata, @@ -234,3 +249,16 @@ def sql_table( query_adapter_callback=query_adapter_callback, resolve_foreign_keys=resolve_foreign_keys, ) + + +__all__ = [ + "sql_database", + "sql_table", + "ReflectionLevel", + "TTypeAdapter", + "engine_from_credentials", + "remove_nullability_adapter", + "TableBackend", + "TQueryAdapter", + "TTableAdapter", +] diff --git a/dlt/sources/sql_database/arrow_helpers.py b/dlt/sources/sql_database/arrow_helpers.py index 1f72205a2a..1de9dffc87 100644 --- a/dlt/sources/sql_database/arrow_helpers.py +++ b/dlt/sources/sql_database/arrow_helpers.py @@ -4,9 +4,6 @@ from dlt.common.configuration import with_config from dlt.common.destination import DestinationCapabilitiesContext -from dlt.common.libs.pyarrow import ( - row_tuples_to_arrow as _row_tuples_to_arrow, -) @with_config @@ -20,6 +17,8 @@ def row_tuples_to_arrow( is always the case if run within the pipeline. This will generate arrow schema compatible with the destination. Otherwise generic capabilities are used """ + from dlt.common.libs.pyarrow import row_tuples_to_arrow as _row_tuples_to_arrow + return _row_tuples_to_arrow( rows, caps or DestinationCapabilitiesContext.generic_capabilities(), columns, tz ) diff --git a/dlt/sources/sql_database/helpers.py b/dlt/sources/sql_database/helpers.py index 235b96ac64..a8be2a6427 100644 --- a/dlt/sources/sql_database/helpers.py +++ b/dlt/sources/sql_database/helpers.py @@ -5,6 +5,7 @@ Callable, Any, Dict, + Iterable, List, Literal, Optional, @@ -22,7 +23,9 @@ from dlt.common.exceptions import MissingDependencyException from dlt.common.schema import TTableSchemaColumns from dlt.common.typing import TDataItem, TSortOrder +from dlt.common.jsonpath import extract_simple_field_name +from dlt.common.utils import is_typeerror_due_to_wrong_call from dlt.extract import Incremental from .arrow_helpers import row_tuples_to_arrow @@ -35,10 +38,22 @@ table_to_resource_hints, ) -from dlt.common.libs.sql_alchemy import Engine, CompileError, create_engine, MetaData, sa +from dlt.common.libs.sql_alchemy import ( + Engine, + CompileError, + create_engine, + MetaData, + sa, + TextClause, +) TableBackend = Literal["sqlalchemy", "pyarrow", "pandas", "connectorx"] -TQueryAdapter = Callable[[SelectAny, Table], SelectAny] +SelectClause = Union[SelectAny, TextClause] +TQueryAdapter = Union[ + Callable[[SelectAny, Table], SelectClause], + Callable[[SelectAny, Table, Incremental[Any], Engine], SelectClause], +] +TTableAdapter = Callable[[Table], Optional[Union[SelectAny, Table]]] class TableLoader: @@ -60,8 +75,16 @@ def __init__( self.query_adapter_callback = query_adapter_callback self.incremental = incremental if incremental: + column_name = extract_simple_field_name(incremental.cursor_path) + + if column_name is None: + raise ValueError( + f"Cursor path '{incremental.cursor_path}' must be a simple column name (e.g." + " 'created_at')" + ) + try: - self.cursor_column = table.c[incremental.cursor_path] + self.cursor_column = table.c[column_name] except KeyError as e: raise KeyError( f"Cursor column '{incremental.cursor_path}' does not exist in table" @@ -126,9 +149,19 @@ def _make_query(self) -> SelectAny: return query # type: ignore[no-any-return] - def make_query(self) -> SelectAny: + def make_query(self) -> SelectClause: if self.query_adapter_callback: - return self.query_adapter_callback(self._make_query(), self.table) + try: + return self.query_adapter_callback( # type: ignore[call-arg] + self._make_query(), self.table, self.incremental, self.engine + ) + except TypeError as type_err: + if not is_typeerror_due_to_wrong_call(type_err, self.query_adapter_callback): + raise + return self.query_adapter_callback( # type: ignore[call-arg] + self._make_query(), self.table + ) + return self._make_query() def load_rows(self, backend_kwargs: Dict[str, Any] = None) -> Iterator[TDataItem]: @@ -140,7 +173,7 @@ def load_rows(self, backend_kwargs: Dict[str, Any] = None) -> Iterator[TDataItem else: yield from self._load_rows(query, backend_kwargs) - def _load_rows(self, query: SelectAny, backend_kwargs: Dict[str, Any]) -> TDataItem: + def _load_rows(self, query: SelectClause, backend_kwargs: Dict[str, Any]) -> TDataItem: with self.engine.connect() as conn: result = conn.execution_options(yield_per=self.chunk_size).execute(query) # NOTE: cursor returns not normalized column names! may be quite useful in case of Oracle dialect @@ -161,11 +194,13 @@ def _load_rows(self, query: SelectAny, backend_kwargs: Dict[str, Any]) -> TDataI yield df elif self.backend == "pyarrow": yield row_tuples_to_arrow( - partition, columns=self.columns, tz=backend_kwargs.get("tz", "UTC") + partition, + columns=_add_missing_columns(self.columns, columns), + tz=backend_kwargs.get("tz", "UTC"), ) def _load_rows_connectorx( - self, query: SelectAny, backend_kwargs: Dict[str, Any] + self, query: SelectClause, backend_kwargs: Dict[str, Any] ) -> Iterator[TDataItem]: try: import connectorx as cx @@ -203,7 +238,7 @@ def table_rows( chunk_size: int, backend: TableBackend, incremental: Optional[Incremental[Any]] = None, - table_adapter_callback: Callable[[Table], None] = None, + table_adapter_callback: TTableAdapter = None, reflection_level: ReflectionLevel = "minimal", backend_kwargs: Dict[str, Any] = None, type_adapter_callback: Optional[TTypeAdapter] = None, @@ -219,10 +254,7 @@ def table_rows( extend_existing=True, resolve_fks=resolve_foreign_keys, ) - default_table_adapter(table, included_columns) - if table_adapter_callback: - table_adapter_callback(table) - + table = _execute_table_adapter(table, table_adapter_callback, included_columns) hints = table_to_resource_hints( table, reflection_level, @@ -306,6 +338,42 @@ def _unwrap(table: TDataItem) -> TDataItem: return _unwrap +def remove_nullability_adapter(table: Table) -> Table: + """A table adapter that removes nullability from columns.""" + for col in table.columns: + # subqueries may not have nullable attr + if hasattr(col, "nullable"): + col.nullable = None + return table + + +def _add_missing_columns( + schema_columns: TTableSchemaColumns, result_columns: Iterable[str] +) -> TTableSchemaColumns: + """Adds columns present in cursor but not present in schema""" + for column_name in result_columns: + if column_name not in schema_columns: + schema_columns[column_name] = {"name": column_name} + return schema_columns + + +def _execute_table_adapter( + table: Table, adapter: Optional[TTableAdapter], included_columns: Optional[List[str]] +) -> Table: + """Executes default table adapter on `table` and then `adapter` if defined""" + default_table_adapter(table, included_columns) + if adapter: + # backward compat: old adapters do not return a value + maybe_query = adapter(table) + if maybe_query is not None: + # here we ignore that returned table may be a Select (subquery) + # otherwise typing gets really complicated + # TODO: maybe type that later + table = maybe_query # type: ignore[assignment] + + return table + + def _detect_precision_hints_deprecated(value: Optional[bool]) -> None: if value is None: return diff --git a/dlt/sources/sql_database/schema_types.py b/dlt/sources/sql_database/schema_types.py index b2b53c46c2..e75cbdc5d5 100644 --- a/dlt/sources/sql_database/schema_types.py +++ b/dlt/sources/sql_database/schema_types.py @@ -71,8 +71,11 @@ def sqla_col_to_column_schema( """ col: TColumnSchema = { "name": sql_col.name, - "nullable": sql_col.nullable, } + # NOTE: nullability info may not be available for subquery columns + if hasattr(sql_col, "nullable") and sql_col.nullable is not None: + col["nullable"] = sql_col.nullable + if reflection_level == "minimal": # normalized into subtables if isinstance(sql_col.type, sqltypes.JSON) and skip_nested_columns_on_minimal: diff --git a/docs/examples/partial_loading/partial_loading.py b/docs/examples/partial_loading/partial_loading.py index 1d73a2aec0..60485e0a7e 100644 --- a/docs/examples/partial_loading/partial_loading.py +++ b/docs/examples/partial_loading/partial_loading.py @@ -104,9 +104,7 @@ def delete_old_backfills(load_info: LoadInfo, p: dlt.Pipeline, table_name: str) pattern = re.compile(rf"{load_id}") # Compile regex pattern for the current load ID # Initialize the filesystem client - fs_client: FilesystemClient = p.destination.client( # type: ignore - p.default_schema, initial_config=p._get_destination_client_initial_config(p.destination) - ) + fs_client: FilesystemClient = p._get_destination_clients()[0] # type: ignore # Construct the table directory path table_dir = os.path.join(fs_client.dataset_path, table_name) diff --git a/docs/website/docs/dlt-ecosystem/destinations/clickhouse.md b/docs/website/docs/dlt-ecosystem/destinations/clickhouse.md index 5ca25af55c..40ee5d71e8 100644 --- a/docs/website/docs/dlt-ecosystem/destinations/clickhouse.md +++ b/docs/website/docs/dlt-ecosystem/destinations/clickhouse.md @@ -126,7 +126,7 @@ clickhouse destination. :::tip -`dataset_name` is optional for Clikchouse. When skipped `dlt` will create all tables without prefix. Note that staging dataset +`dataset_name` is optional for ClickHouse. When skipped `dlt` will create all tables without prefix. Note that staging dataset tables will still be prefixed with `_staging` (or other name that you configure). ::: @@ -229,8 +229,7 @@ To set up GCS staging with HMAC authentication in dlt: 1. Create HMAC keys for your GCS service account by following the [Google Cloud guide](https://cloud.google.com/storage/docs/authentication/managing-hmackeys#create). -2. Configure the HMAC keys (`aws_access_key_id` and `aws_secret_access_key`) in your dlt project's ClickHouse destination settings in `config.toml`, similar to how you would configure AWS S3 - credentials: +2. Configure the HMAC keys (`aws_access_key_id` and `aws_secret_access_key`) as well as `endpoint_url` in your dlt project's ClickHouse destination settings in `config.toml`, similar to how you would configure AWS S3 credentials: ```toml [destination.filesystem] diff --git a/docs/website/docs/dlt-ecosystem/destinations/filesystem.md b/docs/website/docs/dlt-ecosystem/destinations/filesystem.md index 4e9bf1068e..9b243b9429 100644 --- a/docs/website/docs/dlt-ecosystem/destinations/filesystem.md +++ b/docs/website/docs/dlt-ecosystem/destinations/filesystem.md @@ -122,14 +122,21 @@ endpoint_url = "https://.r2.cloudflarestorage.com" # copy your endpo #### Adding additional configuration -To pass any additional arguments to `fsspec`, you may supply `kwargs` and `client_kwargs` in the config as a **stringified dictionary**: +To pass any additional arguments to `fsspec`, you may supply `kwargs` and `client_kwargs` in toml config. ```toml -[destination.filesystem] -kwargs = '{"use_ssl": true, "auto_mkdir": true}' -client_kwargs = '{"verify": "public.crt"}' +[destination.filesystem.kwargs] +use_ssl=true +auto_mkdir=true + +[destination.filesystem.client_kwargs] +verify="public.crt" ``` +To pass additional arguments via env variables, use **stringified dictionary**: +`DESTINATION__FILESYSTEM__KWARGS='{"use_ssl": true, "auto_mkdir": true}` + + ### Google storage Run `pip install "dlt[gs]"` which will install the `gcfs` package. @@ -159,6 +166,31 @@ 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. +`dlt` supports both forms of the blob storage urls: +```toml +[destination.filesystem] +bucket_url = "az:///path" # replace with your container name and path +``` + +and + +```toml +[destination.filesystem] +bucket_url = "abfss://@.dfs.core.windows.net/path" +``` + +You can use `az`, `abfss`, `azure` and `abfs` url schemes. + +If you need to use a custom host for your storage account, you can set it up like below: +```toml +[destination.filesystem.credentials] +# The storage account name is always required +azure_account_host = "." +``` +Remember to include `storage_account_name` with your base host ie. `dlt_ci.blob.core.usgovcloudapi.net`. +`dlt` will use this host to connect to azure blob storage without any modifications: + + Two forms of Azure credentials are supported: #### SAS token credentials @@ -166,9 +198,6 @@ Two forms of Azure credentials are supported: Supply storage account name and either SAS token or storage account key ```toml -[destination.filesystem] -bucket_url = "az://[your_container name]" # replace with your container name - [destination.filesystem.credentials] # The storage account name is always required azure_storage_account_name = "account_name" # please set me up! @@ -186,10 +215,8 @@ Note that `azure_storage_account_name` is still required as it can't be inferred Supply a client ID, client secret, and a tenant ID for a service principal authorized to access your container. ```toml -[destination.filesystem] -bucket_url = "az://[your_container name]" # replace with your container name - [destination.filesystem.credentials] +azure_storage_account_name = "account_name" # please set me up! azure_client_id = "client_id" # please set me up! azure_client_secret = "client_secret" azure_tenant_id = "tenant_id" # please set me up! @@ -503,7 +530,7 @@ layout="{table_name}/{load_id}.{file_id}.{ext}" # current preconfigured naming s # layout = "{table_name}/{load_package_timestamp}/{load_id}.{file_id}.{ext}" # Parquet-like layout (note: it is not compatible with the internal datetime of the parquet file) -# layout = "{table_name}/year={year}/month={month}/day={day}/{load_id}.{file_id}.{ext}" +# layout = "{table_name}/year={YYYY}/month={MM}/day={DD}/{load_id}.{file_id}.{ext}" # Custom placeholders # extra_placeholders = { "owner" = "admin", "department" = "finance" } diff --git a/docs/website/docs/dlt-ecosystem/destinations/lancedb.md b/docs/website/docs/dlt-ecosystem/destinations/lancedb.md index b2aec665ab..035f27fe32 100644 --- a/docs/website/docs/dlt-ecosystem/destinations/lancedb.md +++ b/docs/website/docs/dlt-ecosystem/destinations/lancedb.md @@ -33,8 +33,10 @@ Configure the destination in the dlt secrets file located at `~/.dlt/secrets.tom ```toml [destination.lancedb] -embedding_model_provider = "cohere" -embedding_model = "embed-english-v3.0" +embedding_model_provider = "ollama" +embedding_model = "mxbai-embed-large" +embedding_model_provider_host = "http://localhost:11434" # Optional: custom endpoint for providers that support it + [destination.lancedb.credentials] uri = ".lancedb" api_key = "api_key" # API key to connect to LanceDB Cloud. Leave out if you are using LanceDB OSS. @@ -47,6 +49,7 @@ embedding_model_provider_api_key = "embedding_model_provider_api_key" # Not need - The `embedding_model` specifies the model used by the embedding provider for generating embeddings. Check with the embedding provider which options are available. Reference https://lancedb.github.io/lancedb/embeddings/default_embedding_functions/. +- The `embedding_model_provider_host` specifies the full host URL with protocol and port for providers that support custom endpoints (like Ollama). If not specified, the provider's default endpoint will be used. - The `embedding_model_provider_api_key` is the API key for the embedding model provider used to generate embeddings. If you're using a provider that doesn't need authentication, such as Ollama, you don't need to supply this key. :::info Available model providers @@ -61,6 +64,7 @@ embedding_model_provider_api_key = "embedding_model_provider_api_key" # Not need - "sentence-transformers" - "huggingface" - "colbert" +- "ollama" ::: ### Define your data source diff --git a/docs/website/docs/dlt-ecosystem/destinations/postgres.md b/docs/website/docs/dlt-ecosystem/destinations/postgres.md index bb9aba9051..922b187a7e 100644 --- a/docs/website/docs/dlt-ecosystem/destinations/postgres.md +++ b/docs/website/docs/dlt-ecosystem/destinations/postgres.md @@ -117,7 +117,57 @@ In the example above, `arrow_table` will be converted to CSV with **pyarrow** an ## Supported column hints `postgres` will create unique indexes for all columns with `unique` hints. This behavior **may be disabled**. -### Table and column identifiers +### Spatial Types + +To enable GIS capabilities in your Postgres destination, use the `x-postgres-geometry` and `x-postgres-srid` hints for columns containing geometric data. +The `postgres_adapter` facilitates applying these hints conveniently, with a default SRID of `4326`. + +**Supported Geometry Types:** + +- WKT (Well-Known Text) +- Hex Representation + +If you have geometry data in binary format, you will need to convert it to hexadecimal representation before loading. + +**Example:** Using `postgres_adapter` with Different Geometry Types + +```py +from dlt.destinations.impl.postgres.postgres_adapter import postgres_adapter + +# Sample data with various geometry types +data_wkt = [ + {"type": "Point_wkt", "geom": "POINT (1 1)"}, + {"type": "Point_wkt", "geom": "Polygon([(0, 0), (1, 0), (1, 1), (0, 1), (0, 0)])"}, + ] + +data_wkb_hex = [ + {"type": "Point_wkb_hex", "geom": "0101000000000000000000F03F000000000000F03F"}, + {"type": "Point_wkb_hex", "geom": "01020000000300000000000000000000000000000000000000000000000000F03F000000000000F03F00000000000000400000000000000040"}, +] + + + +# Apply postgres_adapter to the 'geom' column with default SRID 4326 +resource_wkt = postgres_adapter(data_wkt, geometry="geom") +resource_wkb_hex = postgres_adapter(data_wkb_hex, geometry="geom") + +# If you need a different SRID +resource_wkt = postgres_adapter(data_wkt, geometry="geom", srid=3242) +``` + +Ensure that the PostGIS extension is enabled in your Postgres database: + +```sql +CREATE EXTENSION postgis; +``` + +This configuration allows `dlt` to map the `geom` column to the PostGIS `geometry` type for spatial queries and analyses. + +:::warning +`LinearRing` geometry type isn't supported. +::: + +## Table and column identifiers Postgres supports both case-sensitive and case-insensitive identifiers. All unquoted and lowercase identifiers resolve case-insensitively in SQL statements. Case insensitive [naming conventions](../../general-usage/naming-convention.md#case-sensitive-and-insensitive-destinations) like the default **snake_case** will generate case-insensitive identifiers. Case sensitive (like **sql_cs_v1**) will generate case-sensitive identifiers that must be quoted in SQL statements. ## Additional destination options diff --git a/docs/website/docs/dlt-ecosystem/destinations/redshift.md b/docs/website/docs/dlt-ecosystem/destinations/redshift.md index 3108004712..4b57877f00 100644 --- a/docs/website/docs/dlt-ecosystem/destinations/redshift.md +++ b/docs/website/docs/dlt-ecosystem/destinations/redshift.md @@ -67,6 +67,10 @@ You can also pass a database connection string similar to the one used by the `p destination.redshift.credentials="redshift://loader:@localhost/dlt_data?connect_timeout=15" ``` +:::note +Use the PostgreSQL driver for PostgreSQL-based setups or the Amazon Redshift driver for native Redshift; [see documentation](https://docs.aws.amazon.com/redshift/latest/dg/c_redshift-postgres-jdbc.html). +::: + ## Write disposition All [write dispositions](../../general-usage/incremental-loading#choosing-a-write-disposition) are supported. diff --git a/docs/website/docs/dlt-ecosystem/notebooks.md b/docs/website/docs/dlt-ecosystem/notebooks.md deleted file mode 100644 index 4486b81b68..0000000000 --- a/docs/website/docs/dlt-ecosystem/notebooks.md +++ /dev/null @@ -1,27 +0,0 @@ ---- -title: dlt in notebooks -description: Run dlt in notebooks like Colab, Databricks or Jupyter -keywords: [notebook, jupyter] ---- -# dlt in notebooks - -## Colab -You'll need to install `dlt` like any other dependency: -```sh -!pip install dlt -``` - -You can configure secrets using **Secrets** sidebar. Just create a variable with the name `secrets.toml` and paste -the content of the **toml** file from your `.dlt` folder into it. We support `config.toml` variable as well. - -:::note -`dlt` will not reload the secrets automatically. Please restart your interpreter in Colab options when you add/change -content of the variables above. -::: - -## Streamlit -`dlt` will look for `secrets.toml` and `config.toml` in the `.dlt` folder. If `secrets.toml` are not found, it will use -`secrets.toml` from `.streamlit` folder. -If you run locally, maintain your usual `.dlt` folder. When running on streamlit cloud, paste the content of `dlt` -`secrets.toml` into the `streamlit` secrets. - diff --git a/docs/website/docs/dlt-ecosystem/verified-sources/arrow-pandas.md b/docs/website/docs/dlt-ecosystem/verified-sources/arrow-pandas.md index 11d4382a22..fa5cf7b128 100644 --- a/docs/website/docs/dlt-ecosystem/verified-sources/arrow-pandas.md +++ b/docs/website/docs/dlt-ecosystem/verified-sources/arrow-pandas.md @@ -39,6 +39,8 @@ pipeline = dlt.pipeline("orders_pipeline", destination="snowflake") pipeline.run(df, table_name="orders") ``` +Note that Pandas indexes are not save by default (up from `dlt` version 1.4.1). If for some reason you need the destination, +use `Table.from_pandas` with `preserve_index` set to True to explicitly convert the dataframe into arrow table. A `pyarrow` table can be loaded in the same way: diff --git a/docs/website/docs/dlt-ecosystem/verified-sources/rest_api/basic.md b/docs/website/docs/dlt-ecosystem/verified-sources/rest_api/basic.md index d23f3f139e..14d9ecb04b 100644 --- a/docs/website/docs/dlt-ecosystem/verified-sources/rest_api/basic.md +++ b/docs/website/docs/dlt-ecosystem/verified-sources/rest_api/basic.md @@ -24,7 +24,7 @@ source = rest_api_source({ "token": dlt.secrets["your_api_token"], }, "paginator": { - "type": "json_response", + "type": "json_link", "next_url_path": "paging.next", }, }, @@ -308,6 +308,32 @@ A resource configuration is used to define a [dlt resource](../../../general-usa - `include_from_parent`: A list of fields from the parent resource to be included in the resource output. See the [resource relationships](#include-fields-from-the-parent-resource) section for more details. - `processing_steps`: A list of [processing steps](#processing-steps-filter-and-transform-data) to filter and transform the data. - `selected`: A flag to indicate if the resource is selected for loading. This could be useful when you want to load data only from child resources and not from the parent resource. +- `auth`: An optional `AuthConfig` instance. If passed, is used over the one defined in the [client](#client) definition. Example: +```py +from dlt.sources.helpers.rest_client.auth import HttpBasicAuth + +config = { + "client": { + "auth": { + "type": "bearer", + "token": dlt.secrets["your_api_token"], + } + }, + "resources": [ + "resource-using-bearer-auth", + { + "name": "my-resource-with-special-auth", + "endpoint": { + # ... + "auth": HttpBasicAuth("user", dlt.secrets["your_basic_auth_password"]) + }, + # ... + } + ] + # ... +} +``` +This would use `Bearer` auth as defined in the `client` for `resource-using-bearer-auth` and `Http Basic` auth for `my-resource-with-special-auth`. You can also pass additional resource parameters that will be used to configure the dlt resource. See [dlt resource API reference](../../../api_reference/extract/decorators#resource) for more details. @@ -335,7 +361,8 @@ The endpoint configuration defines how to query the API endpoint. Quick example: The fields in the endpoint configuration are: -- `path`: The path to the API endpoint. +- `path`: The path to the API endpoint. By default this path is appended to the given `base_url`. If this is a fully qualified URL starting with `http:` or `https:` it will be +used as-is and `base_url` will be ignored. - `method`: The HTTP method to be used. The default is `GET`. - `params`: Query parameters to be sent with each request. For example, `sort` to order the results or `since` to specify [incremental loading](#incremental-loading). This is also used to define [resource relationships](#define-resource-relationships). - `json`: The JSON payload to be sent with the request (for POST and PUT requests). diff --git a/docs/website/docs/dlt-ecosystem/verified-sources/sql_database/advanced.md b/docs/website/docs/dlt-ecosystem/verified-sources/sql_database/advanced.md index 2d384e6411..6ff3a267d2 100644 --- a/docs/website/docs/dlt-ecosystem/verified-sources/sql_database/advanced.md +++ b/docs/website/docs/dlt-ecosystem/verified-sources/sql_database/advanced.md @@ -23,6 +23,10 @@ Incremental loading uses a cursor column (e.g., timestamp or auto-incrementing I 1. **Set end_value for backfill**: Set `end_value` if you want to backfill data from a certain range. 1. **Order returned rows**: Set `row_order` to `asc` or `desc` to order returned rows. +:::info Special characters in the cursor column name +If your cursor column name contains special characters (e.g., `$`) you need to escape it when passing it to the `incremental` function. For example, if your cursor column is `example_$column`, you should pass it as `"'example_$column'"` or `'"example_$column"'` to the `incremental` function: `incremental("'example_$column'", initial_value=...)`. +::: + #### Examples 1. **Incremental loading with the resource `sql_table`**. @@ -90,8 +94,8 @@ Depending on the selected backend, some of the types might require additional pr The `reflection_level` argument controls how much information is reflected: -- `reflection_level = "minimal"`: Only column names and nullability are detected. Data types are inferred from the data. -- `reflection_level = "full"`: Column names, nullability, and data types are detected. For decimal types, we always add precision and scale. **This is the default.** +- `reflection_level = "minimal"`: Only column names and nullability are detected. Data types are inferred from the data. **This is the default.** +- `reflection_level = "full"`: Column names, nullability, and data types are detected. For decimal types, we always add precision and scale. - `reflection_level = "full_with_precision"`: Column names, nullability, data types, and precision/scale are detected, also for types like text and binary. Integer sizes are set to bigint and to int for all other types. If the SQL type is unknown or not supported by `dlt`, then, in the pyarrow backend, the column will be skipped, whereas in the other backends the type will be inferred directly from the data irrespective of the `reflection_level` specified. In the latter case, this often means that some types are coerced to strings and `dataclass` based values from sqlalchemy are inferred as `json` (JSON in most destinations). @@ -104,6 +108,8 @@ In that case, you may try **minimal** reflection level where all data types are most of the coercion problems. ::: +### Adapt reflected types to your needs + You can also override the SQL type by passing a `type_adapter_callback` function. This function takes a `SQLAlchemy` data type as input and returns a new type (or `None` to force the column to be inferred from the data) as output. This is useful, for example, when: @@ -134,6 +140,28 @@ source = sql_database( dlt.pipeline("demo").run(source) ``` +### Remove nullability information +`dlt` adds `NULL`/`NOT NULL` information to reflected schemas in **all reflection levels**. There are cases where you do not want this information to be present +ie. +* if you plan to use replication source that will (soft) delete rows. +* if you expect that columns will be dropped from the source table. + +In such cases you can use a table adapter that removes nullability (`dlt` will create nullable tables as a default): + +```py +from dlt.sources.sql_database import sql_table, remove_nullability_adapter + +read_table = sql_table( + table="chat_message", + reflection_level="full_with_precision", + table_adapter_callback=remove_nullability_adapter, +) +print(read_table.compute_table_schema()) +``` + +You can call `remove_nullability_adapter` from your custom table adapter if you need to combine both. + + ## Configuring with TOML or environment variables You can set most of the arguments of `sql_database()` and `sql_table()` directly in the TOML files or as environment variables. `dlt` automatically injects these values into the pipeline script. diff --git a/docs/website/docs/dlt-ecosystem/verified-sources/sql_database/configuration.md b/docs/website/docs/dlt-ecosystem/verified-sources/sql_database/configuration.md index 2ea3788f48..bfeaaf185b 100644 --- a/docs/website/docs/dlt-ecosystem/verified-sources/sql_database/configuration.md +++ b/docs/website/docs/dlt-ecosystem/verified-sources/sql_database/configuration.md @@ -75,7 +75,7 @@ Read more about sources and resources here: [General usage: source](../../../gen ``` - :::note + :::note When using the `sql_database` source, specifying table names directly in the source arguments (e.g., `sql_database(table_names=["family", "clan"])`) ensures that only those tables are reflected and turned into resources. In contrast, if you use `.with_resources("family", "clan")`, the entire schema is reflected first, and resources are generated for all tables before filtering for the specified ones. For large schemas, specifying `table_names` can improve performance. ::: @@ -199,11 +199,12 @@ pipeline = dlt.pipeline( pipeline_name="rfam_cx", destination="postgres", dataset_name="rfam_data_arrow" ) -def _double_as_decimal_adapter(table: sa.Table) -> None: +def _double_as_decimal_adapter(table: sa.Table) -> sa.Table: """Emits decimals instead of floats.""" for column in table.columns.values(): if isinstance(column.type, sa.Float): column.type.asdecimal = False + return table sql_alchemy_source = sql_database( "mysql+pymysql://rfamro@mysql-rfam-public.ebi.ac.uk:4497/Rfam?&binary_prefix=true", @@ -215,7 +216,7 @@ sql_alchemy_source = sql_database( info = pipeline.run(sql_alchemy_source) print(info) ``` -For more information on the `tz` parameter within `backend_kwargs` supported by PyArrow, please refer to the +For more information on the `tz` parameter within `backend_kwargs` supported by PyArrow, please refer to the [official documentation.](https://arrow.apache.org/docs/python/generated/pyarrow.timestamp.html) ### Pandas @@ -242,11 +243,12 @@ pipeline = dlt.pipeline( pipeline_name="rfam_cx", destination="postgres", dataset_name="rfam_data_pandas_2" ) -def _double_as_decimal_adapter(table: sa.Table) -> None: +def _double_as_decimal_adapter(table: sa.Table) -> sa.Table: """Emits decimals instead of floats.""" for column in table.columns.values(): if isinstance(column.type, sa.Float): column.type.asdecimal = True + return table sql_alchemy_source = sql_database( "mysql+pymysql://rfamro@mysql-rfam-public.ebi.ac.uk:4497/Rfam?&binary_prefix=true", diff --git a/docs/website/docs/dlt-ecosystem/verified-sources/sql_database/usage.md b/docs/website/docs/dlt-ecosystem/verified-sources/sql_database/usage.md index bdc440630d..4edff1b658 100644 --- a/docs/website/docs/dlt-ecosystem/verified-sources/sql_database/usage.md +++ b/docs/website/docs/dlt-ecosystem/verified-sources/sql_database/usage.md @@ -31,6 +31,91 @@ source = sql_database( ).with_resources("orders") ``` +## Write custom SQL custom queries +We recommend that you create a SQL VIEW in your source database and extract data from it. In that case `dlt` will infer all column types and read data in +shape you define in a view without any further customization. + +If creating a view is not feasible, you can fully rewrite the automatically generated query with extended version of `query_adapter_callback`: + +```py +import sqlalchemy as sa + +def query_adapter_callback( + query, table, incremental=None, engine=None + ) -> TextClause: + + if incremental and incremental.start_value is not None: + t_query = sa.text( + f"SELECT *, 1 as add_int, 'const' as add_text FROM {table.fullname} WHERE" + f" {incremental.cursor_path} > :start_value" + ).bindparams(**{"start_value": incremental.start_value}) + else: + t_query = sa.text(f"SELECT *, 1 as add_int, 'const' as add_text FROM {table.fullname}") + + return t_query +``` +In the snippet above we do a few interesting things: +1. We create a text query with `sa.text` +2. We change the condition on selecting incremental column from the default `ge` to `greater` (f" {incremental.cursor_path} > :start_value") +3. We add additional computed columns: `1 as add_int, 'const' as add_text`. You can also join other table here. + +We recommend that you explicitly type additional columns that you added with `table_adapter_callback`: + +```py +from sqlalchemy.sql import sqltypes + +def add_new_columns(table) -> None: + required_columns = [ + ("add_int", sqltypes.BigInteger, {"nullable": True}), + ("add_text", sqltypes.Text, {"default": None, "nullable": True}), + ] + for col_name, col_type, col_kwargs in required_columns: + if col_name not in table.c: + table.append_column(sa.Column(col_name, col_type, **col_kwargs)) +``` +Otherwise `dlt` will attempt to infer the types from the extracted data. + +Here's how you call `sql_table` with those adapters: +```py +import dlt +from dlt.sources.sql_database import sql_table + +table = sql_table( + table="chat_channel", + table_adapter_callback=add_new_columns, + query_adapter_callback=query_adapter_callback, + incremental=dlt.sources.incremental("updated_at"), +) +``` + +## Add computed columns and custom incremental clauses + +You can add computed columns to the table definition by converting it into a subquery: +```py +def add_max_timestamp(table): + computed_max_timestamp = sa.sql.type_coerce( + sa.func.greatest(table.c.created_at, table.c.updated_at), + sqltypes.DateTime, + ).label("max_timestamp") + subquery = sa.select(*table.c, computed_max_timestamp).subquery() + return subquery +``` +We add new `max_timestamp` column that is a MAX of `created_at` and `updated_at` columns and then we convert it into a subquery +because we intend to use it for incremental loading which will attach a `WHERE` clause to it. + +```py +import dlt +from dlt.sources.sql_database import sql_table + +read_table = sql_table( + table="chat_message", + table_adapter_callback=add_max_timestamp, + incremental=dlt.sources.incremental("max_timestamp"), +) +``` +`dlt` will use your subquery instead of original `chat_message` table to generate incremental query. Note that you can further +customize subquery with query adapter as in the example above. + ## Transforming the data before load You have direct access to the extracted data through the resource objects (`sql_table()` or `sql_database().with_resource())`), each of which represents a single SQL table. These objects are generators that yield individual rows of the table, which can be modified by using custom Python functions. These functions can be applied to the resource using `add_map`. diff --git a/docs/website/docs/dlt-ecosystem/visualizations/exploring-the-data.md b/docs/website/docs/dlt-ecosystem/visualizations/exploring-the-data.md deleted file mode 100644 index 79ac7b89ad..0000000000 --- a/docs/website/docs/dlt-ecosystem/visualizations/exploring-the-data.md +++ /dev/null @@ -1,128 +0,0 @@ ---- -title: Explore the loaded data -description: How to explore the data that has been loaded -keywords: [exploring, loaded data, data quality] ---- - -# Explore the loaded data - -Once you have run a pipeline locally, you can launch a web app that displays the loaded data. - -To do so, run the [cli command](../../reference/command-line-interface.md#show-tables-and-data-in-the-destination) -below with your pipeline name. The pipeline name is the name of the -Python file where your pipeline is defined and also displayed in your terminal when loading: - -```sh -dlt pipeline {pipeline_name} show -``` - -This will open a streamlit app with: - -- Information about the loads. -- Tables and sample data. -- A SQL client that you can use to run queries. - -## Exploring the data in Python - -You can quickly fetch loaded data from a destination using SQL. The data will be available as a -stream of rows or a data frame. Both methods use the same credentials that you set up for your -pipeline and hide many intricacies of correctly setting up the connection to your destination. - -### Querying the data using the `dlt` SQL client - -Execute any SQL query and get results following the Python -[dbapi](https://peps.python.org/pep-0249/) spec. Below, we fetch data from the customers table: - -```py -pipeline = dlt.pipeline(destination="bigquery", dataset_name="crm") -with pipeline.sql_client() as client: - with client.execute_query( - "SELECT id, name, email FROM customers WHERE id = %s", - 10 - ) as cursor: - # get all data from the cursor as a list of rows - print(cursor.fetchall()) -``` - -In the above, we used `dbapi` parameter placeholders and fetched the data using the `fetchall` method -that reads all the rows from the cursor. - -### Querying data into a data frame - -You can fetch the results of any SQL query as a data frame. If the destination supports that -natively (i.e., BigQuery and DuckDB), `dlt` uses the native method. Thanks to that, reading data -frames may be really fast! The example below reads GitHub reactions data from the `issues` table and -counts reaction types. - -```py -pipeline = dlt.pipeline( - pipeline_name="github_pipeline", - destination="duckdb", - dataset_name="github_reactions", - dev_mode=True -) -with pipeline.sql_client() as client: - with client.execute_query( - 'SELECT "reactions__+1", "reactions__-1", reactions__laugh, reactions__hooray, reactions__rocket FROM issues' - ) as cursor: - # calling `df` on a cursor, returns the data as a pandas DataFrame - reactions = cursor.df() -counts = reactions.sum(0).sort_values(0, ascending=False) -``` - -The `df` method above returns all the data in the cursor as a data frame. You can also fetch data in -chunks by passing the `chunk_size` argument to the `df` method. - -### Access destination native connection - -The native connection to your destination like BigQuery `Client` or DuckDB `DuckDBPyConnection` is -available in case you want to do anything special. Below, we take the native connection to `duckdb` -to get `DuckDBPyRelation` from a query: - -```py -import dlt -import duckdb - -pipeline = dlt.pipeline(destination="duckdb", dataset_name="github_reactions") -with pipeline.sql_client() as client: - conn = client.native_connection - rel = conn.sql('SELECT * FROM issues') - rel.limit(3).show() -``` - -## Data quality dashboards - -After deploying a `dlt` pipeline, you might ask yourself: How can we know if the data is and remains -high quality? - -There are two ways to catch errors: - -1. Tests. -1. People [monitoring.](../../running-in-production/monitoring.md) - -## Tests - -The first time you load data from a pipeline you have built, you will likely want to test it. Plot -the data on time series line charts and look for any interruptions or spikes, which will highlight -any gaps or loading issues. - -### Data usage as monitoring - -Setting up monitoring is a good idea. However, in practice, often by the time you notice something is wrong through reviewing charts, someone in the business has likely already noticed something is wrong. That is, if there is usage of the data, then that usage will act as a sort of monitoring. - -### Plotting main metrics on line charts - -In cases where data is not being used much (e.g., only one marketing analyst is using some data alone), then it is a good idea to have them plot their main metrics on "last 7 days" line charts, so it's visible to them that something may be off when they check their metrics. - -It's important to think about granularity here. A daily line chart, for example, would not catch hourly issues well. Typically, you will want to match the granularity of the time dimension (day/hour/etc.) of the line chart with the things that could go wrong, either in the loading process or in the tracked process. - -If a dashboard is the main product of an analyst, they will generally watch it closely. Therefore, it's probably not necessary for a data engineer to include monitoring in their daily activities in these situations. - -## Tools to create dashboards - -[Metabase](https://www.metabase.com/), [Looker Studio](https://lookerstudio.google.com/u/0/), and [Streamlit](https://streamlit.io/) are some common tools that you might use to set up dashboards to explore data. It's worth noting that while many tools are suitable for exploration, different tools enable your organization to achieve different things. Some organizations use multiple tools for different scopes: - -- Tools like [Metabase](https://www.metabase.com/) are intended for data democratization, where the business user can change the dimension or granularity to answer follow-up questions. -- Tools like [Looker Studio](https://lookerstudio.google.com/u/0/) and [Tableau](https://www.tableau.com/) are intended for minimal interaction curated dashboards that business users can filter and read as-is with limited training. -- Tools like [Streamlit](https://streamlit.io/) enable powerful customizations and the building of complex apps by Python-first developers, but they generally do not support self-service out of the box. - diff --git a/docs/website/docs/general-usage/dataset-access/data-quality-dashboard.md b/docs/website/docs/general-usage/dataset-access/data-quality-dashboard.md new file mode 100644 index 0000000000..01e51993c6 --- /dev/null +++ b/docs/website/docs/general-usage/dataset-access/data-quality-dashboard.md @@ -0,0 +1,39 @@ +--- +title: Ensuring data quality +description: Monitoring and testing data quality +keywords: [destination, schema, data, monitoring, testing, quality] +--- + +# Data quality dashboards + +After deploying a `dlt` pipeline, you might ask yourself: How can we know if the data is and remains high quality? + +There are two ways to catch errors: + +1. Tests. +1. People [monitoring.](../../running-in-production/monitoring.md) + +## Tests + +The first time you load data from a pipeline you have built, you will likely want to test it. Plot the data on time series line charts and look for any interruptions or spikes, which will highlight any gaps or loading issues. + +### Data usage as monitoring + +Setting up monitoring is a good idea. However, in practice, often by the time you notice something is wrong through reviewing charts, someone in the business has likely already noticed something is wrong. That is, if there is usage of the data, then that usage will act as a sort of monitoring. + +### Plotting main metrics on line charts + +In cases where data is not being used much (e.g., only one marketing analyst is using some data alone), then it is a good idea to have them plot their main metrics on "last 7 days" line charts, so it's visible to them that something may be off when they check their metrics. + +It's important to think about granularity here. A daily line chart, for example, would not catch hourly issues well. Typically, you will want to match the granularity of the time dimension (day/hour/etc.) of the line chart with the things that could go wrong, either in the loading process or in the tracked process. + +If a dashboard is the main product of an analyst, they will generally watch it closely. Therefore, it's probably not necessary for a data engineer to include monitoring in their daily activities in these situations. + +## Tools to create dashboards + +[Metabase](https://www.metabase.com/), [Looker Studio](https://lookerstudio.google.com/u/0/), and [Streamlit](https://streamlit.io/) are some common tools that you might use to set up dashboards to explore data. It's worth noting that while many tools are suitable for exploration, different tools enable your organization to achieve different things. Some organizations use multiple tools for different scopes: + +- Tools like [Metabase](https://www.metabase.com/) are intended for data democratization, where the business user can change the dimension or granularity to answer follow-up questions. +- Tools like [Looker Studio](https://lookerstudio.google.com/u/0/) and [Tableau](https://www.tableau.com/) are intended for minimal interaction curated dashboards that business users can filter and read as-is with limited training. +- Tools like [Streamlit](https://streamlit.io/) enable powerful customizations and the building of complex apps by Python-first developers, but they generally do not support self-service out of the box. + diff --git a/docs/website/docs/general-usage/dataset-access/dataset.md b/docs/website/docs/general-usage/dataset-access/dataset.md new file mode 100644 index 0000000000..68635383c5 --- /dev/null +++ b/docs/website/docs/general-usage/dataset-access/dataset.md @@ -0,0 +1,240 @@ +--- +title: Accessing loaded data in Python +description: Conveniently accessing the data loaded to any destination in python +keywords: [destination, schema, data, access, retrieval] +--- + +# Accessing loaded data in Python + +This guide explains how to access and manipulate data that has been loaded into your destination using the `dlt` Python library. After running your pipelines and loading data, you can use the `ReadableDataset` and `ReadableRelation` classes to interact with your data programmatically. + +**Note:** The `ReadableDataset` and `ReadableRelation` objects are **lazy-loading**. They will only query and retrieve data when you perform an action that requires it, such as fetching data into a DataFrame or iterating over the data. This means that simply creating these objects does not load data into memory, making your code more efficient. + +## Quick start example + +Here's a full example of how to retrieve data from a pipeline and load it into a Pandas DataFrame or a PyArrow Table. + +```py +# Assuming you have a Pipeline object named 'pipeline' +# and you have loaded data to a table named 'items' in the destination + +# Step 1: Get the readable dataset from the pipeline +dataset = pipeline._dataset() + +# Step 2: Access a table as a ReadableRelation +items_relation = dataset.items # Or dataset["items"] + +# Step 3: Fetch the entire table as a Pandas DataFrame +df = items_relation.df() + +# Alternatively, fetch as a PyArrow Table +arrow_table = items_relation.arrow() +``` + +## Getting started + +Assuming you have a `Pipeline` object (let's call it `pipeline`), you can obtain a `ReadableDataset` and access your tables as `ReadableRelation` objects. + +### Access the `ReadableDataset` + +```py +# Get the readable dataset from the pipeline +dataset = pipeline._dataset() +``` + +### Access tables as `ReadableRelation` + +You can access tables in your dataset using either attribute access or item access. + +```py +# Using attribute access +items_relation = dataset.items + +# Using item access +items_relation = dataset["items"] +``` + +## Reading data + +Once you have a `ReadableRelation`, you can read data in various formats and sizes. + +### Fetch the entire table + +:::caution +Loading full tables into memory without limiting or iterating over them can consume a large amount of memory and may cause your program to crash if the table is too large. It's recommended to use chunked iteration or apply limits when dealing with large datasets. +::: + +#### As a Pandas DataFrame + +```py +df = items_relation.df() +``` + +#### As a PyArrow Table + +```py +arrow_table = items_relation.arrow() +``` + +#### As a list of Python tuples + +```py +items_list = items_relation.fetchall() +``` + +## Lazy loading behavior + +The `ReadableDataset` and `ReadableRelation` objects are **lazy-loading**. This means that they do not immediately fetch data when you create them. Data is only retrieved when you perform an action that requires it, such as calling `.df()`, `.arrow()`, or iterating over the data. This approach optimizes performance and reduces unnecessary data loading. + +## Iterating over data in chunks + +To handle large datasets efficiently, you can process data in smaller chunks. + +### Iterate as Pandas DataFrames + +```py +for df_chunk in items_relation.iter_df(chunk_size=500): + # Process each DataFrame chunk + pass +``` + +### Iterate as PyArrow Tables + +```py +for arrow_chunk in items_relation.iter_arrow(chunk_size=500): + # Process each PyArrow chunk + pass +``` + +### Iterate as lists of tuples + +```py +for items_chunk in items_relation.iter_fetch(chunk_size=500): + # Process each chunk of tuples + pass +``` + +The methods available on the ReadableRelation correspond to the methods available on the cursor returned by the SQL client. Please refer to the [SQL client](./sql-client.md#supported-methods-on-the-cursor) guide for more information. + +## Modifying queries + +You can refine your data retrieval by limiting the number of records, selecting specific columns, or chaining these operations. + +### Limit the number of records + +```py +# Get the first 50 items as a PyArrow table +arrow_table = items_relation.limit(50).arrow() +``` + +#### Using `head()` to get the first 5 records + +```py +df = items_relation.head().df() +``` + +### Select specific columns + +```py +# Select only 'col1' and 'col2' columns +items_list = items_relation.select("col1", "col2").fetchall() + +# Alternate notation with brackets +items_list = items_relation[["col1", "col2"]].fetchall() + +# Only get one column +items_list = items_relation["col1"].fetchall() + +``` + +### Chain operations + +You can combine `select`, `limit`, and other methods. + +```py +# Select columns and limit the number of records +arrow_table = items_relation.select("col1", "col2").limit(50).arrow() +``` + +## Supported destinations + +All SQL and filesystem destinations supported by `dlt` can utilize this data access interface. For filesystem destinations, `dlt` [uses **DuckDB** under the hood](./sql-client.md#the-filesystem-sql-client) to create views from Parquet or JSONL files dynamically. This allows you to query data stored in files using the same interface as you would with SQL databases. If you plan on accessing data in buckets or the filesystem a lot this way, it is advised to load data as Parquet instead of JSONL, as **DuckDB** is able to only load the parts of the data actually needed for the query to work. + +## Examples + +### Fetch one record as a tuple + +```py +record = items_relation.fetchone() +``` + +### Fetch many records as tuples + +```py +records = items_relation.fetchmany(chunk_size=10) +``` + +### Iterate over data with limit and column selection + +**Note:** When iterating over filesystem tables, the underlying DuckDB may give you a different chunk size depending on the size of the parquet files the table is based on. + +```py + +# Dataframes +for df_chunk in items_relation.select("col1", "col2").limit(100).iter_df(chunk_size=20): + ... + +# Arrow tables +for arrow_table in items_relation.select("col1", "col2").limit(100).iter_arrow(chunk_size=20): + ... + +# Python tuples +for records in items_relation.select("col1", "col2").limit(100).iter_fetch(chunk_size=20): + # Process each modified DataFrame chunk + ... +``` + +## Advanced usage + +### Using custom SQL queries to create `ReadableRelations` + +You can use custom SQL queries directly on the dataset to create a `ReadableRelation`: + +```py +# Join 'items' and 'other_items' tables +custom_relation = dataset("SELECT * FROM items JOIN other_items ON items.id = other_items.id") +arrow_table = custom_relation.arrow() +``` + +:::note +When using custom SQL queries with `dataset()`, methods like `limit` and `select` won't work. Include any filtering or column selection directly in your SQL query. +::: + + +### Loading a `ReadableRelation` into a pipeline table + +Since the `iter_arrow` and `iter_df` methods are generators that iterate over the full `ReadableRelation` in chunks, you can use them as a resource for another (or even the same) `dlt` pipeline: + +```py +# Create a readable relation with a limit of 1m rows +limited_items_relation = dataset.items.limit(1_000_000) + +# Create a new pipeline +other_pipeline = dlt.pipeline(pipeline_name="other_pipeline", destination="duckdb") + +# We can now load these 1m rows into this pipeline in 10k chunks +other_pipeline.run(limited_items_relation.iter_arrow(chunk_size=10_000), table_name="limited_items") +``` + +### Using `ibis` to query the data + +Visit the [Native Ibis integration](./ibis-backend.md) guide to learn more. + +## Important considerations + +- **Memory usage:** Loading full tables into memory without iterating or limiting can consume significant memory, potentially leading to crashes if the dataset is large. Always consider using limits or chunked iteration. + +- **Lazy evaluation:** `ReadableDataset` and `ReadableRelation` objects delay data retrieval until necessary. This design improves performance and resource utilization. + +- **Custom SQL queries:** When executing custom SQL queries, remember that additional methods like `limit()` or `select()` won't modify the query. Include all necessary clauses directly in your SQL statement. + diff --git a/docs/website/docs/general-usage/dataset-access/ibis-backend.md b/docs/website/docs/general-usage/dataset-access/ibis-backend.md new file mode 100644 index 0000000000..8f4b0fb6b6 --- /dev/null +++ b/docs/website/docs/general-usage/dataset-access/ibis-backend.md @@ -0,0 +1,49 @@ +--- +title: Native Ibis integration +description: Accessing your data with native Ibis backends +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/). + +`dlt` provides an easy way to hand over your loaded dataset to an Ibis backend connection. + +:::tip +Not all destinations supported by `dlt` have an equivalent Ibis backend. Natively supported destinations include DuckDB (including Motherduck), Postgres, Redshift, Snowflake, Clickhouse, MSSQL (including Synapse), and BigQuery. The filesystem destination is supported via the [Filesystem SQL client](./sql-client#the-filesystem-sql-client); please install the DuckDB backend for Ibis to use it. Mutating data with Ibis on the filesystem will not result in any actual changes to the persisted files. +::: + +## Prerequisites + +To use the Ibis backend, you will need to have the `ibis-framework` package with the correct Ibis extra installed. The following example will install the DuckDB backend: + +```sh +pip install ibis-framework[duckdb] +``` + +## Get an Ibis connection from your dataset + +`dlt` datasets have a helper method to return an Ibis connection to the destination they live on. The returned object is a native Ibis connection to the destination, which you can use to read and even transform data. Please consult the [Ibis documentation](https://ibis-project.org/docs/backends/) to learn more about what you can do with Ibis. + +```py +# get the dataset from the pipeline +dataset = pipeline._dataset() +dataset_name = pipeline.dataset_name + +# get the native ibis connection from the dataset +ibis_connection = dataset.ibis() + +# list all tables in the dataset +# NOTE: You need to provide the dataset name to ibis, in ibis datasets are named databases +print(ibis_connection.list_tables(database=dataset_name)) + +# get the items table +table = ibis_connection.table("items", database=dataset_name) + +# print the first 10 rows +print(table.limit(10).execute()) + +# Visit the ibis docs to learn more about the available methods +``` + diff --git a/docs/website/docs/general-usage/dataset-access/index.md b/docs/website/docs/general-usage/dataset-access/index.md new file mode 100644 index 0000000000..d51ca09804 --- /dev/null +++ b/docs/website/docs/general-usage/dataset-access/index.md @@ -0,0 +1,19 @@ +--- +title: Accessing loaded data +description: How to access your loaded datasets +keywords: [datasets, data, access] +--- +import DocCardList from '@theme/DocCardList'; + +# Accessing loaded data + +After one or more successful runs of your pipeline, you can inspect or access the loaded data in various ways: + +* We have a simple [`streamlit` app](./streamlit.md) that you can use to view your data locally in your webapp. +* We have a [Python interface](./dataset.md) that allows you to access your data in Python as Python tuples, `arrow` tables, or `pandas` dataframes with a simple dataset object or an SQL interface. You can even run SQL commands on the filesystem destination via `DuckDB` or forward data from any table into another pipeline. +* We have an [`ibis` interface](./ibis-backend.md) that allows you to hand over your loaded data to the powerful [ibis-framework](https://ibis-project.org/) library. +* Lastly, we have some advice for [monitoring and ensuring the quality of your data](./data-quality-dashboard.md). + +# Learn more + + diff --git a/docs/website/docs/general-usage/dataset-access/sql-client.md b/docs/website/docs/general-usage/dataset-access/sql-client.md new file mode 100644 index 0000000000..bcd23cfd32 --- /dev/null +++ b/docs/website/docs/general-usage/dataset-access/sql-client.md @@ -0,0 +1,84 @@ +--- +title: The SQL Client +description: Technical details about the destination sql client +keywords: [data, dataset, sql] +--- + +# The SQL client + +:::note +This page contains technical details about the implementation of the SQL client as well as information on how to use low-level APIs. If you simply want to query your data, it's advised to read the pages in this section on accessing data via `dlt` datasets, Streamlit, or Ibis. +::: + +Most `dlt` destinations use an implementation of the `SqlClientBase` class to connect to the physical destination to which your data is loaded. DDL statements, data insert or update commands, as well as SQL merge and replace queries, are executed via a connection on this client. It also is used for reading data for the [Streamlit app](./streamlit.md) and [data access via `dlt` datasets](./dataset.md). + +All SQL destinations make use of an SQL client; additionally, the filesystem has a special implementation of the SQL client which you can read about [below](#the-filesystem-sql-client). + +## Executing a query on the SQL client + +You can access the SQL client of your destination via the `sql_client` method on your pipeline. The code below shows how to use the SQL client to execute a query. + +```py +pipeline = dlt.pipeline(destination="bigquery", dataset_name="crm") +with pipeline.sql_client() as client: + with client.execute_query( + "SELECT id, name, email FROM customers WHERE id = %s", + 10 + ) as cursor: + # get all data from the cursor as a list of tuples + print(cursor.fetchall()) +``` + +## Retrieving the data in different formats + +The cursor returned by `execute_query` has several methods for retrieving the data. The supported formats are Python tuples, Pandas DataFrame, and Arrow table. + +The code below shows how to retrieve the data as a Pandas DataFrame and then manipulate it in memory: + +```py +pipeline = dlt.pipeline(pipeline_name="my_pipeline", destination="duckdb") +with pipeline.sql_client() as client: + with client.execute_query( + 'SELECT "reactions__+1", "reactions__-1", reactions__laugh, reactions__hooray, reactions__rocket FROM issues' + ) as cursor: + # calling `df` on a cursor, returns the data as a pandas DataFrame + reactions = cursor.df() +counts = reactions.sum(0).sort_values(0, ascending=False) +``` + +## Supported methods on the cursor + +- `fetchall()`: returns all rows as a list of tuples; +- `fetchone()`: returns a single row as a tuple; +- `fetchmany(size=None)`: returns a number of rows as a list of tuples; if no size is provided, all rows are returned; +- `df(chunk_size=None, **kwargs)`: returns the data as a Pandas DataFrame; if `chunk_size` is provided, the data is retrieved in chunks of the given size; +- `arrow(chunk_size=None, **kwargs)`: returns the data as an Arrow table; if `chunk_size` is provided, the data is retrieved in chunks of the given size; +- `iter_fetch(chunk_size: int)`: iterates over the data in chunks of the given size as lists of tuples; +- `iter_df(chunk_size: int)`: iterates over the data in chunks of the given size as Pandas DataFrames; +- `iter_arrow(chunk_size: int)`: iterates over the data in chunks of the given size as Arrow tables. + +:::info +Which retrieval method you should use very much depends on your use case and the destination you are using. Some drivers for our destinations provided by their vendors natively support Arrow or Pandas DataFrames; in these cases, we will use that interface. If they do not, `dlt` will convert lists of tuples into these formats. +::: + +## The filesystem SQL client + +The filesystem destination implements a special but extremely useful version of the SQL client. While during a normal pipeline run, the filesystem does not make use of an SQL client but rather copies the files resulting from a load into the folder or bucket you have specified, it is possible to query this data using SQL via this client. For this to work, `dlt` uses an in-memory `DuckDB` database instance and makes your filesystem tables available as views on this database. For the most part, you can use the filesystem SQL client just like any other SQL client. `dlt` uses sqlglot to discover which tables you are trying to access and, as mentioned above, `DuckDB` to make them queryable. + +The code below shows how to use the filesystem SQL client to query the data: + +```py +pipeline = dlt.pipeline(destination="filesystem", dataset_name="my_dataset") +with pipeline.sql_client() as client: + with client.execute_query("SELECT * FROM my_table") as cursor: + print(cursor.fetchall()) +``` + +A few things to know or keep in mind when using the filesystem SQL client: + +- The SQL database you are actually querying is an in-memory database, so if you do any kind of mutating queries, these will not be persisted to your folder or bucket. +- You must have loaded your data as `JSONL` or `Parquet` files for this SQL client to work. For optimal performance, you should use `Parquet` files, as `DuckDB` is able to only read the bytes needed to execute your query from a folder or bucket in this case. +- Keep in mind that if you do any filtering, sorting, or full table loading with the SQL client, the in-memory `DuckDB` instance will have to download and query a lot of data from your bucket or folder if you have a large table. +- If you are accessing data on a bucket, `dlt` will temporarily store your credentials in `DuckDB` to let it connect to the bucket. +- Some combinations of buckets and table formats may not be fully supported at this time. + diff --git a/docs/website/docs/general-usage/dataset-access/streamlit.md b/docs/website/docs/general-usage/dataset-access/streamlit.md new file mode 100644 index 0000000000..2d76aac660 --- /dev/null +++ b/docs/website/docs/general-usage/dataset-access/streamlit.md @@ -0,0 +1,57 @@ +--- +title: Viewing your data with Streamlit +description: Viewing your data with streamlit +keywords: [data, dataset, streamlit] +--- + +# Viewing your data with Streamlit + +Once you have run a pipeline locally, you can launch a web app that displays the loaded data. For this to work, you will need to have the `streamlit` package installed. + +:::tip +The Streamlit app does not work with all destinations supported by `dlt`. Only destinations that provide a SQL client will work. The filesystem destination has support via the [Filesystem SQL client](./sql-client#the-filesystem-sql-client) and will work in most cases. Vector databases generally are unsupported. +::: + +## Prerequisites + +To install Streamlit, run the following command: + +```sh +pip install streamlit +``` + +## Launching the Streamlit app + +You can use the `show` [CLI command](../../reference/command-line-interface.md#show-tables-and-data-in-the-destination) +with your pipeline name: + +```sh +dlt pipeline {pipeline_name} show +``` + +Use the pipeline name you defined in your Python code with the `pipeline_name` argument. If you are unsure, you can use the `dlt pipeline --list` command to list all pipelines. + +## Credentials + +`dlt` will look for `secrets.toml` and `config.toml` in the `.dlt` folder. + +If `secrets.toml` are not found, it will use +`secrets.toml` from `.streamlit` folder. + +If you run locally, maintain your usual `.dlt` folder. + +When running on streamlit cloud, paste the content of `dlt` +`secrets.toml` into the `streamlit` secrets. + +## Inspecting your data + +You can now inspect the schema and your data. Use the left sidebar to switch between: + +* Exploring your data (default); +* Information about your loads. + + +## Further reading + +If you are running `dlt` in Python interactively or in a notebook, read the [Accessing loaded data in Python](./dataset.md) guide. + diff --git a/docs/website/docs/general-usage/destination-tables.md b/docs/website/docs/general-usage/destination-tables.md index a49ed8578e..bc42618b77 100644 --- a/docs/website/docs/general-usage/destination-tables.md +++ b/docs/website/docs/general-usage/destination-tables.md @@ -35,7 +35,7 @@ will behave similarly and have similar concepts. ::: -Running this pipeline will create a database schema in the destination database (DuckDB) along with a table named `users`. Quick tip: you can use the `show` command of the `dlt pipeline` CLI [to see the tables](../dlt-ecosystem/visualizations/exploring-the-data.md#exploring-the-data) in the destination database. +Running this pipeline will create a database schema in the destination database (DuckDB) along with a table named `users`. Quick tip: you can use the `show` command of the `dlt pipeline` CLI [to see the tables](../general-usage/dataset-access/streamlit) in the destination database. ## Database schema @@ -190,7 +190,7 @@ The `_dlt_loads` table will look like this: The `_dlt_loads` table tracks complete loads and allows chaining transformations on top of them. Many destinations do not support distributed and long-running transactions (e.g., Amazon Redshift). In that case, the user may see the partially loaded data. It is possible to filter such data out: any row with a `load_id` that does not exist in `_dlt_loads` is not yet completed. The same procedure may be used to identify and delete data for packages that never got completed. -For each load, you can test and [alert](../running-in-production/alerting.md) on anomalies (e.g., no data, too much loaded to a table). There are also some useful load stats in the `Load info` tab of the [Streamlit app](../dlt-ecosystem/visualizations/exploring-the-data.md#exploring-the-data) mentioned above. +For each load, you can test and [alert](../running-in-production/alerting.md) on anomalies (e.g., no data, too much loaded to a table). There are also some useful load stats in the `Load info` tab of the [Streamlit app](../general-usage/dataset-access/streamlit) mentioned above. You can add [transformations](../dlt-ecosystem/transformations/) and chain them together using the `status` column. You start the transformation for all the data with a particular `load_id` with a status of 0 and then update it to 1. The next transformation starts with the status of 1 and is then updated to 2. This can be repeated for every additional transformation. diff --git a/docs/website/docs/general-usage/naming-convention.md b/docs/website/docs/general-usage/naming-convention.md index f1766d1797..c10ac3e3d0 100644 --- a/docs/website/docs/general-usage/naming-convention.md +++ b/docs/website/docs/general-usage/naming-convention.md @@ -69,6 +69,45 @@ Note that many destinations are exclusively case-insensitive, of which some pres ### Identifier shortening Identifier shortening happens during normalization. `dlt` takes the maximum length of the identifier from the destination capabilities and will trim the identifiers that are too long. The default shortening behavior generates short deterministic hashes of the source identifiers and places them in the middle of the destination identifier. This (with a high probability) avoids shortened identifier collisions. +### Compound (flattened) identifiers +`dlt` combines several identifiers in order to name nested tables and flattened columns. For example: +```json +{ + "column": + { + "value": 1 + } +} +``` +generates flattened column name `column__value`. Where `__` is a path separator (in **snake case**). Each component in the combined identifier is normalized +separately and shortened as a whole. + +:::note +Combined identifier is also a valid single identifier. Starting from +`dlt` version above 1.4.0 normalization is fully idempotent and normalized +`column__value` will be still `column__value`. +::: + +:::caution +Previously double underscores were contracted into single underscore. That +prevented using data loaded by `dlt` as a data source without identifier modifications. `dlt` maintains backward compatibility for version >1.4.0 as follows: + +* All schemas stored locally or at destination will be migrated to backward compatible mode by setting a flag `use_break_path_on_normalize` ie.: +```yaml +normalizers: + names: dlt.common.normalizers.names.snake_case + use_break_path_on_normalize: true + json: + module: dlt.common.normalizers.json.relational +``` +* Backward compatible behavior may be explicitly enabled by setting +`SCHEMA__USE_BREAK_PATH_ON_NORMALIZE` to `TRUE` or via `config.toml`: +```toml +[schema] +use_break_path_on_normalize=true +``` +::: + ### 🚧 [WIP] Name convention changes are lossy `dlt` does not store the source identifiers in the schema so when the naming convention changes (or we increase the maximum identifier length), it is not able to generate a fully correct set of new identifiers. Instead, it will re-normalize already normalized identifiers. We are currently working to store the full identifier lineage - source identifiers will be stored and mapped to the destination in the schema. diff --git a/docs/website/docs/intro.md b/docs/website/docs/intro.md index 76e3a34736..b20d41c494 100644 --- a/docs/website/docs/intro.md +++ b/docs/website/docs/intro.md @@ -56,7 +56,7 @@ source = rest_api_source({ "token": dlt.secrets["your_api_token"], }, "paginator": { - "type": "json_response", + "type": "json_link", "next_url_path": "paging.next", }, }, diff --git a/docs/website/docs/tutorial/sql-database.md b/docs/website/docs/tutorial/sql-database.md index abaec53ce2..97764768d1 100644 --- a/docs/website/docs/tutorial/sql-database.md +++ b/docs/website/docs/tutorial/sql-database.md @@ -271,6 +271,6 @@ Congratulations on completing the tutorial! You learned how to set up a SQL Data Interested in learning more about dlt? Here are some suggestions: - Learn more about the SQL Database source configuration in [the SQL Database source reference](../dlt-ecosystem/verified-sources/sql_database) -- Learn more about different credential types in [Built-in credentials](../general-usage/credentials/complex_types#built-in-credentials) +- Learn how to extract [single tables and use fast `arrow` and `connectorx` backends](../dlt-ecosystem/verified-sources/sql_database/configuration.md) +- Learn how to [rewrite table schemas and queries](../dlt-ecosystem/verified-sources/sql_database/usage.md) - Learn how to [create a custom source](./load-data-from-an-api.md) in the advanced tutorial - diff --git a/docs/website/docs/walkthroughs/adjust-a-schema.md b/docs/website/docs/walkthroughs/adjust-a-schema.md index d76bdad229..ce547c5b09 100644 --- a/docs/website/docs/walkthroughs/adjust-a-schema.md +++ b/docs/website/docs/walkthroughs/adjust-a-schema.md @@ -113,7 +113,7 @@ players_games: ``` Run the pipeline script again and make sure that the change is visible in the export schema. Then, -[launch the Streamlit app](../dlt-ecosystem/visualizations/exploring-the-data.md) to see the changed data. +[launch the Streamlit app](../general-usage/dataset-access/streamlit) to see the changed data. :::note Do not rename the tables or columns in the YAML file. `dlt` infers those from the data, so the schema will be recreated. diff --git a/docs/website/docs/walkthroughs/deploy-a-pipeline/deploy_snippets/deploy-with-modal-snippets.py b/docs/website/docs/walkthroughs/deploy-a-pipeline/deploy_snippets/deploy-with-modal-snippets.py index 5c50f06a04..8a488159c1 100644 --- a/docs/website/docs/walkthroughs/deploy-a-pipeline/deploy_snippets/deploy-with-modal-snippets.py +++ b/docs/website/docs/walkthroughs/deploy-a-pipeline/deploy_snippets/deploy-with-modal-snippets.py @@ -21,11 +21,7 @@ def test_modal_snippet() -> None: # @@@DLT_SNIPPET_END modal_image # @@@DLT_SNIPPET_START modal_function - @app.function( - volumes={"/data/": vol}, - schedule=modal.Period(days=1), - serialized=True - ) + @app.function(volumes={"/data/": vol}, schedule=modal.Period(days=1), serialized=True) def load_tables() -> None: import dlt import os diff --git a/docs/website/docs/walkthroughs/run-a-pipeline.md b/docs/website/docs/walkthroughs/run-a-pipeline.md index 49abe8675f..3c0e30ccf3 100644 --- a/docs/website/docs/walkthroughs/run-a-pipeline.md +++ b/docs/website/docs/walkthroughs/run-a-pipeline.md @@ -140,7 +140,24 @@ destination, etc. Please refer to [Running in production](../running-in-production/running.md#inspect-and-save-the-load-info-and-trace) for more details. -## 5. Detect and handle problems +## Run dlt in Notebooks + +### Colab +You'll need to install `dlt` like any other dependency: +```sh +!pip install dlt +``` + +You can configure secrets using **Secrets** sidebar. Just create a variable with the name `secrets.toml` and paste +the content of the **toml** file from your `.dlt` folder into it. We support `config.toml` variable as well. + +:::note +`dlt` will not reload the secrets automatically. Please restart your interpreter in Colab options when you add/change +content of the variables above. +::: + + +## Troubleshooting What happens if something goes wrong? In most cases, the `dlt` `run` command raises exceptions. We put a lot of effort into making the exception messages easy to understand. Reading them is the first step diff --git a/docs/website/netlify.toml b/docs/website/netlify.toml index 51cc4ee21f..4cf2a06234 100644 --- a/docs/website/netlify.toml +++ b/docs/website/netlify.toml @@ -42,3 +42,7 @@ to = "/docs/reference/telemetry" [[redirects]] from = "/docs/walkthroughs" to = "/docs/intro" + +[[redirects]] +from = "/docs/visualizations" +to = "/docs/general-usage/dataset-access" \ No newline at end of file diff --git a/docs/website/sidebars.js b/docs/website/sidebars.js index 1edee20c81..274f3e82b3 100644 --- a/docs/website/sidebars.js +++ b/docs/website/sidebars.js @@ -193,7 +193,21 @@ const sidebars = { items: [ 'walkthroughs/create-a-pipeline', 'walkthroughs/run-a-pipeline', - 'dlt-ecosystem/visualizations/exploring-the-data', + { + type: 'category', + label: 'Accessing loaded data', + link: { + type: 'doc', + id: 'general-usage/dataset-access/index', + }, + items: [ + 'general-usage/dataset-access/streamlit', + 'general-usage/dataset-access/dataset', + 'general-usage/dataset-access/ibis-backend', + 'general-usage/dataset-access/sql-client', + 'general-usage/dataset-access/data-quality-dashboard', + ] + }, { type: 'category', label: 'Transform the data', @@ -257,7 +271,6 @@ const sidebars = { 'general-usage/full-loading', ] }, - 'dlt-ecosystem/notebooks' ] }, { diff --git a/mypy.ini b/mypy.ini index 089fde35aa..769e84b13a 100644 --- a/mypy.ini +++ b/mypy.ini @@ -119,4 +119,19 @@ ignore_missing_imports = True ignore_missing_imports = True [mypy-pytz.*] -ignore_missing_imports = True \ No newline at end of file +ignore_missing_imports = True + +[mypy-tornado.*] +ignore_missing_imports = True + +[mypy-adlfs.*] +ignore_missing_imports = True + +[mypy-snowflake.*] +ignore_missing_imports = True + +[mypy-backports.*] +ignore_missing_imports = True + +[mypy-time_machine.*] +ignore_missing_imports = True diff --git a/poetry.lock b/poetry.lock index 00980992fa..732ba0e219 100644 --- a/poetry.lock +++ b/poetry.lock @@ -13,13 +13,13 @@ files = [ [[package]] name = "adlfs" -version = "2024.4.1" +version = "2024.7.0" description = "Access Azure Datalake Gen1 with fsspec and dask" optional = true python-versions = ">=3.8" files = [ - {file = "adlfs-2024.4.1-py3-none-any.whl", hash = "sha256:acea94612ddacaa34ea8c6babcc95b8da6982f930cdade7a86fbd17382403e16"}, - {file = "adlfs-2024.4.1.tar.gz", hash = "sha256:75530a45447f358ae53c5c39c298b8d966dae684be84db899f63b94cd96fc000"}, + {file = "adlfs-2024.7.0-py3-none-any.whl", hash = "sha256:2005c8e124fda3948f2a6abb2dbebb2c936d2d821acaca6afd61932edfa9bc07"}, + {file = "adlfs-2024.7.0.tar.gz", hash = "sha256:106995b91f0eb5e775bcd5957d180d9a14faef3271a063b1f65c66fd5ab05ddf"}, ] [package.dependencies] @@ -772,6 +772,17 @@ files = [ {file = "async_timeout-4.0.3-py3-none-any.whl", hash = "sha256:7405140ff1230c310e51dc27b3145b9092d659ce68ff733fb0cefe3ee42be028"}, ] +[[package]] +name = "atpublic" +version = "5.0" +description = "Keep all y'all's __all__'s in sync" +optional = true +python-versions = ">=3.8" +files = [ + {file = "atpublic-5.0-py3-none-any.whl", hash = "sha256:b651dcd886666b1042d1e38158a22a4f2c267748f4e97fde94bc492a4a28a3f3"}, + {file = "atpublic-5.0.tar.gz", hash = "sha256:d5cb6cbabf00ec1d34e282e8ce7cbc9b74ba4cb732e766c24e2d78d1ad7f723f"}, +] + [[package]] name = "attrs" version = "23.1.0" @@ -1818,6 +1829,9 @@ files = [ [package.dependencies] certifi = "*" lz4 = "*" +numpy = {version = "*", optional = true, markers = "extra == \"numpy\""} +pandas = {version = "*", optional = true, markers = "extra == \"pandas\""} +pyarrow = {version = "*", optional = true, markers = "extra == \"arrow\""} pytz = "*" urllib3 = ">=1.26" zstandard = "*" @@ -3508,6 +3522,31 @@ opentelemetry = ["opentelemetry-api (>=1.1.0)", "opentelemetry-instrumentation ( pandas = ["db-dtypes (>=0.3.0,<2.0.0dev)", "importlib-metadata (>=1.0.0)", "pandas (>=1.1.0)", "pyarrow (>=3.0.0)"] tqdm = ["tqdm (>=4.7.4,<5.0.0dev)"] +[[package]] +name = "google-cloud-bigquery-storage" +version = "2.27.0" +description = "Google Cloud Bigquery Storage API client library" +optional = true +python-versions = ">=3.7" +files = [ + {file = "google_cloud_bigquery_storage-2.27.0-py2.py3-none-any.whl", hash = "sha256:3bfa8f74a61ceaffd3bfe90be5bbef440ad81c1c19ac9075188cccab34bffc2b"}, + {file = "google_cloud_bigquery_storage-2.27.0.tar.gz", hash = "sha256:522faba9a68bea7e9857071c33fafce5ee520b7b175da00489017242ade8ec27"}, +] + +[package.dependencies] +google-api-core = {version = ">=1.34.0,<2.0.dev0 || >=2.11.dev0,<3.0.0dev", extras = ["grpc"]} +google-auth = ">=2.14.1,<3.0.0dev" +proto-plus = [ + {version = ">=1.22.0,<2.0.0dev", markers = "python_version < \"3.11\""}, + {version = ">=1.22.2,<2.0.0dev", markers = "python_version >= \"3.11\""}, +] +protobuf = ">=3.20.2,<4.21.0 || >4.21.0,<4.21.1 || >4.21.1,<4.21.2 || >4.21.2,<4.21.3 || >4.21.3,<4.21.4 || >4.21.4,<4.21.5 || >4.21.5,<6.0.0dev" + +[package.extras] +fastavro = ["fastavro (>=0.21.2)"] +pandas = ["importlib-metadata (>=1.0.0)", "pandas (>=0.21.1)"] +pyarrow = ["pyarrow (>=0.15.0)"] + [[package]] name = "google-cloud-core" version = "2.3.3" @@ -4363,6 +4402,67 @@ files = [ {file = "hyperframe-6.0.1.tar.gz", hash = "sha256:ae510046231dc8e9ecb1a6586f63d2347bf4c8905914aa84ba585ae85f28a914"}, ] +[[package]] +name = "ibis-framework" +version = "10.0.0.dev256" +description = "The portable Python dataframe library" +optional = true +python-versions = "<4.0,>=3.10" +files = [ + {file = "ibis_framework-10.0.0.dev256-py3-none-any.whl", hash = "sha256:d6f21278e6fd78920bbe986df2c871921142635cc4f7d5d2048cae26e307a3df"}, + {file = "ibis_framework-10.0.0.dev256.tar.gz", hash = "sha256:e9f97d8177fd88f4a3578be20519c1da79a6a7ffac678b46b790bfde67405930"}, +] + +[package.dependencies] +atpublic = ">=2.3,<6" +clickhouse-connect = {version = ">=0.5.23,<1", extras = ["arrow", "numpy", "pandas"], optional = true, markers = "extra == \"clickhouse\""} +db-dtypes = {version = ">=0.3,<2", optional = true, markers = "extra == \"bigquery\""} +duckdb = {version = ">=0.10,<1.2", optional = true, markers = "extra == \"duckdb\""} +google-cloud-bigquery = {version = ">=3,<4", optional = true, markers = "extra == \"bigquery\""} +google-cloud-bigquery-storage = {version = ">=2,<3", optional = true, markers = "extra == \"bigquery\""} +numpy = {version = ">=1.23.2,<3", optional = true, markers = "extra == \"bigquery\" or extra == \"clickhouse\" or extra == \"databricks\" or extra == \"datafusion\" or extra == \"druid\" or extra == \"duckdb\" or extra == \"exasol\" or extra == \"flink\" or extra == \"impala\" or extra == \"mssql\" or extra == \"mysql\" or extra == \"oracle\" or extra == \"polars\" or extra == \"postgres\" or extra == \"pyspark\" or extra == \"snowflake\" or extra == \"sqlite\" or extra == \"risingwave\" or extra == \"trino\""} +packaging = {version = ">=21.3,<25", optional = true, markers = "extra == \"duckdb\" or extra == \"oracle\" or extra == \"polars\" or extra == \"pyspark\""} +pandas = {version = ">=1.5.3,<3", optional = true, markers = "extra == \"bigquery\" or extra == \"clickhouse\" or extra == \"databricks\" or extra == \"datafusion\" or extra == \"druid\" or extra == \"duckdb\" or extra == \"exasol\" or extra == \"flink\" or extra == \"impala\" or extra == \"mssql\" or extra == \"mysql\" or extra == \"oracle\" or extra == \"polars\" or extra == \"postgres\" or extra == \"pyspark\" or extra == \"snowflake\" or extra == \"sqlite\" or extra == \"risingwave\" or extra == \"trino\""} +parsy = ">=2,<3" +psycopg2 = {version = ">=2.8.4,<3", optional = true, markers = "extra == \"postgres\" or extra == \"risingwave\""} +pyarrow = {version = ">=10.0.1,<19", optional = true, markers = "extra == \"bigquery\" or extra == \"clickhouse\" or extra == \"databricks\" or extra == \"datafusion\" or extra == \"druid\" or extra == \"duckdb\" or extra == \"exasol\" or extra == \"flink\" or extra == \"impala\" or extra == \"mssql\" or extra == \"mysql\" or extra == \"oracle\" or extra == \"polars\" or extra == \"postgres\" or extra == \"pyspark\" or extra == \"snowflake\" or extra == \"sqlite\" or extra == \"risingwave\" or extra == \"trino\""} +pyarrow-hotfix = {version = ">=0.4,<1", optional = true, markers = "extra == \"bigquery\" or extra == \"clickhouse\" or extra == \"databricks\" or extra == \"datafusion\" or extra == \"druid\" or extra == \"duckdb\" or extra == \"exasol\" or extra == \"flink\" or extra == \"impala\" or extra == \"mssql\" or extra == \"mysql\" or extra == \"oracle\" or extra == \"polars\" or extra == \"postgres\" or extra == \"pyspark\" or extra == \"snowflake\" or extra == \"sqlite\" or extra == \"risingwave\" or extra == \"trino\""} +pydata-google-auth = {version = ">=1.4.0,<2", optional = true, markers = "extra == \"bigquery\""} +pyodbc = {version = ">=4.0.39,<6", optional = true, markers = "extra == \"mssql\""} +python-dateutil = ">=2.8.2,<3" +pytz = ">=2022.7" +rich = {version = ">=12.4.4,<14", optional = true, markers = "extra == \"bigquery\" or extra == \"clickhouse\" or extra == \"databricks\" or extra == \"datafusion\" or extra == \"druid\" or extra == \"duckdb\" or extra == \"exasol\" or extra == \"flink\" or extra == \"impala\" or extra == \"mssql\" or extra == \"mysql\" or extra == \"oracle\" or extra == \"polars\" or extra == \"postgres\" or extra == \"pyspark\" or extra == \"snowflake\" or extra == \"sqlite\" or extra == \"risingwave\" or extra == \"trino\""} +snowflake-connector-python = {version = ">=3.0.2,<3.3.0b1 || >3.3.0b1,<4", optional = true, markers = "extra == \"snowflake\""} +sqlglot = ">=23.4,<25.30" +toolz = ">=0.11,<2" +typing-extensions = ">=4.3.0,<5" + +[package.extras] +bigquery = ["db-dtypes (>=0.3,<2)", "google-cloud-bigquery (>=3,<4)", "google-cloud-bigquery-storage (>=2,<3)", "numpy (>=1.23.2,<3)", "pandas (>=1.5.3,<3)", "pyarrow (>=10.0.1,<19)", "pyarrow-hotfix (>=0.4,<1)", "pydata-google-auth (>=1.4.0,<2)", "rich (>=12.4.4,<14)"] +clickhouse = ["clickhouse-connect[arrow,numpy,pandas] (>=0.5.23,<1)", "numpy (>=1.23.2,<3)", "pandas (>=1.5.3,<3)", "pyarrow (>=10.0.1,<19)", "pyarrow-hotfix (>=0.4,<1)", "rich (>=12.4.4,<14)"] +databricks = ["databricks-sql-connector-core (>=4,<5)", "numpy (>=1.23.2,<3)", "pandas (>=1.5.3,<3)", "pyarrow (>=10.0.1,<19)", "pyarrow-hotfix (>=0.4,<1)", "rich (>=12.4.4,<14)"] +datafusion = ["datafusion (>=0.6,<43)", "numpy (>=1.23.2,<3)", "pandas (>=1.5.3,<3)", "pyarrow (>=10.0.1,<19)", "pyarrow-hotfix (>=0.4,<1)", "rich (>=12.4.4,<14)"] +decompiler = ["black (>=22.1.0,<25)"] +deltalake = ["deltalake (>=0.9.0,<1)"] +druid = ["numpy (>=1.23.2,<3)", "pandas (>=1.5.3,<3)", "pyarrow (>=10.0.1,<19)", "pyarrow-hotfix (>=0.4,<1)", "pydruid (>=0.6.7,<1)", "rich (>=12.4.4,<14)"] +duckdb = ["duckdb (>=0.10,<1.2)", "numpy (>=1.23.2,<3)", "packaging (>=21.3,<25)", "pandas (>=1.5.3,<3)", "pyarrow (>=10.0.1,<19)", "pyarrow-hotfix (>=0.4,<1)", "rich (>=12.4.4,<14)"] +examples = ["pins[gcs] (>=0.8.3,<1)"] +exasol = ["numpy (>=1.23.2,<3)", "pandas (>=1.5.3,<3)", "pyarrow (>=10.0.1,<19)", "pyarrow-hotfix (>=0.4,<1)", "pyexasol[pandas] (>=0.25.2,<1)", "rich (>=12.4.4,<14)"] +flink = ["numpy (>=1.23.2,<3)", "pandas (>=1.5.3,<3)", "pyarrow (>=10.0.1,<19)", "pyarrow-hotfix (>=0.4,<1)", "rich (>=12.4.4,<14)"] +geospatial = ["geoarrow-types (>=0.2,<1)", "geopandas (>=0.6,<2)", "pyproj (>=3.3.0,<4)", "shapely (>=2,<3)"] +impala = ["impyla (>=0.17,<1)", "numpy (>=1.23.2,<3)", "pandas (>=1.5.3,<3)", "pyarrow (>=10.0.1,<19)", "pyarrow-hotfix (>=0.4,<1)", "rich (>=12.4.4,<14)"] +mssql = ["numpy (>=1.23.2,<3)", "pandas (>=1.5.3,<3)", "pyarrow (>=10.0.1,<19)", "pyarrow-hotfix (>=0.4,<1)", "pyodbc (>=4.0.39,<6)", "rich (>=12.4.4,<14)"] +mysql = ["mysqlclient (>=2.2.4,<3)", "numpy (>=1.23.2,<3)", "pandas (>=1.5.3,<3)", "pyarrow (>=10.0.1,<19)", "pyarrow-hotfix (>=0.4,<1)", "rich (>=12.4.4,<14)"] +oracle = ["numpy (>=1.23.2,<3)", "oracledb (>=1.3.1,<3)", "packaging (>=21.3,<25)", "pandas (>=1.5.3,<3)", "pyarrow (>=10.0.1,<19)", "pyarrow-hotfix (>=0.4,<1)", "rich (>=12.4.4,<14)"] +polars = ["numpy (>=1.23.2,<3)", "packaging (>=21.3,<25)", "pandas (>=1.5.3,<3)", "polars (>=1,<2)", "pyarrow (>=10.0.1,<19)", "pyarrow-hotfix (>=0.4,<1)", "rich (>=12.4.4,<14)"] +postgres = ["numpy (>=1.23.2,<3)", "pandas (>=1.5.3,<3)", "psycopg2 (>=2.8.4,<3)", "pyarrow (>=10.0.1,<19)", "pyarrow-hotfix (>=0.4,<1)", "rich (>=12.4.4,<14)"] +pyspark = ["numpy (>=1.23.2,<3)", "packaging (>=21.3,<25)", "pandas (>=1.5.3,<3)", "pyarrow (>=10.0.1,<19)", "pyarrow-hotfix (>=0.4,<1)", "pyspark (>=3.3.3,<4)", "rich (>=12.4.4,<14)"] +risingwave = ["numpy (>=1.23.2,<3)", "pandas (>=1.5.3,<3)", "psycopg2 (>=2.8.4,<3)", "pyarrow (>=10.0.1,<19)", "pyarrow-hotfix (>=0.4,<1)", "rich (>=12.4.4,<14)"] +snowflake = ["numpy (>=1.23.2,<3)", "pandas (>=1.5.3,<3)", "pyarrow (>=10.0.1,<19)", "pyarrow-hotfix (>=0.4,<1)", "rich (>=12.4.4,<14)", "snowflake-connector-python (>=3.0.2,!=3.3.0b1,<4)"] +sqlite = ["numpy (>=1.23.2,<3)", "pandas (>=1.5.3,<3)", "pyarrow (>=10.0.1,<19)", "pyarrow-hotfix (>=0.4,<1)", "regex (>=2021.7.6)", "rich (>=12.4.4,<14)"] +trino = ["numpy (>=1.23.2,<3)", "pandas (>=1.5.3,<3)", "pyarrow (>=10.0.1,<19)", "pyarrow-hotfix (>=0.4,<1)", "rich (>=12.4.4,<14)", "trino (>=0.321,<1)"] +visualization = ["graphviz (>=0.16,<1)"] + [[package]] name = "idna" version = "3.4" @@ -6441,6 +6541,17 @@ files = [ [package.dependencies] future = "*" +[[package]] +name = "parsy" +version = "2.1" +description = "Easy-to-use parser combinators, for parsing in pure Python" +optional = true +python-versions = ">=3.7" +files = [ + {file = "parsy-2.1-py3-none-any.whl", hash = "sha256:8f18e7b11985e7802e7e3ecbd8291c6ca243d29820b1186e4c84605db4efffa0"}, + {file = "parsy-2.1.tar.gz", hash = "sha256:fd5dd18d7b0b61f8275ee88665f430a20c02cf5a82d88557f35330530186d7ac"}, +] + [[package]] name = "pathspec" version = "0.11.2" @@ -6865,6 +6976,24 @@ files = [ [package.extras] test = ["enum34", "ipaddress", "mock", "pywin32", "wmi"] +[[package]] +name = "psycopg2" +version = "2.9.10" +description = "psycopg2 - Python-PostgreSQL Database Adapter" +optional = true +python-versions = ">=3.8" +files = [ + {file = "psycopg2-2.9.10-cp310-cp310-win32.whl", hash = "sha256:5df2b672140f95adb453af93a7d669d7a7bf0a56bcd26f1502329166f4a61716"}, + {file = "psycopg2-2.9.10-cp310-cp310-win_amd64.whl", hash = "sha256:c6f7b8561225f9e711a9c47087388a97fdc948211c10a4bccbf0ba68ab7b3b5a"}, + {file = "psycopg2-2.9.10-cp311-cp311-win32.whl", hash = "sha256:47c4f9875125344f4c2b870e41b6aad585901318068acd01de93f3677a6522c2"}, + {file = "psycopg2-2.9.10-cp311-cp311-win_amd64.whl", hash = "sha256:0435034157049f6846e95103bd8f5a668788dd913a7c30162ca9503fdf542cb4"}, + {file = "psycopg2-2.9.10-cp312-cp312-win32.whl", hash = "sha256:65a63d7ab0e067e2cdb3cf266de39663203d38d6a8ed97f5ca0cb315c73fe067"}, + {file = "psycopg2-2.9.10-cp312-cp312-win_amd64.whl", hash = "sha256:4a579d6243da40a7b3182e0430493dbd55950c493d8c68f4eec0b302f6bbf20e"}, + {file = "psycopg2-2.9.10-cp39-cp39-win32.whl", hash = "sha256:9d5b3b94b79a844a986d029eee38998232451119ad653aea42bb9220a8c5066b"}, + {file = "psycopg2-2.9.10-cp39-cp39-win_amd64.whl", hash = "sha256:88138c8dedcbfa96408023ea2b0c369eda40fe5d75002c0964c78f46f11fa442"}, + {file = "psycopg2-2.9.10.tar.gz", hash = "sha256:12ec0b40b0273f95296233e8750441339298e6a572f7039da5b260e3c8b60e11"}, +] + [[package]] name = "psycopg2-binary" version = "2.9.7" @@ -7010,6 +7139,17 @@ numpy = ">=1.16.6" [package.extras] test = ["cffi", "hypothesis", "pandas", "pytest", "pytz"] +[[package]] +name = "pyarrow-hotfix" +version = "0.6" +description = "" +optional = true +python-versions = ">=3.5" +files = [ + {file = "pyarrow_hotfix-0.6-py3-none-any.whl", hash = "sha256:dcc9ae2d220dff0083be6a9aa8e0cdee5182ad358d4931fce825c545e5c89178"}, + {file = "pyarrow_hotfix-0.6.tar.gz", hash = "sha256:79d3e030f7ff890d408a100ac16d6f00b14d44a502d7897cd9fc3e3a534e9945"}, +] + [[package]] name = "pyasn1" version = "0.5.0" @@ -7212,6 +7352,22 @@ files = [ [package.dependencies] typing-extensions = ">=4.6.0,<4.7.0 || >4.7.0" +[[package]] +name = "pydata-google-auth" +version = "1.9.0" +description = "PyData helpers for authenticating to Google APIs" +optional = true +python-versions = ">=3.9" +files = [ + {file = "pydata-google-auth-1.9.0.tar.gz", hash = "sha256:2f546e88f007dfdb050087556eb46d6008e351386a7b368096797fae5df374f2"}, + {file = "pydata_google_auth-1.9.0-py2.py3-none-any.whl", hash = "sha256:e17a44ce8de5b48883667357c03595b85d80938bf1fb714d65bfac9a9f9c8add"}, +] + +[package.dependencies] +google-auth = ">=1.25.0,<3.0dev" +google-auth-oauthlib = ">=0.4.0" +setuptools = "*" + [[package]] name = "pydoc-markdown" version = "4.8.2" @@ -8565,6 +8721,64 @@ docs = ["furo", "jaraco.packaging (>=9.3)", "jaraco.tidelift (>=1.4)", "pygments testing = ["build[virtualenv]", "filelock (>=3.4.0)", "flake8-2020", "ini2toml[lite] (>=0.9)", "jaraco.develop (>=7.21)", "jaraco.envs (>=2.2)", "jaraco.path (>=3.2.0)", "pip (>=19.1)", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=2.2)", "pytest-mypy (>=0.9.1)", "pytest-perf", "pytest-ruff", "pytest-timeout", "pytest-xdist", "tomli-w (>=1.0.0)", "virtualenv (>=13.0.0)", "wheel"] testing-integration = ["build[virtualenv]", "filelock (>=3.4.0)", "jaraco.envs (>=2.2)", "jaraco.path (>=3.2.0)", "pytest", "pytest-enabler", "pytest-xdist", "tomli", "virtualenv (>=13.0.0)", "wheel"] +[[package]] +name = "shapely" +version = "2.0.6" +description = "Manipulation and analysis of geometric objects" +optional = false +python-versions = ">=3.7" +files = [ + {file = "shapely-2.0.6-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:29a34e068da2d321e926b5073539fd2a1d4429a2c656bd63f0bd4c8f5b236d0b"}, + {file = "shapely-2.0.6-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:e1c84c3f53144febf6af909d6b581bc05e8785d57e27f35ebaa5c1ab9baba13b"}, + {file = "shapely-2.0.6-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2ad2fae12dca8d2b727fa12b007e46fbc522148a584f5d6546c539f3464dccde"}, + {file = "shapely-2.0.6-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b3304883bd82d44be1b27a9d17f1167fda8c7f5a02a897958d86c59ec69b705e"}, + {file = "shapely-2.0.6-cp310-cp310-win32.whl", hash = "sha256:3ec3a0eab496b5e04633a39fa3d5eb5454628228201fb24903d38174ee34565e"}, + {file = "shapely-2.0.6-cp310-cp310-win_amd64.whl", hash = "sha256:28f87cdf5308a514763a5c38de295544cb27429cfa655d50ed8431a4796090c4"}, + {file = "shapely-2.0.6-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:5aeb0f51a9db176da9a30cb2f4329b6fbd1e26d359012bb0ac3d3c7781667a9e"}, + {file = "shapely-2.0.6-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:9a7a78b0d51257a367ee115f4d41ca4d46edbd0dd280f697a8092dd3989867b2"}, + {file = "shapely-2.0.6-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f32c23d2f43d54029f986479f7c1f6e09c6b3a19353a3833c2ffb226fb63a855"}, + {file = "shapely-2.0.6-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b3dc9fb0eb56498912025f5eb352b5126f04801ed0e8bdbd867d21bdbfd7cbd0"}, + {file = "shapely-2.0.6-cp311-cp311-win32.whl", hash = "sha256:d93b7e0e71c9f095e09454bf18dad5ea716fb6ced5df3cb044564a00723f339d"}, + {file = "shapely-2.0.6-cp311-cp311-win_amd64.whl", hash = "sha256:c02eb6bf4cfb9fe6568502e85bb2647921ee49171bcd2d4116c7b3109724ef9b"}, + {file = "shapely-2.0.6-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:cec9193519940e9d1b86a3b4f5af9eb6910197d24af02f247afbfb47bcb3fab0"}, + {file = "shapely-2.0.6-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:83b94a44ab04a90e88be69e7ddcc6f332da7c0a0ebb1156e1c4f568bbec983c3"}, + {file = "shapely-2.0.6-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:537c4b2716d22c92036d00b34aac9d3775e3691f80c7aa517c2c290351f42cd8"}, + {file = "shapely-2.0.6-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:98fea108334be345c283ce74bf064fa00cfdd718048a8af7343c59eb40f59726"}, + {file = "shapely-2.0.6-cp312-cp312-win32.whl", hash = "sha256:42fd4cd4834747e4990227e4cbafb02242c0cffe9ce7ef9971f53ac52d80d55f"}, + {file = "shapely-2.0.6-cp312-cp312-win_amd64.whl", hash = "sha256:665990c84aece05efb68a21b3523a6b2057e84a1afbef426ad287f0796ef8a48"}, + {file = "shapely-2.0.6-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:42805ef90783ce689a4dde2b6b2f261e2c52609226a0438d882e3ced40bb3013"}, + {file = "shapely-2.0.6-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:6d2cb146191a47bd0cee8ff5f90b47547b82b6345c0d02dd8b25b88b68af62d7"}, + {file = "shapely-2.0.6-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e3fdef0a1794a8fe70dc1f514440aa34426cc0ae98d9a1027fb299d45741c381"}, + {file = "shapely-2.0.6-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2c665a0301c645615a107ff7f52adafa2153beab51daf34587170d85e8ba6805"}, + {file = "shapely-2.0.6-cp313-cp313-win32.whl", hash = "sha256:0334bd51828f68cd54b87d80b3e7cee93f249d82ae55a0faf3ea21c9be7b323a"}, + {file = "shapely-2.0.6-cp313-cp313-win_amd64.whl", hash = "sha256:d37d070da9e0e0f0a530a621e17c0b8c3c9d04105655132a87cfff8bd77cc4c2"}, + {file = "shapely-2.0.6-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:fa7468e4f5b92049c0f36d63c3e309f85f2775752e076378e36c6387245c5462"}, + {file = "shapely-2.0.6-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ed5867e598a9e8ac3291da6cc9baa62ca25706eea186117034e8ec0ea4355653"}, + {file = "shapely-2.0.6-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:81d9dfe155f371f78c8d895a7b7f323bb241fb148d848a2bf2244f79213123fe"}, + {file = "shapely-2.0.6-cp37-cp37m-win32.whl", hash = "sha256:fbb7bf02a7542dba55129062570211cfb0defa05386409b3e306c39612e7fbcc"}, + {file = "shapely-2.0.6-cp37-cp37m-win_amd64.whl", hash = "sha256:837d395fac58aa01aa544495b97940995211e3e25f9aaf87bc3ba5b3a8cd1ac7"}, + {file = "shapely-2.0.6-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:c6d88ade96bf02f6bfd667ddd3626913098e243e419a0325ebef2bbd481d1eb6"}, + {file = "shapely-2.0.6-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:8b3b818c4407eaa0b4cb376fd2305e20ff6df757bf1356651589eadc14aab41b"}, + {file = "shapely-2.0.6-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1bbc783529a21f2bd50c79cef90761f72d41c45622b3e57acf78d984c50a5d13"}, + {file = "shapely-2.0.6-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2423f6c0903ebe5df6d32e0066b3d94029aab18425ad4b07bf98c3972a6e25a1"}, + {file = "shapely-2.0.6-cp38-cp38-win32.whl", hash = "sha256:2de00c3bfa80d6750832bde1d9487e302a6dd21d90cb2f210515cefdb616e5f5"}, + {file = "shapely-2.0.6-cp38-cp38-win_amd64.whl", hash = "sha256:3a82d58a1134d5e975f19268710e53bddd9c473743356c90d97ce04b73e101ee"}, + {file = "shapely-2.0.6-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:392f66f458a0a2c706254f473290418236e52aa4c9b476a072539d63a2460595"}, + {file = "shapely-2.0.6-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:eba5bae271d523c938274c61658ebc34de6c4b33fdf43ef7e938b5776388c1be"}, + {file = "shapely-2.0.6-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:7060566bc4888b0c8ed14b5d57df8a0ead5c28f9b69fb6bed4476df31c51b0af"}, + {file = "shapely-2.0.6-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b02154b3e9d076a29a8513dffcb80f047a5ea63c897c0cd3d3679f29363cf7e5"}, + {file = "shapely-2.0.6-cp39-cp39-win32.whl", hash = "sha256:44246d30124a4f1a638a7d5419149959532b99dfa25b54393512e6acc9c211ac"}, + {file = "shapely-2.0.6-cp39-cp39-win_amd64.whl", hash = "sha256:2b542d7f1dbb89192d3512c52b679c822ba916f93479fa5d4fc2fe4fa0b3c9e8"}, + {file = "shapely-2.0.6.tar.gz", hash = "sha256:997f6159b1484059ec239cacaa53467fd8b5564dabe186cd84ac2944663b0bf6"}, +] + +[package.dependencies] +numpy = ">=1.14,<3" + +[package.extras] +docs = ["matplotlib", "numpydoc (==1.1.*)", "sphinx", "sphinx-book-theme", "sphinx-remove-toctrees"] +test = ["pytest", "pytest-cov"] + [[package]] name = "shellingham" version = "1.5.4" @@ -9332,6 +9546,17 @@ files = [ {file = "tomlkit-0.12.1.tar.gz", hash = "sha256:38e1ff8edb991273ec9f6181244a6a391ac30e9f5098e7535640ea6be97a7c86"}, ] +[[package]] +name = "toolz" +version = "1.0.0" +description = "List processing tools and functional utilities" +optional = true +python-versions = ">=3.8" +files = [ + {file = "toolz-1.0.0-py3-none-any.whl", hash = "sha256:292c8f1c4e7516bf9086f8850935c799a874039c8bcf959d47b600e4c44a6236"}, + {file = "toolz-1.0.0.tar.gz", hash = "sha256:2c86e3d9a04798ac556793bced838816296a2f085017664e4995cb40a1047a02"}, +] + [[package]] name = "tqdm" version = "4.66.1" @@ -10278,6 +10503,7 @@ lancedb = ["lancedb", "pyarrow", "tantivy"] motherduck = ["duckdb", "pyarrow"] mssql = ["pyodbc"] parquet = ["pyarrow"] +postgis = ["psycopg2-binary", "psycopg2cffi"] postgres = ["psycopg2-binary", "psycopg2cffi"] qdrant = ["qdrant-client"] redshift = ["psycopg2-binary", "psycopg2cffi"] @@ -10292,4 +10518,4 @@ weaviate = ["weaviate-client"] [metadata] lock-version = "2.0" python-versions = ">=3.8.1,<3.13" -content-hash = "f932b4e28b8d08489ca58a67e6844006a21dc73d453c3eae8469cd57760bb891" +content-hash = "c0607d05ab37a1a6addf3ae7264bf5972cb6ce6e46df1dcdc2da3cff72e5008e" diff --git a/pyproject.toml b/pyproject.toml index 6eb162ac55..7377b03fde 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -1,6 +1,6 @@ [tool.poetry] name = "dlt" -version = "1.4.0" +version = "1.4.1" description = "dlt is an open-source python-first scalable data loading library that does not require any backend to run." authors = ["dltHub Inc. "] maintainers = [ "Marcin Rudolf ", "Adrian Brudaru ", "Anton Burnashev ", "David Scharf " ] @@ -40,7 +40,7 @@ click = ">=7.1" requirements-parser = ">=0.5.0" setuptools = ">=65.6.0" humanize = ">=4.4.0" -astunparse = ">=1.6.3" +astunparse = { "version" = ">=1.6.3", "python" = "<3.9"} gitpython = ">=3.1.29" pytz = ">=2022.6" giturlparse = ">=0.10.0" @@ -89,7 +89,6 @@ alembic = {version = ">1.10.0", optional = true} paramiko = {version = ">=3.3.0", optional = true} sqlglot = {version = ">=20.0.0", optional = true} db-dtypes = { version = ">=1.2.0", optional = true } -aiohttp = { version = ">=3.9", optional = true } [tool.poetry.extras] gcp = ["grpcio", "google-cloud-bigquery", "db-dtypes", "gcsfs"] @@ -119,6 +118,7 @@ lancedb = ["lancedb", "pyarrow", "tantivy"] deltalake = ["deltalake", "pyarrow"] sql_database = ["sqlalchemy"] sqlalchemy = ["sqlalchemy", "alembic"] +postgis = ["psycopg2-binary", "psycopg2cffi"] [tool.poetry.scripts] dlt = "dlt.cli._dlt:_main" @@ -167,6 +167,8 @@ pytest-mock = "^3.14.0" types-regex = "^2024.5.15.20240519" flake8-print = "^5.0.0" mimesis = "^7.0.0" +ibis-framework = { version = ">=9.0.0", markers = "python_version >= '3.10'", optional = true, extras = ["duckdb", "postgres", "bigquery", "snowflake", "mssql", "clickhouse"]} +shapely = ">=2.0.6" [tool.poetry.group.sources] optional = true diff --git a/tests/common/cases/schemas/eth/ethereum_schema_v11.yml b/tests/common/cases/schemas/eth/ethereum_schema_v11.yml new file mode 100644 index 0000000000..fd6717c614 --- /dev/null +++ b/tests/common/cases/schemas/eth/ethereum_schema_v11.yml @@ -0,0 +1,394 @@ +version: 18 +version_hash: XfkJ8E1tZzG/Sb3lfEZrEVshTMKdB7JpOP2HA7eS6EI= +engine_version: 11 +name: ethereum +tables: + _dlt_loads: + columns: + load_id: + nullable: false + data_type: text + schema_name: + nullable: true + data_type: text + status: + nullable: false + data_type: bigint + inserted_at: + nullable: false + data_type: timestamp + schema_version_hash: + nullable: true + data_type: text + write_disposition: skip + description: Created by DLT. Tracks completed loads + schema_contract: {} + resource: _dlt_loads + _dlt_version: + columns: + version: + nullable: false + data_type: bigint + engine_version: + nullable: false + data_type: bigint + inserted_at: + nullable: false + data_type: timestamp + schema_name: + nullable: false + data_type: text + version_hash: + nullable: false + data_type: text + schema: + nullable: false + data_type: text + write_disposition: skip + description: Created by DLT. Tracks schema updates + schema_contract: {} + resource: _dlt_version + blocks: + description: Ethereum blocks + x-annotation: this will be preserved on save + write_disposition: append + filters: + includes: [] + excludes: [] + columns: + _dlt_load_id: + nullable: false + description: load id coming from the extractor + data_type: text + _dlt_id: + nullable: false + unique: true + data_type: text + row_key: true + number: + nullable: false + primary_key: true + data_type: bigint + parent_hash: + nullable: true + data_type: text + hash: + nullable: false + cluster: true + unique: true + data_type: text + base_fee_per_gas: + nullable: false + data_type: wei + difficulty: + nullable: false + data_type: wei + extra_data: + nullable: true + data_type: text + gas_limit: + nullable: false + data_type: bigint + gas_used: + nullable: false + data_type: bigint + logs_bloom: + nullable: true + data_type: binary + miner: + nullable: true + data_type: text + mix_hash: + nullable: true + data_type: text + nonce: + nullable: true + data_type: text + receipts_root: + nullable: true + data_type: text + sha3_uncles: + nullable: true + data_type: text + size: + nullable: true + data_type: bigint + state_root: + nullable: false + data_type: text + timestamp: + nullable: false + unique: true + sort: true + data_type: timestamp + total_difficulty: + nullable: true + data_type: wei + transactions_root: + nullable: false + data_type: text + schema_contract: {} + resource: blocks + x-normalizer: + seen-data: true + blocks__transactions: + columns: + _dlt_id: + nullable: false + unique: true + data_type: text + row_key: true + block_number: + nullable: false + primary_key: true + data_type: bigint + merge_key: true + transaction_index: + nullable: false + primary_key: true + data_type: bigint + hash: + nullable: false + unique: true + data_type: text + block_hash: + nullable: false + cluster: true + data_type: text + block_timestamp: + nullable: false + sort: true + data_type: timestamp + chain_id: + nullable: true + data_type: text + from: + nullable: true + data_type: text + gas: + nullable: true + data_type: bigint + gas_price: + nullable: true + data_type: bigint + input: + nullable: true + data_type: text + max_fee_per_gas: + nullable: true + data_type: wei + max_priority_fee_per_gas: + nullable: true + data_type: wei + nonce: + nullable: true + data_type: bigint + r: + nullable: true + data_type: text + s: + nullable: true + data_type: text + status: + nullable: true + data_type: bigint + to: + nullable: true + data_type: text + type: + nullable: true + data_type: text + v: + nullable: true + data_type: bigint + value: + nullable: false + data_type: wei + eth_value: + nullable: true + data_type: decimal + x-normalizer: + seen-data: true + write_disposition: append + resource: blocks__transactions + blocks__transactions__logs: + columns: + _dlt_id: + nullable: false + unique: true + data_type: text + row_key: true + address: + nullable: false + data_type: text + block_timestamp: + nullable: false + sort: true + data_type: timestamp + block_hash: + nullable: false + cluster: true + data_type: text + block_number: + nullable: false + primary_key: true + merge_key: true + data_type: bigint + transaction_index: + nullable: false + primary_key: true + merge_key: true + data_type: bigint + log_index: + nullable: false + primary_key: true + data_type: bigint + data: + nullable: true + data_type: text + removed: + nullable: true + data_type: bool + transaction_hash: + nullable: false + data_type: text + x-normalizer: + seen-data: true + write_disposition: append + resource: blocks__transactions__logs + blocks__transactions__logs__topics: + parent: blocks__transactions__logs + columns: + _dlt_parent_id: + nullable: false + data_type: text + parent_key: true + _dlt_list_idx: + nullable: false + data_type: bigint + _dlt_id: + nullable: false + unique: true + data_type: text + row_key: true + _dlt_root_id: + nullable: false + root_key: true + data_type: text + value: + nullable: true + data_type: text + x-normalizer: + seen-data: true + blocks__transactions__access_list: + parent: blocks__transactions + columns: + _dlt_parent_id: + nullable: false + data_type: text + parent_key: true + _dlt_list_idx: + nullable: false + data_type: bigint + _dlt_id: + nullable: false + unique: true + data_type: text + row_key: true + _dlt_root_id: + nullable: false + root_key: true + data_type: text + address: + nullable: true + data_type: text + x-normalizer: + seen-data: true + blocks__transactions__access_list__storage_keys: + parent: blocks__transactions__access_list + columns: + _dlt_parent_id: + nullable: false + data_type: text + parent_key: true + _dlt_list_idx: + nullable: false + data_type: bigint + _dlt_id: + nullable: false + unique: true + data_type: text + row_key: true + _dlt_root_id: + nullable: false + root_key: true + data_type: text + value: + nullable: true + data_type: text + x-normalizer: + seen-data: true + blocks__uncles: + parent: blocks + columns: + _dlt_parent_id: + nullable: false + data_type: text + parent_key: true + _dlt_list_idx: + nullable: false + data_type: bigint + _dlt_id: + nullable: false + unique: true + data_type: text + row_key: true + _dlt_root_id: + nullable: false + root_key: true + data_type: text + value: + nullable: true + data_type: text + x-normalizer: + seen-data: true +settings: + default_hints: + not_null: + - re:^_dlt_id$ + - _dlt_root_id + - _dlt_parent_id + - _dlt_list_idx + unique: + - _dlt_id + cluster: + - block_hash + partition: + - block_timestamp + root_key: + - _dlt_root_id + row_key: + - _dlt_id + parent_key: + - _dlt_parent_id + preferred_types: + timestamp: timestamp + block_timestamp: timestamp + schema_contract: {} +normalizers: + names: dlt.common.normalizers.names.snake_case + json: + module: dlt.common.normalizers.json.relational + config: + propagation: + root: + _dlt_id: _dlt_root_id + tables: + blocks: + timestamp: block_timestamp + hash: block_hash +previous_hashes: +- oHfYGTI2GHOxuzwVz6+yvMilXUvHYhxrxkanC2T6MAI= +- C5An8WClbavalXDdNSqXbdI7Swqh/mTWMcwWKCF//EE= +- yjMtV4Zv0IJlfR5DPMwuXxGg8BRhy7E79L26XAHWEGE= + diff --git a/tests/common/cases/schemas/github/issues.schema.json b/tests/common/cases/schemas/github/issues.schema.json index 4c4f5425ae..5a1b0c6f84 100644 --- a/tests/common/cases/schemas/github/issues.schema.json +++ b/tests/common/cases/schemas/github/issues.schema.json @@ -1,1322 +1,1100 @@ { - "version": 2, - "version_hash": "IeCTkq8epwbjSy1O3jdkPPUkTPCt4hLj6RYo8uZ02JI=", - "engine_version": 5, - "name": "event", - "tables": { - "_dlt_version": { - "name": "_dlt_version", - "columns": { - "version": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "version", - "data_type": "bigint", - "nullable": false - }, - "engine_version": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "engine_version", - "data_type": "bigint", - "nullable": false - }, - "inserted_at": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "inserted_at", - "data_type": "timestamp", - "nullable": false - }, - "schema_name": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "schema_name", - "data_type": "text", - "nullable": false - }, - "version_hash": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "version_hash", - "data_type": "text", - "nullable": false - }, - "schema": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "schema", - "data_type": "text", - "nullable": false - } + "version": 3, + "version_hash": "o6olKmaCAQVWDWR4eT4aZ1V/RiH+003516xq7Zrva+Q=", + "engine_version": 11, + "name": "event", + "tables": { + "_dlt_version": { + "columns": { + "version": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "bigint", + "nullable": false + }, + "engine_version": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "bigint", + "nullable": false + }, + "inserted_at": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "timestamp", + "nullable": false + }, + "schema_name": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": false + }, + "version_hash": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": false + }, + "schema": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": false + } + }, + "write_disposition": "skip", + "description": "Created by DLT. Tracks schema updates", + "schema_contract": {}, + "resource": "_dlt_version" }, - "write_disposition": "skip", - "description": "Created by DLT. Tracks schema updates" - }, - "_dlt_loads": { - "name": "_dlt_loads", - "columns": { - "load_id": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "load_id", - "data_type": "text", - "nullable": false - }, - "schema_name": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "schema_name", - "data_type": "text", - "nullable": true - }, - "status": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "status", - "data_type": "bigint", - "nullable": false - }, - "inserted_at": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "inserted_at", - "data_type": "timestamp", - "nullable": false - } + "_dlt_loads": { + "columns": { + "load_id": { + "data_type": "text", + "nullable": false + }, + "schema_name": { + "data_type": "text", + "nullable": true + }, + "status": { + "data_type": "bigint", + "nullable": false + }, + "inserted_at": { + "data_type": "timestamp", + "nullable": false + }, + "schema_version_hash": { + "data_type": "text", + "nullable": true + } + }, + "write_disposition": "skip", + "resource": "_dlt_loads", + "description": "Created by DLT. Tracks completed loads", + "schema_contract": {} }, - "write_disposition": "skip", - "description": "Created by DLT. Tracks completed loads" - }, - "issues": { - "name": "issues", - "columns": { - "url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "url", - "data_type": "text", - "nullable": true - }, - "repository_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "repository_url", - "data_type": "text", - "nullable": true - }, - "labels_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "labels_url", - "data_type": "text", - "nullable": true - }, - "comments_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "comments_url", - "data_type": "text", - "nullable": true - }, - "events_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "events_url", - "data_type": "text", - "nullable": true - }, - "html_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "html_url", - "data_type": "text", - "nullable": true - }, - "id": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "id", - "data_type": "bigint", - "nullable": true - }, - "node_id": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "node_id", - "data_type": "text", - "nullable": true - }, - "number": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "number", - "data_type": "bigint", - "nullable": true - }, - "title": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "title", - "data_type": "text", - "nullable": true - }, - "user__login": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "user__login", - "data_type": "text", - "nullable": true - }, - "user__id": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "user__id", - "data_type": "bigint", - "nullable": true - }, - "user__node_id": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "user__node_id", - "data_type": "text", - "nullable": true - }, - "user__avatar_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "user__avatar_url", - "data_type": "text", - "nullable": true - }, - "user__gravatar_id": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "user__gravatar_id", - "data_type": "text", - "nullable": true - }, - "user__url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "user__url", - "data_type": "text", - "nullable": true - }, - "user__html_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "user__html_url", - "data_type": "text", - "nullable": true - }, - "user__followers_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "user__followers_url", - "data_type": "text", - "nullable": true - }, - "user__following_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "user__following_url", - "data_type": "text", - "nullable": true - }, - "user__gists_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "user__gists_url", - "data_type": "text", - "nullable": true - }, - "user__starred_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "user__starred_url", - "data_type": "text", - "nullable": true - }, - "user__subscriptions_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "user__subscriptions_url", - "data_type": "text", - "nullable": true - }, - "user__organizations_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "user__organizations_url", - "data_type": "text", - "nullable": true - }, - "user__repos_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "user__repos_url", - "data_type": "text", - "nullable": true - }, - "user__events_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "user__events_url", - "data_type": "text", - "nullable": true - }, - "user__received_events_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "user__received_events_url", - "data_type": "text", - "nullable": true - }, - "user__type": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "user__type", - "data_type": "text", - "nullable": true - }, - "user__site_admin": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "user__site_admin", - "data_type": "bool", - "nullable": true - }, - "state": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "state", - "data_type": "text", - "nullable": true - }, - "locked": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "locked", - "data_type": "bool", - "nullable": true - }, - "assignee__login": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "assignee__login", - "data_type": "text", - "nullable": true - }, - "assignee__id": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "assignee__id", - "data_type": "bigint", - "nullable": true - }, - "assignee__node_id": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "assignee__node_id", - "data_type": "text", - "nullable": true - }, - "assignee__avatar_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "assignee__avatar_url", - "data_type": "text", - "nullable": true - }, - "assignee__gravatar_id": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "assignee__gravatar_id", - "data_type": "text", - "nullable": true - }, - "assignee__url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "assignee__url", - "data_type": "text", - "nullable": true - }, - "assignee__html_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "assignee__html_url", - "data_type": "text", - "nullable": true - }, - "assignee__followers_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "assignee__followers_url", - "data_type": "text", - "nullable": true - }, - "assignee__following_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "assignee__following_url", - "data_type": "text", - "nullable": true - }, - "assignee__gists_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "assignee__gists_url", - "data_type": "text", - "nullable": true - }, - "assignee__starred_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "assignee__starred_url", - "data_type": "text", - "nullable": true - }, - "assignee__subscriptions_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "assignee__subscriptions_url", - "data_type": "text", - "nullable": true - }, - "assignee__organizations_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "assignee__organizations_url", - "data_type": "text", - "nullable": true - }, - "assignee__repos_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "assignee__repos_url", - "data_type": "text", - "nullable": true - }, - "assignee__events_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "assignee__events_url", - "data_type": "text", - "nullable": true - }, - "assignee__received_events_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "assignee__received_events_url", - "data_type": "text", - "nullable": true - }, - "assignee__type": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "assignee__type", - "data_type": "text", - "nullable": true - }, - "assignee__site_admin": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "assignee__site_admin", - "data_type": "bool", - "nullable": true - }, - "comments": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "comments", - "data_type": "bigint", - "nullable": true - }, - "created_at": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "created_at", - "data_type": "timestamp", - "nullable": true - }, - "updated_at": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "updated_at", - "data_type": "timestamp", - "nullable": true - }, - "closed_at": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "closed_at", - "data_type": "timestamp", - "nullable": true - }, - "author_association": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "author_association", - "data_type": "text", - "nullable": true - }, - "body": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "body", - "data_type": "text", - "nullable": true - }, - "reactions__url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "reactions__url", - "data_type": "text", - "nullable": true - }, - "reactions__total_count": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "reactions__total_count", - "data_type": "bigint", - "nullable": true - }, - "reactions___1": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "reactions___1", - "data_type": "bigint", - "nullable": true - }, - "reactions__laugh": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "reactions__laugh", - "data_type": "bigint", - "nullable": true - }, - "reactions__hooray": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "reactions__hooray", - "data_type": "bigint", - "nullable": true - }, - "reactions__confused": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "reactions__confused", - "data_type": "bigint", - "nullable": true - }, - "reactions__heart": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "reactions__heart", - "data_type": "bigint", - "nullable": true - }, - "reactions__rocket": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "reactions__rocket", - "data_type": "bigint", - "nullable": true - }, - "reactions__eyes": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "reactions__eyes", - "data_type": "bigint", - "nullable": true - }, - "timeline_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "timeline_url", - "data_type": "text", - "nullable": true - }, - "state_reason": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "state_reason", - "data_type": "text", - "nullable": true - }, - "_dlt_load_id": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "_dlt_load_id", - "data_type": "text", - "nullable": false - }, - "_dlt_id": { - "partition": false, - "cluster": false, - "unique": true, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "_dlt_id", - "data_type": "text", - "nullable": false - }, - "draft": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "draft", - "data_type": "bool", - "nullable": true - }, - "pull_request__url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "pull_request__url", - "data_type": "text", - "nullable": true - }, - "pull_request__html_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "pull_request__html_url", - "data_type": "text", - "nullable": true - }, - "pull_request__diff_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "pull_request__diff_url", - "data_type": "text", - "nullable": true - }, - "pull_request__patch_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "pull_request__patch_url", - "data_type": "text", - "nullable": true - }, - "pull_request__merged_at": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "pull_request__merged_at", - "data_type": "timestamp", - "nullable": true - } + "issues": { + "columns": { + "url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "repository_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "labels_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "comments_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "events_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "html_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "id": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "bigint", + "nullable": true + }, + "node_id": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "number": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "bigint", + "nullable": true + }, + "title": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "user__login": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "user__id": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "bigint", + "nullable": true + }, + "user__node_id": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "user__avatar_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "user__gravatar_id": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "user__url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "user__html_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "user__followers_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "user__following_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "user__gists_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "user__starred_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "user__subscriptions_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "user__organizations_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "user__repos_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "user__events_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "user__received_events_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "user__type": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "user__site_admin": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "bool", + "nullable": true + }, + "state": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "locked": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "bool", + "nullable": true + }, + "assignee__login": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "assignee__id": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "bigint", + "nullable": true + }, + "assignee__node_id": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "assignee__avatar_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "assignee__gravatar_id": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "assignee__url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "assignee__html_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "assignee__followers_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "assignee__following_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "assignee__gists_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "assignee__starred_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "assignee__subscriptions_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "assignee__organizations_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "assignee__repos_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "assignee__events_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "assignee__received_events_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "assignee__type": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "assignee__site_admin": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "bool", + "nullable": true + }, + "comments": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "bigint", + "nullable": true + }, + "created_at": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "timestamp", + "nullable": true + }, + "updated_at": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "timestamp", + "nullable": true + }, + "closed_at": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "timestamp", + "nullable": true + }, + "author_association": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "body": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "reactions__url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "reactions__total_count": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "bigint", + "nullable": true + }, + "reactions___1": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "bigint", + "nullable": true + }, + "reactions__laugh": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "bigint", + "nullable": true + }, + "reactions__hooray": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "bigint", + "nullable": true + }, + "reactions__confused": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "bigint", + "nullable": true + }, + "reactions__heart": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "bigint", + "nullable": true + }, + "reactions__rocket": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "bigint", + "nullable": true + }, + "reactions__eyes": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "bigint", + "nullable": true + }, + "timeline_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "state_reason": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "_dlt_load_id": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": false + }, + "_dlt_id": { + "partition": false, + "cluster": false, + "unique": true, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": false, + "row_key": true + }, + "draft": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "bool", + "nullable": true + }, + "pull_request__url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "pull_request__html_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "pull_request__diff_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "pull_request__patch_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "pull_request__merged_at": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "timestamp", + "nullable": true + } + }, + "write_disposition": "append", + "schema_contract": {}, + "x-normalizer": { + "seen-data": true + }, + "resource": "issues" }, - "write_disposition": "append" - }, - "issues__labels": { - "name": "issues__labels", - "columns": { - "id": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "id", - "data_type": "bigint", - "nullable": true - }, - "node_id": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "node_id", - "data_type": "text", - "nullable": true - }, - "url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "url", - "data_type": "text", - "nullable": true - }, - "name": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "name", - "data_type": "text", - "nullable": true - }, - "color": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "color", - "data_type": "text", - "nullable": true - }, - "default": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "default", - "data_type": "bool", - "nullable": true - }, - "description": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "description", - "data_type": "text", - "nullable": true - }, - "_dlt_parent_id": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": true, - "name": "_dlt_parent_id", - "data_type": "text", - "nullable": false - }, - "_dlt_list_idx": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "_dlt_list_idx", - "data_type": "bigint", - "nullable": false - }, - "_dlt_id": { - "partition": false, - "cluster": false, - "unique": true, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "_dlt_id", - "data_type": "text", - "nullable": false + "issues__labels": { + "columns": { + "id": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "bigint", + "nullable": true + }, + "node_id": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "name": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "color": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "default": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "bool", + "nullable": true + }, + "description": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "_dlt_parent_id": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": false, + "parent_key": true + }, + "_dlt_list_idx": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "bigint", + "nullable": false + }, + "_dlt_id": { + "partition": false, + "cluster": false, + "unique": true, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": false, + "row_key": true + } + }, + "parent": "issues", + "x-normalizer": { + "seen-data": true } }, - "parent": "issues" - }, - "issues__assignees": { - "name": "issues__assignees", - "columns": { - "login": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "login", - "data_type": "text", - "nullable": true - }, - "id": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "id", - "data_type": "bigint", - "nullable": true - }, - "node_id": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "node_id", - "data_type": "text", - "nullable": true - }, - "avatar_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "avatar_url", - "data_type": "text", - "nullable": true - }, - "gravatar_id": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "gravatar_id", - "data_type": "text", - "nullable": true - }, - "url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "url", - "data_type": "text", - "nullable": true - }, - "html_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "html_url", - "data_type": "text", - "nullable": true - }, - "followers_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "followers_url", - "data_type": "text", - "nullable": true - }, - "following_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "following_url", - "data_type": "text", - "nullable": true - }, - "gists_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "gists_url", - "data_type": "text", - "nullable": true - }, - "starred_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "starred_url", - "data_type": "text", - "nullable": true - }, - "subscriptions_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "subscriptions_url", - "data_type": "text", - "nullable": true - }, - "organizations_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "organizations_url", - "data_type": "text", - "nullable": true - }, - "repos_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "repos_url", - "data_type": "text", - "nullable": true - }, - "events_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "events_url", - "data_type": "text", - "nullable": true - }, - "received_events_url": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "received_events_url", - "data_type": "text", - "nullable": true - }, - "type": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "type", - "data_type": "text", - "nullable": true - }, - "site_admin": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "site_admin", - "data_type": "bool", - "nullable": true - }, - "_dlt_parent_id": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": true, - "name": "_dlt_parent_id", - "data_type": "text", - "nullable": false - }, - "_dlt_list_idx": { - "partition": false, - "cluster": false, - "unique": false, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "_dlt_list_idx", - "data_type": "bigint", - "nullable": false - }, - "_dlt_id": { - "partition": false, - "cluster": false, - "unique": true, - "sort": false, - "primary_key": false, - "foreign_key": false, - "name": "_dlt_id", - "data_type": "text", - "nullable": false + "issues__assignees": { + "columns": { + "login": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "id": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "bigint", + "nullable": true + }, + "node_id": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "avatar_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "gravatar_id": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "html_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "followers_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "following_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "gists_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "starred_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "subscriptions_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "organizations_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "repos_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "events_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "received_events_url": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "type": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": true + }, + "site_admin": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "bool", + "nullable": true + }, + "_dlt_parent_id": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": false, + "parent_key": true + }, + "_dlt_list_idx": { + "partition": false, + "cluster": false, + "unique": false, + "sort": false, + "primary_key": false, + "data_type": "bigint", + "nullable": false + }, + "_dlt_id": { + "partition": false, + "cluster": false, + "unique": true, + "sort": false, + "primary_key": false, + "data_type": "text", + "nullable": false, + "row_key": true + } + }, + "parent": "issues", + "x-normalizer": { + "seen-data": true } - }, - "parent": "issues" - } - }, - "settings": { - "detections": [ - "timestamp", - "iso_timestamp", - "iso_date" - ], - "default_hints": { - "not_null": [ - "_dlt_id", - "_dlt_root_id", - "_dlt_parent_id", - "_dlt_list_idx", - "_dlt_load_id" - ], - "foreign_key": [ - "_dlt_parent_id" + } + }, + "settings": { + "detections": [ + "timestamp", + "iso_timestamp", + "iso_date" ], - "unique": [ - "_dlt_id" - ] - } - }, - "normalizers": { - "names": "dlt.common.normalizers.names.snake_case", - "json": { - "module": "dlt.common.normalizers.json.relational" - } + "default_hints": { + "not_null": [ + "_dlt_id", + "_dlt_root_id", + "_dlt_parent_id", + "_dlt_list_idx", + "_dlt_load_id" + ], + "unique": [ + "_dlt_id" + ], + "row_key": [ + "_dlt_id" + ], + "parent_key": [ + "_dlt_parent_id" + ] + }, + "schema_contract": {} + }, + "normalizers": { + "names": "dlt.common.normalizers.names.snake_case", + "json": { + "module": "dlt.common.normalizers.json.relational" + } + }, + "previous_hashes": [ + "IeCTkq8epwbjSy1O3jdkPPUkTPCt4hLj6RYo8uZ02JI=" + ] } -} diff --git a/tests/common/normalizers/test_json_relational.py b/tests/common/normalizers/test_json_relational.py index 1553cea04f..35bc80add2 100644 --- a/tests/common/normalizers/test_json_relational.py +++ b/tests/common/normalizers/test_json_relational.py @@ -6,14 +6,12 @@ from dlt.common.utils import digest128, uniq_id from dlt.common.schema import Schema from dlt.common.schema.utils import new_table - +from dlt.common.normalizers.utils import DLT_ID_LENGTH_BYTES from dlt.common.normalizers.json.relational import ( RelationalNormalizerConfigPropagation, DataItemNormalizer as RelationalNormalizer, - DLT_ID_LENGTH_BYTES, ) - -# _flatten, _get_child_row_hash, _normalize_row, normalize_data_item, +from dlt.common.normalizers.json import helpers as normalize_helpers from tests.utils import create_schema_with_name @@ -420,7 +418,7 @@ def test_list_in_list() -> None: schema.update_table(path_table) assert "zen__webpath" in schema.tables # clear cache with json paths - schema.data_item_normalizer._is_nested_type.cache_clear() # type: ignore[attr-defined] + normalize_helpers.is_nested_type.cache_clear() rows = list(schema.normalize_data_item(chats, "1762162.1212", "zen")) # both lists are json types now @@ -890,7 +888,7 @@ def test_caching_perf(norm: RelationalNormalizer) -> None: table["x-normalizer"] = {} start = time() for _ in range(100000): - norm._is_nested_type(norm.schema, "test", "field", 0) + normalize_helpers.is_nested_type(norm.schema, "test", "field", 0) # norm._get_table_nesting_level(norm.schema, "test") print(f"{time() - start}") diff --git a/tests/common/normalizers/test_naming_snake_case.py b/tests/common/normalizers/test_naming_snake_case.py index ee4f43e7f0..e03de65696 100644 --- a/tests/common/normalizers/test_naming_snake_case.py +++ b/tests/common/normalizers/test_naming_snake_case.py @@ -50,6 +50,14 @@ def test_normalize_path(naming_unlimited: NamingConvention) -> None: assert naming_unlimited.normalize_path("Small Love Potion") == "small_love_potion" assert naming_unlimited.normalize_path("Small Love Potion") == "small_love_potion" + # paths with non normalized underscores + # NOTE: empty idents created during break path are removed so underscores are contracted + assert ( + naming_unlimited.normalize_path("Small___Love____Potion_____x") + == "small___love__potion___x" + ) + assert naming_unlimited.normalize_path("small___love__potion___x") == "small___love__potion___x" + def test_normalize_non_alpha_single_underscore() -> None: assert SnakeCaseNamingConvention.RE_NON_ALPHANUMERIC.sub("_", "-=!*") == "_" diff --git a/tests/common/schema/test_import_normalizers.py b/tests/common/schema/test_import_normalizers.py index a1e3d775f0..d444259946 100644 --- a/tests/common/schema/test_import_normalizers.py +++ b/tests/common/schema/test_import_normalizers.py @@ -16,7 +16,7 @@ ) from dlt.common.schema.normalizers import ( DEFAULT_NAMING_NAMESPACE, - explicit_normalizers, + configured_normalizers, import_normalizers, naming_from_reference, serialize_reference, @@ -26,25 +26,25 @@ def test_explicit_normalizers() -> None: - config = explicit_normalizers() + config = configured_normalizers() assert config["names"] is None assert config["json"] is None # pass explicit - config = explicit_normalizers("direct", {"module": "custom"}) + config = configured_normalizers("direct", {"module": "custom"}) assert config["names"] == "direct" assert config["json"] == {"module": "custom"} # pass modules and types, make sure normalizer config is serialized - config = explicit_normalizers(direct) + config = configured_normalizers(direct) assert config["names"] == f"{DEFAULT_NAMING_NAMESPACE}.direct.NamingConvention" - config = explicit_normalizers(direct.NamingConvention) + config = configured_normalizers(direct.NamingConvention) assert config["names"] == f"{DEFAULT_NAMING_NAMESPACE}.direct.NamingConvention" # use environ os.environ["SCHEMA__NAMING"] = "direct" os.environ["SCHEMA__JSON_NORMALIZER"] = '{"module": "custom"}' - config = explicit_normalizers() + config = configured_normalizers() assert config["names"] == "direct" assert config["json"] == {"module": "custom"} @@ -54,7 +54,7 @@ def test_explicit_normalizers_caps_ignored() -> None: destination_caps = DestinationCapabilitiesContext.generic_capabilities() destination_caps.naming_convention = "direct" with Container().injectable_context(destination_caps): - config = explicit_normalizers() + config = configured_normalizers() assert config["names"] is None @@ -121,7 +121,7 @@ def test_naming_from_reference() -> None: def test_import_normalizers() -> None: - config, naming, json_normalizer = import_normalizers(explicit_normalizers()) + config, naming, json_normalizer = import_normalizers(configured_normalizers()) assert isinstance(naming, snake_case.NamingConvention) # no maximum length: we do not know the destination capabilities assert naming.max_length is None @@ -133,7 +133,7 @@ def test_import_normalizers() -> None: os.environ["SCHEMA__JSON_NORMALIZER"] = ( '{"module": "tests.common.normalizers.custom_normalizers"}' ) - config, naming, json_normalizer = import_normalizers(explicit_normalizers()) + config, naming, json_normalizer = import_normalizers(configured_normalizers()) assert config["names"] == "direct" assert config["json"] == {"module": "tests.common.normalizers.custom_normalizers"} assert isinstance(naming, direct.NamingConvention) @@ -142,7 +142,7 @@ def test_import_normalizers() -> None: def test_import_normalizers_with_defaults() -> None: - explicit = explicit_normalizers() + explicit = configured_normalizers() default_: TNormalizersConfig = { "names": "dlt.destinations.impl.weaviate.naming", "json": {"module": "tests.common.normalizers.custom_normalizers"}, @@ -170,7 +170,7 @@ def test_config_sections(sections: str) -> None: os.environ[f"{sections}SCHEMA__JSON_NORMALIZER"] = ( '{"module": "tests.common.normalizers.custom_normalizers"}' ) - config, _, _ = import_normalizers(explicit_normalizers(schema_name="test_schema")) + config, _, _ = import_normalizers(configured_normalizers(schema_name="test_schema")) assert config["names"] == "direct" assert config["json"] == {"module": "tests.common.normalizers.custom_normalizers"} @@ -181,11 +181,11 @@ def test_import_normalizers_with_caps() -> None: destination_caps.naming_convention = "direct" destination_caps.max_identifier_length = 127 with Container().injectable_context(destination_caps): - _, naming, _ = import_normalizers(explicit_normalizers()) + _, naming, _ = import_normalizers(configured_normalizers()) assert isinstance(naming, direct.NamingConvention) assert naming.max_length == 127 - _, naming, _ = import_normalizers(explicit_normalizers(snake_case)) + _, naming, _ = import_normalizers(configured_normalizers(snake_case)) assert isinstance(naming, snake_case.NamingConvention) assert naming.max_length == 127 @@ -196,23 +196,23 @@ def test_import_normalizers_with_caps() -> None: } destination_caps.max_table_nesting = 0 with Container().injectable_context(destination_caps): - config, _, relational = import_normalizers(explicit_normalizers()) + config, _, relational = import_normalizers(configured_normalizers()) assert config["json"]["config"]["max_nesting"] == 0 assert relational is RelationalNormalizer # wrong normalizer - config, _, relational = import_normalizers(explicit_normalizers(), default_) + config, _, relational = import_normalizers(configured_normalizers(), default_) assert "config" not in config["json"] def test_import_invalid_naming_module() -> None: with pytest.raises(UnknownNamingModule) as py_ex: - import_normalizers(explicit_normalizers("unknown")) + import_normalizers(configured_normalizers("unknown")) assert py_ex.value.naming_module == "unknown" with pytest.raises(UnknownNamingModule) as py_ex: - import_normalizers(explicit_normalizers("dlt.common.tests")) + import_normalizers(configured_normalizers("dlt.common.tests")) assert py_ex.value.naming_module == "dlt.common.tests" with pytest.raises(InvalidNamingType) as py_ex2: - import_normalizers(explicit_normalizers("dlt.pipeline.helpers")) + import_normalizers(configured_normalizers("dlt.pipeline.helpers")) assert py_ex2.value.naming_module == "dlt.pipeline" assert py_ex2.value.naming_class == "helpers" diff --git a/tests/common/schema/test_normalize_identifiers.py b/tests/common/schema/test_normalize_identifiers.py index f84d857e26..a1cb181525 100644 --- a/tests/common/schema/test_normalize_identifiers.py +++ b/tests/common/schema/test_normalize_identifiers.py @@ -271,12 +271,7 @@ def test_normalize_table_identifiers_table_reference() -> None: def test_update_normalizers() -> None: - schema_dict: TStoredSchema = load_json_case("schemas/github/issues.schema") - schema = Schema.from_dict(schema_dict) # type: ignore[arg-type] - # drop seen data - del schema.tables["issues"]["x-normalizer"] - del schema.tables["issues__labels"]["x-normalizer"] - del schema.tables["issues__assignees"]["x-normalizer"] + schema = make_issues_schema_for_normalizers_update() # save default hints in original form default_hints = schema._settings["default_hints"] @@ -307,8 +302,8 @@ def test_normalize_default_hints(schema_storage_no_import: SchemaStorage) -> Non from dlt.common.destination import DestinationCapabilitiesContext from dlt.common.configuration.container import Container - eth_V9 = load_yml_case("schemas/eth/ethereum_schema_v9") - orig_schema = Schema.from_dict(eth_V9) + eth_V11 = load_yml_case("schemas/eth/ethereum_schema_v11") + orig_schema = Schema.from_dict(eth_V11) # save schema schema_storage_no_import.save_schema(orig_schema) @@ -317,7 +312,7 @@ def test_normalize_default_hints(schema_storage_no_import: SchemaStorage) -> Non ) as caps: assert caps.naming_convention is sql_upper # creating a schema from dict keeps original normalizers - schema = Schema.from_dict(eth_V9) + schema = Schema.from_dict(eth_V11) assert_schema_identifiers_case(schema, str.lower) assert schema._normalizers_config["names"].endswith("snake_case") @@ -350,7 +345,7 @@ def test_normalize_default_hints(schema_storage_no_import: SchemaStorage) -> Non ) norm_schema = Schema.from_dict( - deepcopy(eth_V9), remove_processing_hints=True, bump_version=False + deepcopy(eth_V11), remove_processing_hints=True, bump_version=False ) norm_schema.update_normalizers() assert_schema_identifiers_case(norm_schema, str.upper) @@ -452,3 +447,50 @@ def assert_new_schema_values_custom_normalizers(schema: Schema) -> None: assert schema.naming.break_path("A__B__!C") == ["A", "B", "!C"] row = list(schema.normalize_data_item({"bool": True}, "load_id", "a_table")) assert row[0] == (("a_table", None), {"bool": True}) + + +def test_update_schema_normalizer_props() -> None: + schema = make_issues_schema_for_normalizers_update() + schema_2 = make_issues_schema_for_normalizers_update() + # remove issues table + del schema_2._schema_tables["issues"] + schema_2.update_schema(schema) + + os.environ["SCHEMA__NAMING"] = "tests.common.cases.normalizers.sql_upper" + # apply normalizers + schema_2.update_normalizers() + + # preserve schema_2 str + schema_2_str = schema_2.to_pretty_json() + + # make sure that normalizer props in original schema are preserved + schema._normalizers_config["allow_identifier_change_on_table_with_data"] = True + schema._normalizers_config["use_break_path_on_normalize"] = True + + # set some fake naming convention. during schema update it should not be used + os.environ["SCHEMA__NAMING"] = "tests.common.cases.normalizers.sql_upper_X" + schema.update_schema(schema_2) + assert isinstance(schema.naming, sql_upper.NamingConvention) + assert_schema_identifiers_case(schema, str.upper) + # make sure norm setting still in schema + assert schema._normalizers_config["allow_identifier_change_on_table_with_data"] is True + assert schema._normalizers_config["use_break_path_on_normalize"] is True + # schema 2 not modified during the update + assert schema_2_str == schema_2.to_pretty_json() + + # make sure that explicit settings are passed + schema_2._normalizers_config["allow_identifier_change_on_table_with_data"] = False + schema_2._normalizers_config["use_break_path_on_normalize"] = False + schema.update_schema(schema_2) + assert schema._normalizers_config["allow_identifier_change_on_table_with_data"] is False + assert schema._normalizers_config["use_break_path_on_normalize"] is False + + +def make_issues_schema_for_normalizers_update() -> Schema: + schema_dict: TStoredSchema = load_json_case("schemas/github/issues.schema") + schema = Schema.from_dict(schema_dict) # type: ignore[arg-type] + # drop seen data + del schema.tables["issues"]["x-normalizer"] + del schema.tables["issues__labels"]["x-normalizer"] + del schema.tables["issues__assignees"]["x-normalizer"] + return schema diff --git a/tests/common/schema/test_schema.py b/tests/common/schema/test_schema.py index 7124ca5c80..5cdd42e448 100644 --- a/tests/common/schema/test_schema.py +++ b/tests/common/schema/test_schema.py @@ -570,8 +570,8 @@ def test_update_preferred_types(schema: Schema) -> None: def test_default_table_resource() -> None: """Parent tables without `resource` set default to table name""" - eth_v5 = load_yml_case("schemas/eth/ethereum_schema_v5") - tables = Schema.from_dict(eth_v5).tables + eth_v11 = load_yml_case("schemas/eth/ethereum_schema_v11") + tables = Schema.from_dict(eth_v11).tables assert tables["blocks"]["resource"] == "blocks" assert all([t.get("resource") is None for t in tables.values() if t.get("parent")]) @@ -737,7 +737,7 @@ def assert_new_schema_props_custom_normalizers(schema: Schema) -> None: def assert_is_new_schema(schema: Schema) -> None: assert schema.stored_version is None assert schema.stored_version_hash is None - assert schema.ENGINE_VERSION == 10 + assert schema.ENGINE_VERSION == 11 assert schema._stored_previous_hashes == [] assert schema.is_modified assert schema.is_new @@ -845,9 +845,9 @@ def test_group_tables_by_resource(schema: Schema) -> None: def test_remove_processing_hints() -> None: - eth_V9 = load_yml_case("schemas/eth/ethereum_schema_v9") + eth_V11 = load_yml_case("schemas/eth/ethereum_schema_v11") # here tables contain processing hints - schema = Schema.from_dict(eth_V9) + schema = Schema.from_dict(eth_V11) assert "x-normalizer" in schema.tables["blocks"] # clone with hints removal, note that clone does not bump version @@ -867,16 +867,10 @@ def test_remove_processing_hints() -> None: assert "x-normalizer" not in to_json # load without hints - no_hints = schema.from_dict(eth_V9, remove_processing_hints=True, bump_version=False) + no_hints = schema.from_dict(eth_V11, remove_processing_hints=True, bump_version=False) assert no_hints.stored_version_hash == cloned.stored_version_hash # now load without hints but with version bump cloned._bump_version() - no_hints = schema.from_dict(eth_V9, remove_processing_hints=True) + no_hints = schema.from_dict(eth_V11, remove_processing_hints=True) assert no_hints.stored_version_hash == cloned.stored_version_hash - - -# def test_get_new_table_columns() -> None: -# pytest.fail(reason="must implement!") -# pass -# get_new_table_columns() diff --git a/tests/common/schema/test_versioning.py b/tests/common/schema/test_versioning.py index 39f1ad3211..1577b51115 100644 --- a/tests/common/schema/test_versioning.py +++ b/tests/common/schema/test_versioning.py @@ -86,10 +86,10 @@ def test_infer_column_bumps_version() -> None: def test_preserve_version_on_load() -> None: - eth_v10: TStoredSchema = load_yml_case("schemas/eth/ethereum_schema_v10") - version = eth_v10["version"] - version_hash = eth_v10["version_hash"] - schema = Schema.from_dict(eth_v10) # type: ignore[arg-type] + eth_v11: TStoredSchema = load_yml_case("schemas/eth/ethereum_schema_v11") + version = eth_v11["version"] + version_hash = eth_v11["version_hash"] + schema = Schema.from_dict(eth_v11) # type: ignore[arg-type] # version should not be bumped assert version_hash == schema._stored_version_hash assert version_hash == schema.version_hash @@ -98,8 +98,8 @@ def test_preserve_version_on_load() -> None: @pytest.mark.parametrize("remove_defaults", [True, False]) def test_version_preserve_on_reload(remove_defaults: bool) -> None: - eth_v8: TStoredSchema = load_yml_case("schemas/eth/ethereum_schema_v8") - schema = Schema.from_dict(eth_v8) # type: ignore[arg-type] + eth_v11: TStoredSchema = load_yml_case("schemas/eth/ethereum_schema_v11") + schema = Schema.from_dict(eth_v11) # type: ignore[arg-type] to_save_dict = schema.to_dict(remove_defaults=remove_defaults) assert schema.stored_version == to_save_dict["version"] diff --git a/tests/common/storages/test_schema_storage.py b/tests/common/storages/test_schema_storage.py index 0dcf2930de..2818ea9622 100644 --- a/tests/common/storages/test_schema_storage.py +++ b/tests/common/storages/test_schema_storage.py @@ -3,7 +3,7 @@ import yaml from dlt.common import json -from dlt.common.schema.normalizers import explicit_normalizers +from dlt.common.schema.normalizers import configured_normalizers from dlt.common.schema.schema import Schema from dlt.common.storages.exceptions import ( InStorageSchemaModified, @@ -304,7 +304,7 @@ def test_save_store_schema_over_import_sync(synced_storage: SchemaStorage) -> No def test_save_store_schema(storage: SchemaStorage) -> None: - d_n = explicit_normalizers() + d_n = configured_normalizers() d_n["names"] = "tests.common.normalizers.custom_normalizers" schema = Schema("column_event", normalizers=d_n) assert schema.is_new @@ -357,16 +357,16 @@ def test_save_initial_import_schema(ie_storage: LiveSchemaStorage) -> None: ie_storage.load_schema("ethereum") # save initial import schema where processing hints are removed - eth_V9 = load_yml_case("schemas/eth/ethereum_schema_v9") - schema = Schema.from_dict(eth_V9) + eth_V11 = load_yml_case("schemas/eth/ethereum_schema_v11") + schema = Schema.from_dict(eth_V11) ie_storage.save_import_schema_if_not_exists(schema) # should be available now eth = ie_storage.load_schema("ethereum") assert "x-normalizer" not in eth.tables["blocks"] # won't overwrite initial schema - del eth_V9["tables"]["blocks__uncles"] - schema = Schema.from_dict(eth_V9) + del eth_V11["tables"]["blocks__uncles"] + schema = Schema.from_dict(eth_V11) ie_storage.save_import_schema_if_not_exists(schema) # should be available now eth = ie_storage.load_schema("ethereum") diff --git a/tests/common/storages/utils.py b/tests/common/storages/utils.py index a1334ba1da..5366d8b06f 100644 --- a/tests/common/storages/utils.py +++ b/tests/common/storages/utils.py @@ -218,9 +218,9 @@ def assert_package_info( def prepare_eth_import_folder(storage: SchemaStorage) -> Schema: - eth_V9 = load_yml_case("schemas/eth/ethereum_schema_v9") + eth_V11 = load_yml_case("schemas/eth/ethereum_schema_v11") # remove processing hints before installing as import schema # ethereum schema is a "dirty" schema with processing hints - eth = Schema.from_dict(eth_V9, remove_processing_hints=True) + eth = Schema.from_dict(eth_V11, remove_processing_hints=True) storage._export_schema(eth, storage.config.import_schema_path) return eth diff --git a/tests/common/test_jsonpath.py b/tests/common/test_jsonpath.py new file mode 100644 index 0000000000..c4e9fbc664 --- /dev/null +++ b/tests/common/test_jsonpath.py @@ -0,0 +1,43 @@ +import pytest + +from dlt.common import jsonpath as jp + + +@pytest.mark.parametrize("compiled", [True, False]) +@pytest.mark.parametrize( + "path, expected", + [ + ("col_a", "col_a"), + ("'col.a'", "col.a"), + ("'$col_a'", "$col_a"), + ("'col|a'", "col|a"), + ], +) +def test_extract_simple_field_name_positive(path, expected, compiled): + if compiled: + path = jp.compile_path(path) + + result = jp.extract_simple_field_name(path) + assert result == expected + + +@pytest.mark.parametrize("compiled", [True, False]) +@pytest.mark.parametrize( + "path", + [ + "$.col_a", + "$.col_a.items", + "$.col_a.items[0]", + "$.col_a.items[*]", + "col_a|col_b", + ], +) +def test_extract_simple_field_name_negative(path, compiled): + if compiled: + path = jp.compile_path(path) + + result = jp.extract_simple_field_name(path) + assert result is None + + +# TODO: Test all jsonpath utils diff --git a/tests/common/test_utils.py b/tests/common/test_utils.py index 864bce5b91..9eeded1229 100644 --- a/tests/common/test_utils.py +++ b/tests/common/test_utils.py @@ -14,6 +14,7 @@ digest128, graph_edges_to_nodes, group_dict_of_lists, + is_typeerror_due_to_wrong_call, map_nested_in_place, reveal_pseudo_secret, obfuscate_pseudo_secret, @@ -25,6 +26,7 @@ get_exception_trace, get_exception_trace_chain, update_dict_nested, + removeprefix, ) @@ -404,3 +406,46 @@ def test_group_dict_of_lists_various_length_lists(): # Check if the sizes of the decomposed dicts are decreasing sizes = [len(d) for d in result] assert sizes == sorted(sizes, reverse=True), "Sizes of decomposed dicts are not decreasing" + + +def function_typeerror_exc(a, b): + raise TypeError("wrong type") + + +def test_is_typeerror_due_to_wrong_call() -> None: + def _function_test(a, *, b=None): + return a, b + + try: + _function_test() # type: ignore[call-arg] + except Exception as exc: + assert is_typeerror_due_to_wrong_call(exc, _function_test) is True + + try: + _function_test("a", "b") # type: ignore[misc] + except Exception as exc: + assert is_typeerror_due_to_wrong_call(exc, _function_test) is True + + try: + 1 / 0 + except Exception as exc: + assert is_typeerror_due_to_wrong_call(exc, _function_test) is False + + try: + function_typeerror_exc() # type: ignore[call-arg] + except Exception as exc: + assert is_typeerror_due_to_wrong_call(exc, function_typeerror_exc) is True + + try: + function_typeerror_exc("a", "b") + except Exception as exc: + assert str(exc) == "wrong type" + assert is_typeerror_due_to_wrong_call(exc, function_typeerror_exc) is False + + +def test_removeprefix() -> None: + assert removeprefix("a_data", "a_") == "data" + assert removeprefix("a_data", "a_data") == "" + assert removeprefix("a_data", "a_data_1") == "a_data" + assert removeprefix("", "a_data_1") == "" + assert removeprefix("a_data", "") == "a_data" diff --git a/tests/common/test_validation.py b/tests/common/test_validation.py index 3f8ccfc20f..6899d8d5fe 100644 --- a/tests/common/test_validation.py +++ b/tests/common/test_validation.py @@ -19,13 +19,13 @@ from dlt.common import Decimal, jsonpath from dlt.common.exceptions import DictValidationException from dlt.common.schema.typing import ( - TColumnNames, TStoredSchema, TColumnSchema, TWriteDispositionConfig, ) from dlt.common.schema.utils import simple_regex_validator -from dlt.common.typing import DictStrStr, StrStr, TDataItem, TSortOrder +from dlt.common.typing import DictStrStr, StrStr, TDataItem, TSortOrder, TColumnNames + from dlt.common.validation import validate_dict, validate_dict_ignoring_xkeys @@ -111,7 +111,7 @@ def test_doc() -> TTestRecord: def test_validate_schema_cases() -> None: with open( - "tests/common/cases/schemas/eth/ethereum_schema_v10.yml", mode="r", encoding="utf-8" + "tests/common/cases/schemas/eth/ethereum_schema_v11.yml", mode="r", encoding="utf-8" ) as f: schema_dict: TStoredSchema = yaml.safe_load(f) diff --git a/tests/common/utils.py b/tests/common/utils.py index 9b5e6bccce..a0760ffe86 100644 --- a/tests/common/utils.py +++ b/tests/common/utils.py @@ -19,11 +19,11 @@ def IMPORTED_VERSION_HASH_ETH_V10() -> str: # for import schema tests, change when upgrading the schema version - eth_V10 = load_yml_case("schemas/eth/ethereum_schema_v10") - assert eth_V10["version_hash"] == "veEmgbCPXCIiqyfabeQWwz6UIQ2liETv7LLMpyktCos=" + eth_V11 = load_yml_case("schemas/eth/ethereum_schema_v11") + assert eth_V11["version_hash"] == "XfkJ8E1tZzG/Sb3lfEZrEVshTMKdB7JpOP2HA7eS6EI=" # remove processing hints before installing as import schema # ethereum schema is a "dirty" schema with processing hints - eth = Schema.from_dict(eth_V10, remove_processing_hints=True) + eth = Schema.from_dict(eth_V11, remove_processing_hints=True) return eth.stored_version_hash diff --git a/tests/destinations/test_readable_dbapi_dataset.py b/tests/destinations/test_readable_dbapi_dataset.py index 0e01bb5267..4745735371 100644 --- a/tests/destinations/test_readable_dbapi_dataset.py +++ b/tests/destinations/test_readable_dbapi_dataset.py @@ -91,7 +91,7 @@ def test_computed_schema_columns() -> None: # now add columns relation = dataset.items - dataset.schema.tables["items"] = { # type: ignore[attr-defined] + dataset.schema.tables["items"] = { "columns": {"one": {"data_type": "text"}, "two": {"data_type": "json"}} } diff --git a/tests/extract/cases/eth_source/ethereum.schema.yaml b/tests/extract/cases/eth_source/ethereum.schema.yaml index d224088f8b..e20260bfe7 100644 --- a/tests/extract/cases/eth_source/ethereum.schema.yaml +++ b/tests/extract/cases/eth_source/ethereum.schema.yaml @@ -1,6 +1,6 @@ version: 18 -version_hash: veEmgbCPXCIiqyfabeQWwz6UIQ2liETv7LLMpyktCos= -engine_version: 10 +version_hash: XfkJ8E1tZzG/Sb3lfEZrEVshTMKdB7JpOP2HA7eS6EI= +engine_version: 11 name: ethereum tables: _dlt_loads: diff --git a/tests/extract/test_decorators.py b/tests/extract/test_decorators.py index 5dc4304a63..a14b4a9602 100644 --- a/tests/extract/test_decorators.py +++ b/tests/extract/test_decorators.py @@ -112,9 +112,9 @@ def test_load_schema_for_callable() -> None: schema = s.schema assert schema.name == "ethereum" == s.name # the schema in the associated file has this hash - eth_v9 = load_yml_case("schemas/eth/ethereum_schema_v9") + eth_v11 = load_yml_case("schemas/eth/ethereum_schema_v11") # source removes processing hints so we do - reference_schema = Schema.from_dict(eth_v9, remove_processing_hints=True) + reference_schema = Schema.from_dict(eth_v11, remove_processing_hints=True) assert schema.stored_version_hash == reference_schema.stored_version_hash diff --git a/tests/extract/test_extract.py b/tests/extract/test_extract.py index dbec417f97..9343449aed 100644 --- a/tests/extract/test_extract.py +++ b/tests/extract/test_extract.py @@ -213,6 +213,48 @@ def with_table_hints(): extract_step.extract(source, 20, 1) +def test_extract_hints_mark_incremental(extract_step: Extract) -> None: + os.environ["DATA_WRITER__DISABLE_COMPRESSION"] = "TRUE" + + @dlt.resource(columns=[{"name": "id", "data_type": "bigint"}], primary_key="id") + def with_table_hints(): + # yield a regular dataset first, simulate backfil + yield [{"id": id_, "pk": "A"} for id_ in range(1, 10)] + + # get the resource + resource = dlt.current.source().resources[dlt.current.resource_name()] + table = resource.compute_table_schema() + # also there we see the hints + assert table["columns"]["id"]["primary_key"] is True + assert table["columns"]["id"]["data_type"] == "bigint" + + # start emitting incremental + yield dlt.mark.with_hints( + [{"id": id_, "pk": "A", "created_at": id_ + 10} for id_ in range(100, 110)], + make_hints(incremental=dlt.sources.incremental("created_at", initial_value=105)), + ) + + # get the resource + resource = dlt.current.source().resources[dlt.current.resource_name()] + assert resource.incremental.cursor_path == "created_at" # type: ignore[attr-defined] + assert resource.incremental.primary_key == "id" + # we are able to add the incremental to the pipe. but it won't + # join actually executing pipe which is a clone of a (partial) pipe of the resource + assert isinstance(resource._pipe._steps[1], dlt.sources.incremental) + # NOTE: this results in unbounded exception + # assert resource.incremental.last_value == 299 + table = resource.compute_table_schema() + assert table["columns"]["created_at"]["incremental"] is not None + + yield [{"id": id_, "pk": "A", "created_at": id_ + 10} for id_ in range(110, 120)] + + source = DltSource(dlt.Schema("hintable"), "module", [with_table_hints]) + extract_step.extract(source, 20, 1) + # make sure incremental is in the source schema + table = source.schema.get_table("with_table_hints") + assert table["columns"]["created_at"]["incremental"] is not None + + def test_extract_metrics_on_exception_no_flush(extract_step: Extract) -> None: @dlt.resource def letters(): diff --git a/tests/extract/test_incremental.py b/tests/extract/test_incremental.py index 7ce4228b6c..725872b621 100644 --- a/tests/extract/test_incremental.py +++ b/tests/extract/test_incremental.py @@ -5,7 +5,7 @@ from datetime import datetime, date # noqa: I251 from itertools import chain, count from time import sleep -from typing import Any, Optional +from typing import Any, Optional, Literal, Sequence, Dict from unittest import mock import duckdb @@ -219,8 +219,74 @@ def some_data(created_at=dlt.sources.incremental("created_at")): assert rows == [(1, "a"), (2, "b"), (3, "c"), (3, "d"), (3, "e"), (3, "f"), (4, "g")] +def test_pandas_index_as_dedup_key() -> None: + from dlt.common.libs.pandas import pandas_to_arrow, pandas as pd + + some_data, p = _make_dedup_pipeline("pandas") + + # no index + no_index_r = some_data.with_name(new_name="no_index") + p.run(no_index_r) + p.run(no_index_r) + data_ = p._dataset().no_index.arrow() + assert data_.schema.names == ["created_at", "id"] + assert data_["id"].to_pylist() == ["a", "b", "c", "d", "e", "f", "g"] + + # unnamed index: explicitly converted + unnamed_index_r = some_data.with_name(new_name="unnamed_index").add_map( + lambda df: pandas_to_arrow(df, preserve_index=True) + ) + # use it (as in arrow table) to deduplicate + unnamed_index_r.incremental.primary_key = "__index_level_0__" + p.run(unnamed_index_r) + p.run(unnamed_index_r) + data_ = p._dataset().unnamed_index.arrow() + assert data_.schema.names == ["created_at", "id", "index_level_0"] + # indexes 2 and 3 are removed from second batch because they were in the previous batch + # and the created_at overlapped so they got deduplicated + assert data_["index_level_0"].to_pylist() == [0, 1, 2, 3, 4, 0, 1, 4] + + def _make_named_index(df_: pd.DataFrame) -> pd.DataFrame: + df_.index = pd.RangeIndex(start=0, stop=len(df_), step=1, name="order_id") + return df_ + + # named index explicitly converted + named_index_r = some_data.with_name(new_name="named_index").add_map( + lambda df: pandas_to_arrow(_make_named_index(df), preserve_index=True) + ) + # use it (as in arrow table) to deduplicate + named_index_r.incremental.primary_key = "order_id" + p.run(named_index_r) + p.run(named_index_r) + data_ = p._dataset().named_index.arrow() + assert data_.schema.names == ["created_at", "id", "order_id"] + assert data_["order_id"].to_pylist() == [0, 1, 2, 3, 4, 0, 1, 4] + + # named index explicitly converted + named_index_impl_r = some_data.with_name(new_name="named_index_impl").add_map( + lambda df: _make_named_index(df) + ) + p.run(named_index_impl_r) + p.run(named_index_impl_r) + data_ = p._dataset().named_index_impl.arrow() + assert data_.schema.names == ["created_at", "id"] + assert data_["id"].to_pylist() == ["a", "b", "c", "d", "e", "f", "g"] + + @pytest.mark.parametrize("item_type", ALL_TEST_DATA_ITEM_FORMATS) def test_unique_rows_by_hash_are_deduplicated(item_type: TestDataItemFormat) -> None: + some_data, p = _make_dedup_pipeline(item_type) + p.run(some_data()) + p.run(some_data()) + + with p.sql_client() as c: + with c.execute_query("SELECT created_at, id FROM some_data ORDER BY created_at, id") as cur: + rows = cur.fetchall() + print(rows) + assert rows == [(1, "a"), (2, "b"), (3, "c"), (3, "d"), (3, "e"), (3, "f"), (4, "g")] + + +def _make_dedup_pipeline(item_type: TestDataItemFormat): data1 = [ {"created_at": 1, "id": "a"}, {"created_at": 2, "id": "b"}, @@ -235,7 +301,6 @@ def test_unique_rows_by_hash_are_deduplicated(item_type: TestDataItemFormat) -> {"created_at": 3, "id": "f"}, {"created_at": 4, "id": "g"}, ] - source_items1 = data_to_item_format(item_type, data1) source_items2 = data_to_item_format(item_type, data2) @@ -250,14 +315,7 @@ def some_data(created_at=dlt.sources.incremental("created_at")): pipeline_name=uniq_id(), destination=dlt.destinations.duckdb(credentials=duckdb.connect(":memory:")), ) - p.run(some_data()) - p.run(some_data()) - - with p.sql_client() as c: - with c.execute_query("SELECT created_at, id FROM some_data order by created_at, id") as cur: - rows = cur.fetchall() - - assert rows == [(1, "a"), (2, "b"), (3, "c"), (3, "d"), (3, "e"), (3, "f"), (4, "g")] + return some_data, p def test_nested_cursor_path() -> None: @@ -1468,10 +1526,13 @@ def test_apply_hints_incremental(item_type: TestDataItemFormat) -> None: data = [{"created_at": 1}, {"created_at": 2}, {"created_at": 3}] source_items = data_to_item_format(item_type, data) + should_have_arg = True + @dlt.resource def some_data(created_at: Optional[dlt.sources.incremental[int]] = None): # make sure that incremental from apply_hints is here - if created_at is not None: + if should_have_arg: + assert created_at is not None assert created_at.cursor_path == "created_at" assert created_at.last_value_func is max yield source_items @@ -1505,6 +1566,7 @@ def some_data(created_at: Optional[dlt.sources.incremental[int]] = None): assert list(r) == [] # remove incremental + should_have_arg = False r.apply_hints(incremental=dlt.sources.incremental.EMPTY) assert r.incremental is not None assert r.incremental.incremental is None @@ -1515,6 +1577,7 @@ def some_data(created_at: Optional[dlt.sources.incremental[int]] = None): # as above but we provide explicit incremental when creating resource p = p.drop() + should_have_arg = True r = some_data(created_at=dlt.sources.incremental("created_at", last_value_func=min)) # hints have precedence, as expected r.apply_hints(incremental=dlt.sources.incremental("created_at", last_value_func=max)) @@ -3568,3 +3631,223 @@ def some_data( call for call in logger_spy.call_args_list if "Large number of records" in call.args[0] ] assert len(warning_calls) == 1 + + +def _resource_for_table_hint( + hint_type: Literal[ + "default_arg", "explicit_arg", "apply_hints", "default_arg_override", "decorator" + ], + data: Sequence[Dict[str, Any]], + incremental_arg: dlt.sources.incremental[Any], + incremental_arg_default: dlt.sources.incremental[Any] = None, +) -> DltResource: + if incremental_arg is None and incremental_arg_default is None: + raise ValueError("One of the incremental arguments must be provided.") + + decorator_arg = None + if hint_type == "default_arg": + default_arg = incremental_arg_default + override_arg = None + elif hint_type == "default_arg_override": + default_arg = incremental_arg_default + override_arg = incremental_arg + elif hint_type == "decorator": + default_arg = None + override_arg = None + decorator_arg = incremental_arg_default + else: + default_arg = None + override_arg = incremental_arg + + @dlt.resource(incremental=decorator_arg) + def some_data( + updated_at: dlt.sources.incremental[Any] = default_arg, + ) -> Any: + yield data_to_item_format("object", data) + + if override_arg is None: + return some_data() + + if hint_type == "apply_hints": + rs = some_data() + rs.apply_hints(incremental=override_arg) + return rs + + return some_data(updated_at=override_arg) + + +@pytest.mark.parametrize( + "hint_type", ["default_arg", "explicit_arg", "apply_hints", "default_arg_override", "decorator"] +) +@pytest.mark.parametrize( + "incremental_settings", + [ + { + "last_value_func": "min", + "row_order": "desc", + "on_cursor_value_missing": "include", + }, + {"last_value_func": "max", "on_cursor_value_missing": "raise"}, + ], +) +def test_incremental_table_hint_datetime_column( + hint_type: Literal[ + "default_arg", + "explicit_arg", + "default_arg_override", + "apply_hints", + "decorator", + ], + incremental_settings: Dict[str, Any], +) -> None: + initial_value_override = pendulum.now() + initial_value_default = pendulum.now().subtract(seconds=10) + rs = _resource_for_table_hint( + hint_type, + [{"updated_at": pendulum.now().add(seconds=i)} for i in range(1, 12)], + dlt.sources.incremental( + "updated_at", initial_value=initial_value_override, **incremental_settings + ), + dlt.sources.incremental( + "updated_at", initial_value=initial_value_default, **incremental_settings + ), + ) + + pipeline = dlt.pipeline(pipeline_name=uniq_id()) + pipeline.extract(rs) + + table_schema = pipeline.default_schema.tables["some_data"] + + assert table_schema["columns"]["updated_at"]["incremental"] is True + + +def incremental_instance_or_dict(use_dict: bool, **kwargs): + if use_dict: + return kwargs + return dlt.sources.incremental(**kwargs) + + +@pytest.mark.parametrize("use_dict", [False, True]) +def test_incremental_in_resource_decorator(use_dict: bool) -> None: + # Incremental set in decorator, without any arguments + @dlt.resource( + incremental=incremental_instance_or_dict( + use_dict, cursor_path="value", initial_value=5, last_value_func=min + ) + ) + def no_incremental_arg(): + yield [{"value": i} for i in range(10)] + + result = list(no_incremental_arg()) + # filtering is applied + assert result == [{"value": i} for i in range(0, 6)] + + # Apply hints overrides the decorator settings + rs = no_incremental_arg() + rs.apply_hints( + incremental=incremental_instance_or_dict( + use_dict, cursor_path="value", initial_value=3, last_value_func=max + ) + ) + result = list(rs) + assert result == [{"value": i} for i in range(3, 10)] + + @dlt.resource( + incremental=incremental_instance_or_dict( + use_dict, cursor_path="value", initial_value=5, last_value_func=min + ) + ) + def with_optional_incremental_arg(incremental: Optional[dlt.sources.incremental[int]] = None): + assert incremental is not None + yield [{"value": i} for i in range(10)] + + # Decorator settings are used + result = list(with_optional_incremental_arg()) + assert result == [{"value": i} for i in range(0, 6)] + + +@pytest.mark.parametrize("use_dict", [False, True]) +def test_incremental_in_resource_decorator_default_arg(use_dict: bool) -> None: + @dlt.resource( + incremental=incremental_instance_or_dict( + use_dict, cursor_path="value", initial_value=5, last_value_func=min + ) + ) + def with_default_incremental_arg( + incremental: dlt.sources.incremental[int] = dlt.sources.incremental( + "value", initial_value=3, last_value_func=min + ) + ): + assert incremental.last_value == initial_value + assert incremental.last_value_func == last_value_func + yield [{"value": i} for i in range(10)] + + last_value_func = max + initial_value = 4 + # Explicit argument overrides the default and decorator argument + result = list( + with_default_incremental_arg( + incremental=dlt.sources.incremental( + "value", initial_value=initial_value, last_value_func=last_value_func + ) + ) + ) + assert result == [{"value": i} for i in range(4, 10)] + + # Decorator param overrides function default arg + last_value_func = min + initial_value = 5 + result = list(with_default_incremental_arg()) + assert result == [{"value": i} for i in range(0, 6)] + + +@pytest.mark.parametrize("use_dict", [False, True]) +def test_incremental_table_hint_merged_columns(use_dict: bool) -> None: + @dlt.resource( + incremental=incremental_instance_or_dict( + use_dict, cursor_path="col_a", initial_value=3, last_value_func=min + ) + ) + def some_data(): + yield [{"col_a": i, "foo": i + 2, "col_b": i + 1, "bar": i + 3} for i in range(10)] + + pipeline = dlt.pipeline(pipeline_name=uniq_id()) + pipeline.extract(some_data()) + + table_schema = pipeline.default_schema.tables["some_data"] + assert table_schema["columns"]["col_a"]["incremental"] is True + + rs = some_data() + rs.apply_hints( + incremental=incremental_instance_or_dict( + use_dict, cursor_path="col_b", initial_value=5, last_value_func=max + ) + ) + + pipeline.extract(rs) + + table_schema_2 = pipeline.default_schema.tables["some_data"] + + # Only one column should have the hint + assert "incremental" not in table_schema_2["columns"]["col_a"] + assert table_schema_2["columns"]["col_b"]["incremental"] is True + + +@pytest.mark.parametrize("use_dict", [True, False]) +def test_incremental_column_hint_cursor_is_not_column(use_dict: bool): + @dlt.resource( + incremental=incremental_instance_or_dict( + use_dict, cursor_path="col_a|col_b", initial_value=3, last_value_func=min + ) + ) + def some_data(): + yield [{"col_a": i, "foo": i + 2, "col_b": i + 1, "bar": i + 3} for i in range(10)] + + pipeline = dlt.pipeline(pipeline_name=uniq_id()) + + pipeline.extract(some_data()) + + table_schema = pipeline.default_schema.tables["some_data"] + + for col in table_schema["columns"].values(): + assert "incremental" not in col diff --git a/tests/libs/pyarrow/test_pyarrow_normalizer.py b/tests/libs/pyarrow/test_pyarrow_normalizer.py index 32ee5fdafc..c81d8cd974 100644 --- a/tests/libs/pyarrow/test_pyarrow_normalizer.py +++ b/tests/libs/pyarrow/test_pyarrow_normalizer.py @@ -5,12 +5,12 @@ from dlt.common.libs.pyarrow import normalize_py_arrow_item, NameNormalizationCollision from dlt.common.schema.utils import new_column, TColumnSchema -from dlt.common.schema.normalizers import explicit_normalizers, import_normalizers +from dlt.common.schema.normalizers import configured_normalizers, import_normalizers from dlt.common.destination import DestinationCapabilitiesContext def _normalize(table: pa.Table, columns: List[TColumnSchema]) -> pa.Table: - _, naming, _ = import_normalizers(explicit_normalizers()) + _, naming, _ = import_normalizers(configured_normalizers()) caps = DestinationCapabilitiesContext() columns_schema = {c["name"]: c for c in columns} return normalize_py_arrow_item(table, columns_schema, naming, caps) diff --git a/tests/load/clickhouse/test_clickhouse_configuration.py b/tests/load/clickhouse/test_clickhouse_configuration.py index ad33062f11..eabc3094bd 100644 --- a/tests/load/clickhouse/test_clickhouse_configuration.py +++ b/tests/load/clickhouse/test_clickhouse_configuration.py @@ -56,7 +56,8 @@ def test_clickhouse_configuration() -> None: def test_clickhouse_connection_settings(client: ClickHouseClient) -> None: """Test experimental settings are set correctly for the session.""" - conn = client.sql_client.open_connection() + # with client.sql_client.open_connection() as conn: + conn = client.sql_client.native_connection cursor1 = conn.cursor() cursor2 = conn.cursor() @@ -69,3 +70,26 @@ def test_clickhouse_connection_settings(client: ClickHouseClient) -> None: assert ("allow_experimental_lightweight_delete", "1") in res assert ("enable_http_compression", "1") in res assert ("date_time_input_format", "best_effort") in res + + +def test_client_has_dataset(client: ClickHouseClient) -> None: + # with client.sql_client as sql_client: + assert client.sql_client.has_dataset() + separator = client.config.dataset_table_separator + + def _assert_has_dataset() -> None: + assert not client.sql_client.has_dataset() + client.sql_client.create_dataset() + assert client.sql_client.has_dataset() + client.sql_client.drop_dataset() + assert not client.sql_client.has_dataset() + + try: + # change separator + client.config.dataset_table_separator = "_" + _assert_has_dataset() + + client.config.dataset_table_separator = "" + _assert_has_dataset() + finally: + client.config.dataset_table_separator = separator diff --git a/tests/load/conftest.py b/tests/load/conftest.py index 76a7248e5b..c52fea607d 100644 --- a/tests/load/conftest.py +++ b/tests/load/conftest.py @@ -9,7 +9,7 @@ drop_pipeline, empty_schema, ) -from tests.utils import preserve_environ, patch_home_dir +from tests.utils import preserve_environ, patch_home_dir, autouse_test_storage @pytest.fixture(scope="function", params=DEFAULT_BUCKETS) diff --git a/tests/load/duckdb/test_duckdb_client.py b/tests/load/duckdb/test_duckdb_client.py index a9479a0bb9..49475ce43f 100644 --- a/tests/load/duckdb/test_duckdb_client.py +++ b/tests/load/duckdb/test_duckdb_client.py @@ -19,7 +19,7 @@ from dlt.pipeline.exceptions import PipelineStepFailed from tests.pipeline.utils import assert_table -from tests.utils import patch_home_dir, autouse_test_storage, TEST_STORAGE_ROOT +from tests.utils import autouse_test_storage, TEST_STORAGE_ROOT # mark all tests as essential, do not remove pytestmark = pytest.mark.essential diff --git a/tests/load/filesystem/test_aws_credentials.py b/tests/load/filesystem/test_aws_credentials.py index b782e76b7e..1113b9b35d 100644 --- a/tests/load/filesystem/test_aws_credentials.py +++ b/tests/load/filesystem/test_aws_credentials.py @@ -9,7 +9,6 @@ from tests.common.configuration.utils import environment from tests.load.utils import ALL_FILESYSTEM_DRIVERS -from tests.utils import autouse_test_storage # mark all tests as essential, do not remove pytestmark = pytest.mark.essential diff --git a/tests/load/filesystem/test_azure_credentials.py b/tests/load/filesystem/test_azure_credentials.py index 64da35d9be..811eb41f75 100644 --- a/tests/load/filesystem/test_azure_credentials.py +++ b/tests/load/filesystem/test_azure_credentials.py @@ -3,6 +3,7 @@ from uuid import uuid4 import pytest +from pytest_mock import MockerFixture import dlt from dlt.common import pendulum @@ -17,7 +18,6 @@ from dlt.common.storages.configuration import FilesystemConfiguration from tests.load.utils import ALL_FILESYSTEM_DRIVERS, AZ_BUCKET from tests.common.configuration.utils import environment -from tests.utils import autouse_test_storage from dlt.common.storages.fsspec_filesystem import fsspec_from_config # mark all tests as essential, do not remove @@ -82,6 +82,7 @@ def test_azure_credentials_from_sas_token(environment: Dict[str, str]) -> None: environment["CREDENTIALS__AZURE_STORAGE_SAS_TOKEN"] = ( "sp=rwdlacx&se=2021-01-01T00:00:00Z&sv=2019-12-12&sr=c&sig=1234567890" ) + environment["CREDENTIALS__AZURE_ACCOUNT_HOST"] = "blob.core.usgovcloudapi.net" config = resolve_configuration(AzureCredentials()) @@ -95,6 +96,7 @@ def test_azure_credentials_from_sas_token(environment: Dict[str, str]) -> None: "account_name": environment["CREDENTIALS__AZURE_STORAGE_ACCOUNT_NAME"], "account_key": None, "sas_token": environment["CREDENTIALS__AZURE_STORAGE_SAS_TOKEN"], + "account_host": "blob.core.usgovcloudapi.net", } @@ -124,6 +126,7 @@ def test_azure_credentials_from_default(environment: Dict[str, str]) -> None: "account_key": None, "sas_token": None, "anon": False, + "account_host": None, } @@ -141,6 +144,7 @@ def test_azure_service_principal_credentials(environment: Dict[str, str]) -> Non assert config.to_adlfs_credentials() == { "account_name": environment["CREDENTIALS__AZURE_STORAGE_ACCOUNT_NAME"], + "account_host": None, "client_id": environment["CREDENTIALS__AZURE_CLIENT_ID"], "client_secret": environment["CREDENTIALS__AZURE_CLIENT_SECRET"], "tenant_id": environment["CREDENTIALS__AZURE_TENANT_ID"], @@ -199,3 +203,69 @@ def test_azure_service_principal_fs_operations( assert f"{bucket}/{fn}/{fn}" in files fs.delete(f"{bucket}/{fn}/{fn}") fs.rmdir(f"{bucket}/{fn}") + + +def test_account_host_kwargs(environment: Dict[str, str], mocker: MockerFixture) -> None: + environment["CREDENTIALS__AZURE_STORAGE_ACCOUNT_NAME"] = "fake_account_name" + environment["CREDENTIALS__AZURE_STORAGE_ACCOUNT_KEY"] = "QWERTYUIOPASDFGHJKLZXCVBNM1234567890" + environment["CREDENTIALS__AZURE_SAS_TOKEN_PERMISSIONS"] = "rl" + + # [destination.filesystem] + # bucket_url="..." + # [destination.filesystem.kwargs] + # account_host="blob.core.usgovcloudapi.net" + # [destination.filesystem.credentials] + # ... + + config = resolve_configuration(FilesystemConfiguration(bucket_url="az://dlt-ci-test-bucket")) + config.kwargs = {"account_host": "dlt_ci.blob.core.usgovcloudapi.net"} + + from adlfs import AzureBlobFileSystem + + connect_mock = mocker.spy(AzureBlobFileSystem, "do_connect") + fsspec_from_config(config) + + connect_mock.assert_called_once() + assert connect_mock.call_args[0][0].account_host == "dlt_ci.blob.core.usgovcloudapi.net" + + config = resolve_configuration( + FilesystemConfiguration( + bucket_url="abfss://dlt-ci-test-bucket@dlt_ci.blob.core.usgovcloudapi.net" + ) + ) + connect_mock.reset_mock() + + assert isinstance(config.credentials, AzureCredentialsWithoutDefaults) + assert config.credentials.azure_storage_account_name == "fake_account_name" + # ignores the url from the bucket_url 🤷 + fs, _ = fsspec_from_config(config) + connect_mock.assert_called_once() + assert connect_mock.call_args[0][0].account_url.endswith( + "fake_account_name.blob.core.windows.net" + ) + + # use host + environment["KWARGS"] = '{"account_host": "fake_account_name.blob.core.usgovcloudapi.net"}' + config = resolve_configuration( + FilesystemConfiguration( + bucket_url="abfss://dlt-ci-test-bucket@fake_account_name.blob.core.usgovcloudapi.net" + ) + ) + connect_mock.reset_mock() + + # NOTE: fsspec is caching instances created in the same thread: skip_instance_cache + fs, _ = fsspec_from_config(config) + connect_mock.assert_called_once() + # assert connect_mock.call_args[0][0].account_url.endswith("fake_account_name.blob.core.usgovcloudapi.net") + assert fs.account_url.endswith("fake_account_name.blob.core.usgovcloudapi.net") + + +def test_azure_account_host(environment: Dict[str, str]) -> None: + environment["CREDENTIALS__AZURE_STORAGE_ACCOUNT_NAME"] = "fake_account_name" + environment["CREDENTIALS__AZURE_STORAGE_ACCOUNT_KEY"] = "QWERTYUIOPASDFGHJKLZXCVBNM1234567890" + environment["CREDENTIALS__AZURE_SAS_TOKEN_PERMISSIONS"] = "rl" + environment["CREDENTIALS__AZURE_ACCOUNT_HOST"] = "dlt_ci.blob.core.usgovcloudapi.net" + + config = resolve_configuration(FilesystemConfiguration(bucket_url="az://dlt-ci-test-bucket")) + fs, _ = fsspec_from_config(config) + assert fs.account_host == "dlt_ci.blob.core.usgovcloudapi.net" diff --git a/tests/load/filesystem/test_filesystem_common.py b/tests/load/filesystem/test_filesystem_common.py index 0db93410e5..afcd9105a8 100644 --- a/tests/load/filesystem/test_filesystem_common.py +++ b/tests/load/filesystem/test_filesystem_common.py @@ -14,8 +14,9 @@ from dlt.common.configuration import resolve from dlt.common.configuration.inject import with_config from dlt.common.configuration.specs import AnyAzureCredentials +from dlt.common.exceptions import TerminalValueError from dlt.common.storages import fsspec_from_config, FilesystemConfiguration -from dlt.common.storages.configuration import make_fsspec_url +from dlt.common.storages.configuration import ensure_canonical_az_url, make_fsspec_url from dlt.common.storages.fsspec_filesystem import MTIME_DISPATCH, glob_files from dlt.common.utils import custom_environ, uniq_id from dlt.destinations import filesystem @@ -27,7 +28,6 @@ from tests.common.configuration.utils import environment from tests.common.storages.utils import TEST_SAMPLE_FILES, assert_sample_files from tests.load.utils import ALL_FILESYSTEM_DRIVERS, AWS_BUCKET, WITH_GDRIVE_BUCKETS -from tests.utils import autouse_test_storage from tests.load.filesystem.utils import self_signed_cert @@ -74,6 +74,36 @@ def test_remote_url(bucket_url: str) -> None: assert make_fsspec_url(scheme, fs_path, bucket_url) == bucket_url +def test_make_az_url() -> None: + url = make_fsspec_url( + "azure", "/dlt-ci/path", "abfss://dlt-ci-test-bucket@dlt_ci.blob.core.usgovcloudapi.net" + ) + assert url == "azure://@dlt_ci.blob.core.usgovcloudapi.net/dlt-ci/path" + + +def test_ensure_az_canonical_url() -> None: + url = ensure_canonical_az_url( + "abfss://dlt-ci-test-bucket@dlt_ci.blob.core.usgovcloudapi.net", "https" + ) + assert url == "https://dlt-ci-test-bucket@dlt_ci.blob.core.usgovcloudapi.net" + + with pytest.raises(TerminalValueError): + ensure_canonical_az_url("abfss://dlt-ci-test-bucket/path/path", "https") + + url = ensure_canonical_az_url( + "abfss://dlt-ci-test-bucket/path/path", "https", storage_account_name="fake_dlt" + ) + assert url == "https://dlt-ci-test-bucket@fake_dlt.dfs.core.windows.net/path/path" + + url = ensure_canonical_az_url( + "abfss://dlt-ci-test-bucket/path/path", + "https", + storage_account_name="fake_dlt", + account_host="dlt_ci.blob.core.usgovcloudapi.net", + ) + assert url == "https://dlt-ci-test-bucket@dlt_ci.blob.core.usgovcloudapi.net/path/path" + + def test_filesystem_instance(with_gdrive_buckets_env: str) -> None: @retry(stop=stop_after_attempt(10), wait=wait_fixed(1), reraise=True) def check_file_exists(filedir_: str, file_url_: str): diff --git a/tests/load/lancedb/test_model_providers.py b/tests/load/lancedb/test_model_providers.py new file mode 100644 index 0000000000..7ad5464fe5 --- /dev/null +++ b/tests/load/lancedb/test_model_providers.py @@ -0,0 +1,44 @@ +""" +Test intricacies and configuration related to each provider. +""" + +import os +from typing import Iterator, Any, Generator + +import pytest +from lancedb import DBConnection # type: ignore +from lancedb.embeddings import EmbeddingFunctionRegistry # type: ignore +from lancedb.table import Table # type: ignore + +import dlt +from dlt.common.configuration import resolve_configuration +from dlt.common.typing import DictStrStr +from dlt.common.utils import uniq_id +from dlt.destinations.impl.lancedb import lancedb_adapter +from dlt.destinations.impl.lancedb.configuration import LanceDBClientConfiguration +from dlt.destinations.impl.lancedb.lancedb_client import LanceDBClient +from tests.load.utils import drop_active_pipeline_data, sequence_generator +from tests.pipeline.utils import assert_load_info + +# Mark all tests as essential, don't remove. +pytestmark = pytest.mark.essential + + +@pytest.fixture(autouse=True) +def drop_lancedb_data() -> Iterator[Any]: + yield + drop_active_pipeline_data() + + +def test_lancedb_ollama_endpoint_configuration() -> None: + os.environ["DESTINATION__LANCEDB__EMBEDDING_MODEL_PROVIDER"] = "ollama" + os.environ["DESTINATION__LANCEDB__EMBEDDING_MODEL"] = "nomic-embed-text" + os.environ["DESTINATION__LANCEDB__EMBEDDING_MODEL_PROVIDER_HOST"] = "http://198.163.194.3:24233" + + config = resolve_configuration( + LanceDBClientConfiguration()._bind_dataset_name(dataset_name="dataset"), + sections=("destination", "lancedb"), + ) + assert config.embedding_model_provider == "ollama" + assert config.embedding_model == "nomic-embed-text" + assert config.embedding_model_provider_host == "http://198.163.194.3:24233" diff --git a/tests/load/pipeline/conftest.py b/tests/load/pipeline/conftest.py index a2ba65494b..80c418ed22 100644 --- a/tests/load/pipeline/conftest.py +++ b/tests/load/pipeline/conftest.py @@ -1,2 +1,2 @@ -from tests.utils import autouse_test_storage, duckdb_pipeline_location +from tests.utils import duckdb_pipeline_location from tests.pipeline.utils import drop_dataset_from_env diff --git a/tests/load/pipeline/test_merge_disposition.py b/tests/load/pipeline/test_merge_disposition.py index 2925bfac6f..8b6fc751d9 100644 --- a/tests/load/pipeline/test_merge_disposition.py +++ b/tests/load/pipeline/test_merge_disposition.py @@ -80,7 +80,7 @@ def test_merge_on_keys_in_schema( skip_if_not_supported(merge_strategy, p.destination) - with open("tests/common/cases/schemas/eth/ethereum_schema_v9.yml", "r", encoding="utf-8") as f: + with open("tests/common/cases/schemas/eth/ethereum_schema_v11.yml", "r", encoding="utf-8") as f: schema = dlt.Schema.from_dict(yaml.safe_load(f)) # make block uncles unseen to trigger filtering loader in loader for nested tables diff --git a/tests/load/pipeline/test_postgres.py b/tests/load/pipeline/test_postgres.py index c8dc0e10cc..29ad21941e 100644 --- a/tests/load/pipeline/test_postgres.py +++ b/tests/load/pipeline/test_postgres.py @@ -76,7 +76,7 @@ def test_pipeline_explicit_destination_credentials( credentials="postgresql://loader:loader@localhost:7777/dlt_data", ), ) - c = p._get_destination_clients(Schema("s"), p._get_destination_client_initial_config())[0] + c = p._get_destination_clients(Schema("s"))[0] assert c.config.credentials.port == 7777 # type: ignore[attr-defined] # TODO: may want to clear the env completely and ignore/mock config files somehow to avoid side effects @@ -89,7 +89,7 @@ def test_pipeline_explicit_destination_credentials( credentials="postgresql://loader:loader@localhost:5432/dlt_data", ), ) - c = p._get_destination_clients(Schema("s"), p._get_destination_client_initial_config())[0] + c = p._get_destination_clients(Schema("s"))[0] assert c.config.credentials.host == "localhost" # type: ignore[attr-defined] # explicit partial credentials will use config providers @@ -102,7 +102,7 @@ def test_pipeline_explicit_destination_credentials( credentials="postgresql://localhost:5432/dlt_data", ), ) - c = p._get_destination_clients(Schema("s"), p._get_destination_client_initial_config())[0] + c = p._get_destination_clients(Schema("s"))[0] assert c.config.credentials.username == "UN" # type: ignore[attr-defined] # host is taken form explicit credentials assert c.config.credentials.host == "localhost" # type: ignore[attr-defined] diff --git a/tests/load/pipeline/test_scd2.py b/tests/load/pipeline/test_scd2.py index 2a5b9ed296..962c501619 100644 --- a/tests/load/pipeline/test_scd2.py +++ b/tests/load/pipeline/test_scd2.py @@ -11,7 +11,7 @@ from dlt.common.pipeline import LoadInfo from dlt.common.data_types.typing import TDataType from dlt.common.schema.typing import DEFAULT_VALIDITY_COLUMN_NAMES -from dlt.common.normalizers.json.relational import DataItemNormalizer +from dlt.common.normalizers.json.helpers import get_row_hash from dlt.common.normalizers.naming.snake_case import NamingConvention as SnakeCaseNamingConvention from dlt.common.time import ensure_pendulum_datetime, reduce_pendulum_datetime_precision from dlt.extract.resource import DltResource @@ -30,7 +30,6 @@ from tests.utils import TPythonTableFormat -get_row_hash = DataItemNormalizer.get_row_hash FROM, TO = DEFAULT_VALIDITY_COLUMN_NAMES diff --git a/tests/load/postgres/test_postgres_table_builder.py b/tests/load/postgres/test_postgres_table_builder.py index 4dac400f2a..e2ed0f0b2e 100644 --- a/tests/load/postgres/test_postgres_table_builder.py +++ b/tests/load/postgres/test_postgres_table_builder.py @@ -1,24 +1,35 @@ -import pytest from copy import deepcopy +from typing import Generator, Any, List + +import pytest import sqlfluff +import dlt from dlt.common.exceptions import TerminalValueError -from dlt.common.utils import uniq_id from dlt.common.schema import Schema, utils - +from dlt.common.typing import DictStrStr +from dlt.common.utils import uniq_id from dlt.destinations import postgres -from dlt.destinations.impl.postgres.postgres import PostgresClient from dlt.destinations.impl.postgres.configuration import ( PostgresClientConfiguration, PostgresCredentials, ) - +from dlt.destinations.impl.postgres.postgres import ( + PostgresClient, +) +from dlt.destinations.impl.postgres.postgres_adapter import ( + postgres_adapter, + SRID_HINT, + GEOMETRY_HINT, +) +from dlt.extract import DltResource from tests.cases import ( TABLE_UPDATE, TABLE_UPDATE_ALL_INT_PRECISIONS, - TABLE_UPDATE_ALL_TIMESTAMP_PRECISIONS, ) -from tests.load.utils import empty_schema +from tests.load.postgres.utils import generate_sample_geometry_records +from tests.load.utils import destinations_configs, DestinationTestConfiguration, sequence_generator +from tests.utils import assert_load_info # mark all tests as essential, do not remove pytestmark = pytest.mark.essential @@ -182,3 +193,217 @@ def test_create_dlt_table(client: PostgresClient) -> None: sqlfluff.parse(sql, dialect="postgres") qualified_name = client.sql_client.make_qualified_table_name("_dlt_version") assert f"CREATE TABLE IF NOT EXISTS {qualified_name}" in sql + + +@pytest.mark.parametrize( + "destination_config", + destinations_configs(default_sql_configs=True, subset=["postgres"]), + ids=lambda x: x.name, +) +def test_adapter_geometry_hint_config( + destination_config: DestinationTestConfiguration, +) -> None: + @dlt.resource(columns=[{"name": "content", "data_type": "text"}]) + def some_data() -> Generator[DictStrStr, Any, None]: + yield from next(sequence_generator()) + + assert some_data.columns["content"] == {"name": "content", "data_type": "text"} # type: ignore[index] + + # Default SRID. + postgres_adapter(some_data, geometry=["content"]) + + assert some_data.columns["content"] == { # type: ignore + "name": "content", + "data_type": "text", + GEOMETRY_HINT: True, + SRID_HINT: 4326, + } + + # Nonstandard SRID. + postgres_adapter(some_data, geometry="content", srid=8232) + + assert some_data.columns["content"] == { # type: ignore + "name": "content", + "data_type": "text", + GEOMETRY_HINT: True, + SRID_HINT: 8232, + } + + +@pytest.mark.parametrize( + "destination_config", + destinations_configs(default_sql_configs=True, subset=["postgres"]), + ids=lambda x: x.name, +) +def test_geometry_types( + destination_config: DestinationTestConfiguration, +) -> None: + from shapely import wkt, wkb, LinearRing, Polygon # type: ignore + + @dlt.resource + def geodata_default_wkt(): + yield from generate_sample_geometry_records("wkt") + + @dlt.resource + def geodata_3857_wkt(): + yield from generate_sample_geometry_records("wkt") + + @dlt.resource + def geodata_2163_wkt(): + yield from generate_sample_geometry_records("wkt") + + @dlt.resource + def geodata_default_wkb_hex(): + yield from generate_sample_geometry_records("wkb_hex") + + @dlt.resource + def geodata_3857_wkb_hex(): + yield from generate_sample_geometry_records("wkb_hex") + + @dlt.resource + def geodata_2163_wkb_hex(): + yield from generate_sample_geometry_records("wkb_hex") + + @dlt.resource(file_format="csv") + def geodata_default_csv_wkt(): + yield from generate_sample_geometry_records("wkt") + + @dlt.resource(file_format="csv") + def geodata_3857_csv_wkt(): + yield from generate_sample_geometry_records("wkt") + + @dlt.resource(file_format="csv") + def geodata_2163_csv_wkt(): + yield from generate_sample_geometry_records("wkt") + + @dlt.resource(file_format="csv") + def geodata_default_csv_wkb_hex(): + yield from generate_sample_geometry_records("wkb_hex") + + @dlt.resource(file_format="csv") + def geodata_3857_csv_wkb_hex(): + yield from generate_sample_geometry_records("wkb_hex") + + @dlt.resource(file_format="csv") + def geodata_2163_csv_wkb_hex(): + yield from generate_sample_geometry_records("wkb_hex") + + @dlt.resource + def no_geodata(): + yield from [{"a": 1}, {"a": 2}] + + postgres_adapter(geodata_default_wkt, geometry=["geom"]) + postgres_adapter(geodata_3857_wkt, geometry=["geom"], srid=3857) + postgres_adapter(geodata_2163_wkt, geometry=["geom"], srid=2163) + postgres_adapter(geodata_default_wkb_hex, geometry=["geom"]) + postgres_adapter(geodata_3857_wkb_hex, geometry=["geom"], srid=3857) + postgres_adapter(geodata_2163_wkb_hex, geometry=["geom"], srid=2163) + postgres_adapter(geodata_default_csv_wkt, geometry=["geom"]) + postgres_adapter(geodata_3857_csv_wkt, geometry=["geom"], srid=3857) + postgres_adapter(geodata_2163_csv_wkt, geometry=["geom"], srid=2163) + postgres_adapter(geodata_default_csv_wkb_hex, geometry=["geom"]) + postgres_adapter(geodata_3857_csv_wkb_hex, geometry=["geom"], srid=3857) + postgres_adapter(geodata_2163_csv_wkb_hex, geometry=["geom"], srid=2163) + + @dlt.source + def geodata() -> List[DltResource]: + return [ + geodata_default_wkt, + geodata_3857_wkt, + geodata_2163_wkt, + geodata_default_wkb_hex, + geodata_3857_wkb_hex, + geodata_2163_wkb_hex, + no_geodata, + geodata_default_csv_wkt, + geodata_3857_csv_wkt, + geodata_2163_csv_wkt, + geodata_default_csv_wkb_hex, + geodata_3857_csv_wkb_hex, + geodata_2163_csv_wkb_hex, + ] + + pipeline = destination_config.setup_pipeline("test_geometry_types", dev_mode=True) + info = pipeline.run( + geodata(), + ) + assert_load_info(info) + + # Assert that types were read in as PostGIS geometry types + with pipeline.sql_client() as c: + with c.execute_query(f"""SELECT f_geometry_column +FROM geometry_columns +WHERE f_table_name in + ('geodata_default_wkb', 'geodata_3857_wkb', 'geodata_2163_wkb', 'geodata_default_wkt', 'geodata_3857_wkt', + 'geodata_2163_wkt', 'geodata_default_wkb_hex', 'geodata_3857_wkb_hex', 'geodata_2163_wkb_hex', + 'geodata_default_csv_wkt', 'geodata_3857_csv_wkt', 'geodata_2163_csv_wkt', 'geodata_default_csv_wkb_hex', + 'geodata_3857_csv_wkb_hex', 'geodata_2163_csv_wkb_hex' + ) + AND f_table_schema = '{c.fully_qualified_dataset_name(escape=False)}'""") as cur: + records = cur.fetchall() + assert records + assert {record[0] for record in records} == {"geom"} + + # Verify round-trip integrity + for resource in [ + "geodata_default_wkt", + "geodata_3857_wkt", + "geodata_2163_wkt", + "geodata_default_wkb_hex", + "geodata_3857_wkb_hex", + "geodata_2163_wkb_hex", + "geodata_default_csv_wkt", + "geodata_3857_csv_wkt", + "geodata_2163_csv_wkt", + "geodata_default_csv_wkb_hex", + "geodata_3857_csv_wkb_hex", + "geodata_2163_csv_wkb_hex", + ]: + srid = 4326 if resource.startswith("geodata_default") else int(resource.split("_")[1]) + + query = f""" + SELECT type, ST_AsText(geom) as wkt, ST_SRID(geom) as srid, ST_AsBinary(geom) as wkb + FROM {c.make_qualified_table_name(resource)} + """ + + with c.execute_query(query) as cur: + results = cur.fetchall() + + def get_format(column_name): + if column_name.endswith("wkb_hex"): + return "wkb_hex" + return column_name.split("_")[-1] + + original_geometries = generate_sample_geometry_records(get_format(resource)) + + for result in results: + db_type, db_wkt, db_srid, db_wkb = result + orig_geom = next((g for g in original_geometries if g["type"] == db_type), None) + + assert orig_geom is not None, f"No matching original geometry found for {db_type}" + + assert ( + db_srid == srid + ), f"SRID mismatch for {db_type}: expected {srid}, got {db_srid}" + + if "Empty" in db_type: + assert wkt.loads(db_wkt).is_empty, f"Expected empty geometry for {db_type}" + else: + if "_wkt" in db_type: + orig_geom = wkt.loads(orig_geom["geom"]) + db_geom = wkt.loads(db_wkt) + elif "_wkb_hex" in db_type: + orig_geom = wkb.loads(bytes.fromhex(orig_geom["geom"])) + db_geom = wkb.loads(bytes(db_wkb)) + + tolerance = 1e-8 + if isinstance(orig_geom, LinearRing): + # LinearRing geometries are converted to Polygons for PostGIS compatibility. + db_geom = Polygon(orig_geom) + assert LinearRing(db_geom.exterior.coords).equals_exact( + orig_geom, tolerance + ), f"Geometry mismatch for {db_type}" + else: + assert orig_geom.equals_exact( # type: ignore[attr-defined] + db_geom, tolerance + ), f"Geometry mismatch for {db_type}" diff --git a/tests/load/postgres/utils.py b/tests/load/postgres/utils.py new file mode 100644 index 0000000000..b03a6b5096 --- /dev/null +++ b/tests/load/postgres/utils.py @@ -0,0 +1,68 @@ +from typing import List + +from shapely import ( # type: ignore + Point, + LineString, + Polygon, + MultiPoint, + MultiLineString, + MultiPolygon, + GeometryCollection, + LinearRing, +) +from shapely.wkb import dumps as wkb_dumps # type: ignore + +from dlt.common.typing import DictStrStr + + +def generate_sample_geometry_records(geometry_type: str) -> List[DictStrStr]: + """ + Generate sample geometry records including WKT and WKB representations. + + Returns: + A list of dictionaries, each containing a geometry type, + its Well-Known Text (WKT), and Well-Known Binary (WKB) representation. + """ + geometries = [ + ("Point", Point(1, 1)), + ("LineString", LineString([(0, 0), (1, 1), (2, 2)])), + ("Polygon", Polygon([(0, 0), (1, 0), (1, 1), (0, 1), (0, 0)])), + ("MultiPoint", MultiPoint([(0, 0), (1, 1), (2, 2)])), + ("MultiLineString", MultiLineString([((0, 0), (1, 1)), ((2, 2), (3, 3))])), + ( + "MultiPolygon", + MultiPolygon( + [ + Polygon([(0, 0), (1, 0), (1, 1), (0, 1), (0, 0)]), + Polygon([(2, 2), (3, 2), (3, 3), (2, 3), (2, 2)]), + ] + ), + ), + ( + "GeometryCollection", + GeometryCollection([Point(1, 1), LineString([(0, 0), (1, 1), (2, 2)])]), + ), + ( + "ComplexPolygon", + Polygon( + [(0, 0), (10, 0), (10, 10), (0, 10), (0, 0)], + [[(4, 4), (6, 4), (6, 6), (4, 6), (4, 4)]], + ), + ), + ("EmptyPoint", Point()), + ("EmptyLineString", LineString()), + ("EmptyPolygon", Polygon()), + ("EmptyMultiPoint", MultiPoint()), + ("EmptyMultiLineString", MultiLineString()), + ("EmptyMultiPolygon", MultiPolygon()), + ("EmptyGeometryCollection", GeometryCollection()), + ] + + # LinearRing only works with wkb types + if geometry_type == "wkb": + geometries += [("LinearRing", LinearRing([(0, 0), (1, 0), (1, 1), (0, 1), (0, 0)]))] + + return [ + {"type": f"{name}_{geometry_type}", "geom": getattr(geom, geometry_type)} + for name, geom in geometries + ] diff --git a/tests/load/qdrant/utils.py b/tests/load/qdrant/utils.py index e96e06be87..8a3b37dd48 100644 --- a/tests/load/qdrant/utils.py +++ b/tests/load/qdrant/utils.py @@ -61,6 +61,5 @@ def has_collections(client): if has_collections(client): client.drop_storage() - p._wipe_working_folder() # deactivate context Container()[PipelineContext].deactivate() diff --git a/tests/load/redshift/test_redshift_client.py b/tests/load/redshift/test_redshift_client.py index b60c6a8956..ef0acb33a4 100644 --- a/tests/load/redshift/test_redshift_client.py +++ b/tests/load/redshift/test_redshift_client.py @@ -21,7 +21,7 @@ from dlt.destinations.impl.redshift.redshift import RedshiftClient, psycopg2 from tests.common.utils import COMMON_TEST_CASES_PATH -from tests.utils import TEST_STORAGE_ROOT, autouse_test_storage, skipifpypy +from tests.utils import TEST_STORAGE_ROOT, skipifpypy from tests.load.utils import expect_load_file, prepare_table, yield_client_with_storage # mark all tests as essential, do not remove diff --git a/tests/load/snowflake/test_snowflake_configuration.py b/tests/load/snowflake/test_snowflake_configuration.py index 21973025c7..265a6a0935 100644 --- a/tests/load/snowflake/test_snowflake_configuration.py +++ b/tests/load/snowflake/test_snowflake_configuration.py @@ -4,6 +4,8 @@ from urllib3.util import parse_url from dlt.common.configuration.utils import add_config_to_env +from dlt.common.exceptions import TerminalValueError +from dlt.destinations.impl.snowflake.snowflake import SnowflakeLoadJob from tests.utils import TEST_DICT_CONFIG_PROVIDER pytest.importorskip("snowflake") @@ -271,3 +273,27 @@ def test_snowflake_configuration() -> None: explicit_value="snowflake://user1:pass@host1/db1?warehouse=warehouse1&role=role1", ) assert SnowflakeClientConfiguration(credentials=c).fingerprint() == digest128("host1") + + +def test_snowflake_azure_converter() -> None: + with pytest.raises(TerminalValueError): + SnowflakeLoadJob.ensure_snowflake_azure_url("az://dlt-ci-test-bucket") + + azure_url = SnowflakeLoadJob.ensure_snowflake_azure_url("az://dlt-ci-test-bucket", "my_account") + assert azure_url == "azure://my_account.blob.core.windows.net/dlt-ci-test-bucket" + + azure_url = SnowflakeLoadJob.ensure_snowflake_azure_url( + "az://dlt-ci-test-bucket/path/to/file.parquet", "my_account" + ) + assert ( + azure_url + == "azure://my_account.blob.core.windows.net/dlt-ci-test-bucket/path/to/file.parquet" + ) + + azure_url = SnowflakeLoadJob.ensure_snowflake_azure_url( + "abfss://dlt-ci-test-bucket@my_account.blob.core.windows.net/path/to/file.parquet" + ) + assert ( + azure_url + == "azure://my_account.blob.core.windows.net/dlt-ci-test-bucket/path/to/file.parquet" + ) diff --git a/tests/load/sources/sql_database/test_helpers.py b/tests/load/sources/sql_database/test_helpers.py index 4748f226a9..def5430146 100644 --- a/tests/load/sources/sql_database/test_helpers.py +++ b/tests/load/sources/sql_database/test_helpers.py @@ -1,10 +1,8 @@ import pytest - import dlt from dlt.common.typing import TDataItem - from dlt.common.exceptions import MissingDependencyException try: @@ -246,6 +244,86 @@ class MockIncremental: assert query.compare(expected) +@pytest.mark.parametrize("backend", ["sqlalchemy", "pyarrow", "pandas", "connectorx"]) +def test_cursor_path_field_name_with_a_special_chars( + sql_source_db: SQLAlchemySourceDB, backend: TableBackend +) -> None: + """Test that a field name with special characters in cursor_path is handled correctly.""" + table = sql_source_db.get_table("chat_message") + + # Add a mock column with a special character + special_field_name = "id$field" + if special_field_name not in table.c: + table.append_column(sa.Column(special_field_name, sa.String)) + + class MockIncremental: + cursor_path = "'id$field'" + last_value = None + end_value = None + row_order = None + on_cursor_value_missing = None + + # Should not raise any exception + loader = TableLoader( + sql_source_db.engine, + backend, + table, + table_to_columns(table), + incremental=MockIncremental(), # type: ignore[arg-type] + ) + assert loader.cursor_column == table.c[special_field_name] + + +@pytest.mark.parametrize("backend", ["sqlalchemy", "pyarrow", "pandas", "connectorx"]) +def test_cursor_path_multiple_fields( + sql_source_db: SQLAlchemySourceDB, backend: TableBackend +) -> None: + """Test that a cursor_path with multiple fields raises a ValueError.""" + table = sql_source_db.get_table("chat_message") + + class MockIncremental: + cursor_path = "created_at,updated_at" + last_value = None + end_value = None + row_order = None + on_cursor_value_missing = None + + with pytest.raises(ValueError) as excinfo: + TableLoader( + sql_source_db.engine, + backend, + table, + table_to_columns(table), + incremental=MockIncremental(), # type: ignore[arg-type] + ) + assert "must be a simple column name" in str(excinfo.value) + + +@pytest.mark.parametrize("backend", ["sqlalchemy", "pyarrow", "pandas", "connectorx"]) +def test_cursor_path_complex_expression( + sql_source_db: SQLAlchemySourceDB, backend: TableBackend +) -> None: + """Test that a complex JSONPath expression in cursor_path raises a ValueError.""" + table = sql_source_db.get_table("chat_message") + + class MockIncremental: + cursor_path = "$.users[0].id" + last_value = None + end_value = None + row_order = None + on_cursor_value_missing = None + + with pytest.raises(ValueError) as excinfo: + TableLoader( + sql_source_db.engine, + backend, + table, + table_to_columns(table), + incremental=MockIncremental(), # type: ignore[arg-type] + ) + assert "must be a simple column name" in str(excinfo.value) + + def mock_json_column(field: str) -> TDataItem: """""" import pyarrow as pa 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 069ebd7841..9079638586 100644 --- a/tests/load/sources/sql_database/test_sql_database_source.py +++ b/tests/load/sources/sql_database/test_sql_database_source.py @@ -11,8 +11,8 @@ from dlt.common.schema.typing import TColumnSchema, TSortOrder, TTableSchemaColumns from dlt.common.utils import uniq_id -from dlt.extract.exceptions import ResourceExtractionError +from dlt.extract.exceptions import ResourceExtractionError from dlt.sources import DltResource from tests.pipeline.utils import ( @@ -30,6 +30,7 @@ TableBackend, sql_database, sql_table, + remove_nullability_adapter, ) from dlt.sources.sql_database.helpers import unwrap_json_connector_x from tests.load.sources.sql_database.sql_source import SQLAlchemySourceDB @@ -107,6 +108,34 @@ def test_pass_engine_credentials(sql_source_db: SQLAlchemySourceDB) -> None: assert len(list(table)) == sql_source_db.table_infos["chat_message"]["row_count"] +def test_engine_adapter_callback(sql_source_db: SQLAlchemySourceDB) -> None: + from dlt.common.libs.sql_alchemy import Engine + + adapter_calls: int = 0 + + def set_serializable(engine: Engine) -> Engine: + nonlocal adapter_calls + + engine.execution_options(isolation_level="SERIALIZABLE") + adapter_calls += 1 + return engine + + # verify database + database = sql_database( + sql_source_db.engine.url.render_as_string(False), + schema=sql_source_db.schema, + table_names=["chat_message"], + engine_adapter_callback=set_serializable, + ) + assert adapter_calls == 2 + + assert len(list(database)) == sql_source_db.table_infos["chat_message"]["row_count"] + + # verify table + table = sql_table(sql_source_db.engine, table="chat_message", schema=sql_source_db.schema) + assert len(list(table)) == sql_source_db.table_infos["chat_message"]["row_count"] + + def test_named_sql_table_config(sql_source_db: SQLAlchemySourceDB) -> None: # set the credentials per table name os.environ["SOURCES__SQL_DATABASE__CHAT_MESSAGE__CREDENTIALS"] = ( @@ -176,6 +205,146 @@ def test_general_sql_database_config(sql_source_db: SQLAlchemySourceDB) -> None: assert len(list(sql_database(schema=sql_source_db.schema).with_resources("app_user"))) > 0 +@pytest.mark.parametrize("backend", ["sqlalchemy", "pandas", "pyarrow"]) +@pytest.mark.parametrize("add_new_columns", [True, False]) +def test_text_query_adapter( + sql_source_db: SQLAlchemySourceDB, backend: TableBackend, add_new_columns: bool +) -> None: + from dlt.common.libs.sql_alchemy import Table, sqltypes, sa, Engine, TextClause + from dlt.sources.sql_database.helpers import SelectAny + from dlt.extract.incremental import Incremental + + def new_columns(table: Table) -> None: + required_columns = [ + ("add_int", sqltypes.BigInteger, {"nullable": True}), + ("add_text", sqltypes.Text, {"default": None, "nullable": True}), + ] + for col_name, col_type, col_kwargs in required_columns: + if col_name not in table.c: + table.append_column(sa.Column(col_name, col_type, **col_kwargs)) + + last_query: str = None + + def query_adapter( + query: SelectAny, table: Table, incremental: Optional[Incremental[Any]], engine: Engine + ) -> TextClause: + nonlocal last_query + + if incremental and incremental.start_value is not None: + t_query = sa.text( + f"SELECT *, 1 as add_int, 'const' as add_text FROM {table.fullname} WHERE" + f" {incremental.cursor_path} > :start_value" + ).bindparams(**{"start_value": incremental.start_value}) + else: + t_query = sa.text(f"SELECT *, 1 as add_int, 'const' as add_text FROM {table.fullname}") + + last_query = str(t_query) + return t_query + + read_table = sql_table( + table="chat_channel", + credentials=sql_source_db.credentials, + schema=sql_source_db.schema, + reflection_level="full", + backend=backend, + table_adapter_callback=new_columns if add_new_columns else None, + query_adapter_callback=query_adapter, + incremental=dlt.sources.incremental("updated_at"), + ) + + pipeline = make_pipeline("duckdb") + info = pipeline.run(read_table) + assert_load_info(info) + assert "chat_channel" in last_query + assert "WHERE" not in last_query + + chn_count = load_table_counts(pipeline, "chat_channel")["chat_channel"] + assert chn_count > 0 + + chat_channel_schema = pipeline.default_schema.get_table("chat_channel") + # print(pipeline.default_schema.to_pretty_yaml()) + assert "add_int" in chat_channel_schema["columns"] + assert chat_channel_schema["columns"]["add_int"]["data_type"] == "bigint" + assert chat_channel_schema["columns"]["add_text"]["data_type"] == "text" + + info = pipeline.run(read_table) + assert "WHERE updated_at > :start_value" in last_query + # no msgs were loaded, incremental got correctly rendered + assert load_table_counts(pipeline, "chat_channel")["chat_channel"] == chn_count + + +@pytest.mark.parametrize("backend", ["sqlalchemy", "pandas", "pyarrow"]) +def test_computed_column(sql_source_db: SQLAlchemySourceDB, backend: TableBackend) -> None: + from dlt.common.libs.sql_alchemy import Table, sa, sqltypes + from dlt.sources.sql_database.helpers import SelectAny + + def add_max_timestamp(table: Table) -> SelectAny: + computed_max_timestamp = sa.sql.type_coerce( + sa.func.greatest(table.c.created_at, table.c.updated_at), + sqltypes.DateTime, + ).label("max_timestamp") + subquery = sa.select(*table.c, computed_max_timestamp).subquery() + return subquery + + read_table = sql_table( + table="chat_message", + credentials=sql_source_db.credentials, + schema=sql_source_db.schema, + reflection_level="full", + backend=backend, + table_adapter_callback=add_max_timestamp, + incremental=dlt.sources.incremental("max_timestamp"), + ) + + pipeline = make_pipeline("duckdb") + info = pipeline.run(read_table) + assert_load_info(info) + + msg_count = load_table_counts(pipeline, "chat_message")["chat_message"] + assert msg_count > 0 + + chat_channel_schema = pipeline.default_schema.get_table("chat_message") + # print(pipeline.default_schema.to_pretty_yaml()) + assert "max_timestamp" in chat_channel_schema["columns"] + assert chat_channel_schema["columns"]["max_timestamp"]["data_type"] == "timestamp" + + info = pipeline.run(read_table) + # no msgs were loaded, incremental got correctly rendered + assert load_table_counts(pipeline, "chat_message")["chat_message"] == msg_count + + +def test_remove_nullability(sql_source_db: SQLAlchemySourceDB) -> None: + read_table = sql_table( + table="chat_message", + credentials=sql_source_db.credentials, + schema=sql_source_db.schema, + reflection_level="full_with_precision", + table_adapter_callback=remove_nullability_adapter, + ) + table_schema = read_table.compute_table_schema() + for column in table_schema["columns"].values(): + assert "nullability" not in column + + # also works for subquery + def make_subquery(table): + return remove_nullability_adapter(table.select().subquery()) + + read_table = sql_table( + table="chat_message", + credentials=sql_source_db.credentials, + schema=sql_source_db.schema, + reflection_level="full_with_precision", + table_adapter_callback=make_subquery, + ) + + table_schema = read_table.compute_table_schema() + for column in table_schema["columns"].values(): + assert "nullability" not in column + + data = list(read_table) + assert len(data) == sql_source_db.table_infos["chat_message"]["row_count"] + + @pytest.mark.parametrize("backend", ["sqlalchemy", "pandas", "pyarrow", "connectorx"]) @pytest.mark.parametrize("row_order", ["asc", "desc", None]) @pytest.mark.parametrize("last_value_func", [min, max, lambda x: max(x)]) @@ -1009,7 +1178,10 @@ def test_sql_table_included_columns( def test_query_adapter_callback( sql_source_db: SQLAlchemySourceDB, backend: TableBackend, standalone_resource: bool ) -> None: - def query_adapter_callback(query, table): + from dlt.sources.sql_database.helpers import SelectAny + from dlt.common.libs.sql_alchemy import Table + + def query_adapter_callback(query: SelectAny, table: Table) -> SelectAny: if table.name == "chat_channel": # Only select active channels return query.where(table.c.active.is_(True)) @@ -1021,7 +1193,6 @@ def query_adapter_callback(query, table): schema=sql_source_db.schema, reflection_level="full", backend=backend, - query_adapter_callback=query_adapter_callback, ) if standalone_resource: @@ -1031,11 +1202,13 @@ def dummy_source(): yield sql_table( **common_kwargs, # type: ignore[arg-type] table="chat_channel", + query_adapter_callback=query_adapter_callback, ) yield sql_table( **common_kwargs, # type: ignore[arg-type] table="chat_message", + query_adapter_callback=query_adapter_callback, ) source = dummy_source() @@ -1043,6 +1216,7 @@ def dummy_source(): source = sql_database( **common_kwargs, # type: ignore[arg-type] table_names=["chat_message", "chat_channel"], + query_adapter_callback=query_adapter_callback, ) pipeline = make_pipeline("duckdb") diff --git a/tests/load/test_job_client.py b/tests/load/test_job_client.py index 9f64722a1e..6f699436b3 100644 --- a/tests/load/test_job_client.py +++ b/tests/load/test_job_client.py @@ -36,7 +36,7 @@ from dlt.common.time import ensure_pendulum_datetime from tests.cases import table_update_and_row, assert_all_data_types_row -from tests.utils import TEST_STORAGE_ROOT, autouse_test_storage +from tests.utils import TEST_STORAGE_ROOT from tests.common.utils import load_json_case from tests.load.utils import ( TABLE_UPDATE, diff --git a/tests/load/test_read_interfaces.py b/tests/load/test_read_interfaces.py index c6019ecf2d..1a9c8a383b 100644 --- a/tests/load/test_read_interfaces.py +++ b/tests/load/test_read_interfaces.py @@ -10,6 +10,7 @@ from typing import List from functools import reduce +from dlt.common.storages.file_storage import FileStorage from tests.load.utils import ( destinations_configs, DestinationTestConfiguration, @@ -18,7 +19,7 @@ MEMORY_BUCKET, ) from dlt.destinations import filesystem -from tests.utils import TEST_STORAGE_ROOT +from tests.utils import TEST_STORAGE_ROOT, clean_test_storage from dlt.common.destination.reference import TDestinationReferenceArg from dlt.destinations.dataset import ReadableDBAPIDataset, ReadableRelationUnknownColumnException from tests.load.utils import drop_pipeline_data @@ -48,8 +49,14 @@ def _expected_chunk_count(p: Pipeline) -> List[int]: return [_chunk_size(p), _total_records(p) - _chunk_size(p)] +# this also disables autouse_test_storage on function level which destroys some tests here @pytest.fixture(scope="session") -def populated_pipeline(request) -> Any: +def autouse_test_storage() -> FileStorage: + return clean_test_storage() + + +@pytest.fixture(scope="session") +def populated_pipeline(request, autouse_test_storage) -> Any: """fixture that returns a pipeline object populated with the example data""" destination_config = cast(DestinationTestConfiguration, request.param) @@ -251,6 +258,71 @@ def test_db_cursor_access(populated_pipeline: Pipeline) -> None: assert set(ids) == set(range(total_records)) +@pytest.mark.no_load +@pytest.mark.essential +@pytest.mark.parametrize( + "populated_pipeline", + configs, + indirect=True, + ids=lambda x: x.name, +) +def test_ibis_dataset_access(populated_pipeline: Pipeline) -> None: + # NOTE: we could generalize this with a context for certain deps + import subprocess + + subprocess.check_call( + ["pip", "install", "ibis-framework[duckdb,postgres,bigquery,snowflake,mssql,clickhouse]"] + ) + + from dlt.common.libs.ibis import SUPPORTED_DESTINATIONS + + # check correct error if not supported + if populated_pipeline.destination.destination_type not in SUPPORTED_DESTINATIONS: + with pytest.raises(NotImplementedError): + populated_pipeline._dataset().ibis() + return + + total_records = _total_records(populated_pipeline) + ibis_connection = populated_pipeline._dataset().ibis() + + map_i = lambda x: x + if populated_pipeline.destination.destination_type == "dlt.destinations.snowflake": + map_i = lambda x: x.upper() + + dataset_name = map_i(populated_pipeline.dataset_name) + table_like_statement = None + table_name_prefix = "" + addtional_tables = [] + + # clickhouse has no datasets, but table prefixes and a sentinel table + if populated_pipeline.destination.destination_type == "dlt.destinations.clickhouse": + table_like_statement = dataset_name + "." + table_name_prefix = dataset_name + "___" + dataset_name = None + addtional_tables = ["dlt_sentinel_table"] + + add_table_prefix = lambda x: table_name_prefix + x + + # just do a basic check to see wether ibis can connect + assert set(ibis_connection.list_tables(database=dataset_name, like=table_like_statement)) == { + add_table_prefix(map_i(x)) + for x in ( + [ + "_dlt_loads", + "_dlt_pipeline_state", + "_dlt_version", + "double_items", + "items", + "items__children", + ] + + addtional_tables + ) + } + + items_table = ibis_connection.table(add_table_prefix(map_i("items")), database=dataset_name) + assert items_table.count().to_pandas() == total_records + + @pytest.mark.no_load @pytest.mark.essential @pytest.mark.parametrize( @@ -379,6 +451,34 @@ def test_column_selection(populated_pipeline: Pipeline) -> None: arrow_table = table_relationship.select("unknown_column").head().arrow() +@pytest.mark.no_load +@pytest.mark.essential +@pytest.mark.parametrize( + "populated_pipeline", + configs, + indirect=True, + ids=lambda x: x.name, +) +def test_schema_arg(populated_pipeline: Pipeline) -> None: + """Simple test to ensure schemas may be selected via schema arg""" + + # if there is no arg, the defautl schema is used + dataset = populated_pipeline._dataset() + assert dataset.schema.name == populated_pipeline.default_schema_name + assert "items" in dataset.schema.tables + + # setting a different schema name will try to load that schema, + # not find one and create an empty schema with that name + dataset = populated_pipeline._dataset(schema="unknown_schema") + assert dataset.schema.name == "unknown_schema" + assert "items" not in dataset.schema.tables + + # providing the schema name of the right schema will load it + dataset = populated_pipeline._dataset(schema=populated_pipeline.default_schema_name) + assert dataset.schema.name == populated_pipeline.default_schema_name + assert "items" in dataset.schema.tables + + @pytest.mark.no_load @pytest.mark.essential @pytest.mark.parametrize( @@ -422,7 +522,7 @@ def test_standalone_dataset(populated_pipeline: Pipeline) -> None: ), ) assert "items" not in dataset.schema.tables - assert dataset.schema.name == populated_pipeline.dataset_name + assert dataset.schema.name == "wrong_schema_name" # check that schema is loaded if no schema name given dataset = cast( diff --git a/tests/load/test_sql_client.py b/tests/load/test_sql_client.py index 05c10a900f..ee48222da9 100644 --- a/tests/load/test_sql_client.py +++ b/tests/load/test_sql_client.py @@ -22,7 +22,7 @@ from dlt.destinations.typing import TNativeConn from dlt.common.time import ensure_pendulum_datetime, to_py_datetime -from tests.utils import TEST_STORAGE_ROOT, autouse_test_storage +from tests.utils import TEST_STORAGE_ROOT from tests.load.utils import ( yield_client_with_storage, prepare_table, diff --git a/tests/load/weaviate/utils.py b/tests/load/weaviate/utils.py index b391c2fa38..b98b55fcfa 100644 --- a/tests/load/weaviate/utils.py +++ b/tests/load/weaviate/utils.py @@ -95,6 +95,5 @@ def schema_has_classes(client): if schema_has_classes(client): client.drop_storage() - p._wipe_working_folder() # deactivate context Container()[PipelineContext].deactivate() diff --git a/tests/pipeline/cases/github_pipeline/github_rev.py b/tests/pipeline/cases/github_pipeline/github_rev.py new file mode 100644 index 0000000000..4ebe3048f4 --- /dev/null +++ b/tests/pipeline/cases/github_pipeline/github_rev.py @@ -0,0 +1,26 @@ +import dlt + + +@dlt.source +def github(): + @dlt.resource( + table_name="issues__2", + primary_key="id", + ) + def load_issues(): + # return data with path separators + yield [ + { + "id": 100, + "issue__id": 10, + } + ] + + return load_issues + + +if __name__ == "__main__": + p = dlt.pipeline("dlt_github_pipeline", destination="duckdb", dataset_name="github_3") + github_source = github() + info = p.run(github_source) + print(info) diff --git a/tests/pipeline/test_dlt_versions.py b/tests/pipeline/test_dlt_versions.py index a3d8b489c9..fbd4d412b3 100644 --- a/tests/pipeline/test_dlt_versions.py +++ b/tests/pipeline/test_dlt_versions.py @@ -484,3 +484,59 @@ def test_scd2_pipeline_update(test_storage: FileStorage) -> None: assert len(issues_retired) == 1 assert issues_retired[0][0] == 6272 # print(pipeline.default_schema.to_pretty_yaml()) + + +def test_normalize_path_separator_legacy_behavior(test_storage: FileStorage) -> None: + """Pre 1.4.1 normalized identifiers with path separators into single underscore, + this behavior must be preserved if the schema is updated. + """ + shutil.copytree("tests/pipeline/cases/github_pipeline", TEST_STORAGE_ROOT, dirs_exist_ok=True) + + # execute in test storage + with set_working_dir(TEST_STORAGE_ROOT): + # store dlt data in test storage (like patch_home_dir) + with custom_environ({DLT_DATA_DIR: dlt.current.run().data_dir}): + # save database outside of pipeline dir + with custom_environ( + {"DESTINATION__DUCKDB__CREDENTIALS": "duckdb:///test_github_3.duckdb"} + ): + venv_dir = tempfile.mkdtemp() + # create virtual env with (0.3.0) before the current schema upgrade + with Venv.create(venv_dir, ["dlt[duckdb]==0.3.0"]) as venv: + venv._install_deps(venv.context, ["duckdb" + "==" + pkg_version("duckdb")]) + try: + print( + venv.run_script("../tests/pipeline/cases/github_pipeline/github_rev.py") + ) + except CalledProcessError as cpe: + print(f"script stdout: {cpe.stdout}") + print(f"script stderr: {cpe.stderr}") + raise + + venv = Venv.restore_current() + # load same data again + try: + print(venv.run_script("../tests/pipeline/cases/github_pipeline/github_rev.py")) + except CalledProcessError as cpe: + print(f"script stdout: {cpe.stdout}") + print(f"script stderr: {cpe.stderr}") + raise + pipeline = dlt.attach(GITHUB_PIPELINE_NAME) + print(pipeline.default_schema.to_pretty_yaml()) + # migration set the backward compat flag + assert ( + pipeline.default_schema._normalizers_config["use_break_path_on_normalize"] + is False + ) + # make sure that schema didn't change + assert pipeline.default_schema.data_table_names() == ["issues_2"] + table_ = pipeline.default_schema.tables["issues_2"] + assert set(table_["columns"].keys()) == { + "id", + "issue_id", + "_dlt_id", + "_dlt_load_id", + } + # datasets must be the same + data_ = pipeline._dataset().issues_2.select("issue_id", "id").fetchall() + print(data_) diff --git a/tests/normalize/test_max_nesting.py b/tests/pipeline/test_max_nesting.py similarity index 100% rename from tests/normalize/test_max_nesting.py rename to tests/pipeline/test_max_nesting.py diff --git a/tests/pipeline/test_pipeline.py b/tests/pipeline/test_pipeline.py index 8cac5fafde..b6b82e4a7f 100644 --- a/tests/pipeline/test_pipeline.py +++ b/tests/pipeline/test_pipeline.py @@ -64,6 +64,8 @@ many_delayed, ) +from dlt.destinations.dataset import get_destination_client_initial_config + DUMMY_COMPLETE = dummy(completed_prob=1) # factory set up to complete jobs @@ -417,16 +419,15 @@ def test_destination_staging_config(environment: Any) -> None: fs_dest = filesystem("file:///testing-bucket") p = dlt.pipeline( pipeline_name="staging_pipeline", - destination=redshift(credentials="redshift://loader:loader@localhost:5432/dlt_data"), + destination=dummy(), staging=fs_dest, ) schema = Schema("foo") p._inject_schema(schema) - initial_config = p._get_destination_client_initial_config(p.staging, as_staging=True) - staging_config = fs_dest.configuration(initial_config) # type: ignore[arg-type] - # Ensure that as_staging flag is set in the final resolved conifg - assert staging_config.as_staging_destination is True + _, staging_client = p._get_destination_clients() + + assert staging_client.config.as_staging_destination is True # type: ignore def test_destination_factory_defaults_resolve_from_config(environment: Any) -> None: @@ -450,7 +451,9 @@ def test_destination_credentials_in_factory(environment: Any) -> None: p = dlt.pipeline(pipeline_name="dummy_pipeline", destination=redshift_dest) - initial_config = p._get_destination_client_initial_config(p.destination) + initial_config = get_destination_client_initial_config( + p.destination, "some_schema_name", p.dataset_name + ) dest_config = redshift_dest.configuration(initial_config) # type: ignore[arg-type] # Explicit factory arg supersedes config assert dest_config.credentials.database == "other_db" @@ -458,7 +461,9 @@ def test_destination_credentials_in_factory(environment: Any) -> None: redshift_dest = redshift() p = dlt.pipeline(pipeline_name="dummy_pipeline", destination=redshift_dest) - initial_config = p._get_destination_client_initial_config(p.destination) + initial_config = get_destination_client_initial_config( + p.destination, "some_schema_name", p.dataset_name + ) dest_config = redshift_dest.configuration(initial_config) # type: ignore[arg-type] assert dest_config.credentials.database == "some_db" @@ -1728,6 +1733,111 @@ def nested_resource(): assert pipeline.last_trace.last_normalize_info.row_counts["flattened_dict__values"] == 4 +def test_column_name_with_break_path() -> None: + """Tests how normalization behaves for names with break path ie __ + all the names must be idempotent + """ + pipeline = dlt.pipeline(destination="duckdb", pipeline_name="breaking") + info = pipeline.run( + [{"example_custom_field__c": "custom", "reg_c": "c"}], table_name="custom__path" + ) + assert_load_info(info) + # table name was preserved + table = pipeline.default_schema.get_table("custom__path") + assert pipeline.default_schema.data_table_names() == ["custom__path"] + # column name was preserved + assert table["columns"]["example_custom_field__c"]["data_type"] == "text" + assert set(table["columns"]) == {"example_custom_field__c", "reg_c", "_dlt_id", "_dlt_load_id"} + + # get data + assert_data_table_counts(pipeline, {"custom__path": 1}) + # get data via dataset with dbapi + data_ = pipeline._dataset().custom__path[["example_custom_field__c", "reg_c"]].fetchall() + assert data_ == [("custom", "c")] + + +def test_column_name_with_break_path_legacy() -> None: + """Tests how normalization behaves for names with break path ie __ + in legacy mode table and column names were normalized as single identifier + """ + os.environ["SCHEMA__USE_BREAK_PATH_ON_NORMALIZE"] = "False" + pipeline = dlt.pipeline(destination="duckdb", pipeline_name="breaking") + info = pipeline.run( + [{"example_custom_field__c": "custom", "reg_c": "c"}], table_name="custom__path" + ) + assert_load_info(info) + # table name was contracted + table = pipeline.default_schema.get_table("custom_path") + assert pipeline.default_schema.data_table_names() == ["custom_path"] + # column name was contracted + assert table["columns"]["example_custom_field_c"]["data_type"] == "text" + assert set(table["columns"]) == {"example_custom_field_c", "reg_c", "_dlt_id", "_dlt_load_id"} + + # get data + assert_data_table_counts(pipeline, {"custom_path": 1}) + # get data via dataset with dbapi + data_ = pipeline._dataset().custom_path[["example_custom_field_c", "reg_c"]].fetchall() + assert data_ == [("custom", "c")] + + +def test_column_hint_with_break_path() -> None: + """Up form the v 1.4.1 name normalizer is idempotent on break path""" + now = cast(pendulum.DateTime, pendulum.parse("2024-11-29T10:10")) + + @dlt.resource( + name="flattened__dict", columns=[{"name": "value__timestamp", "data_type": "timestamp"}] + ) + def flattened_dict(): + for delta in range(4): + yield { + "delta": delta, + "value": {"timestamp": now.timestamp() + delta}, + } + + pipeline = dlt.pipeline(destination="duckdb") + info = pipeline.run(flattened_dict()) + assert_load_info(info) + + assert pipeline.default_schema.data_table_names() == ["flattened__dict"] + table = pipeline.default_schema.get_table("flattened__dict") + assert set(table["columns"]) == {"delta", "value__timestamp", "_dlt_id", "_dlt_load_id"} + assert table["columns"]["value__timestamp"]["data_type"] == "timestamp" + + # make sure data is there + data_ = pipeline._dataset().flattened__dict[["delta", "value__timestamp"]].limit(1).fetchall() + assert data_ == [(0, now)] + + +def test_column_hint_with_break_path_legacy() -> None: + """Up form the v 1.4.1 name normalizer is idempotent on break path""" + + os.environ["SCHEMA__USE_BREAK_PATH_ON_NORMALIZE"] = "False" + now = cast(pendulum.DateTime, pendulum.parse("2024-11-29T10:10")) + + @dlt.resource( + name="flattened__dict", columns=[{"name": "value__timestamp", "data_type": "timestamp"}] + ) + def flattened_dict(): + for delta in range(4): + yield { + "delta": delta, + "value": {"timestamp": now.timestamp() + delta}, + } + + pipeline = dlt.pipeline(destination="duckdb") + info = pipeline.run(flattened_dict()) + assert_load_info(info) + # table name contracted + assert pipeline.default_schema.data_table_names() == ["flattened_dict"] + table = pipeline.default_schema.get_table("flattened_dict") + # hint applied + assert set(table["columns"]) == {"delta", "value__timestamp", "_dlt_id", "_dlt_load_id"} + assert table["columns"]["value__timestamp"]["data_type"] == "timestamp" + # make sure data is there + data_ = pipeline._dataset().flattened_dict[["delta", "value__timestamp"]].limit(1).fetchall() + assert data_ == [(0, now)] + + def test_empty_rows_are_included() -> None: """Empty rows where all values are `None` or empty dicts create rows in the dataset with `NULL` in all columns diff --git a/tests/sources/helpers/rest_client/test_client.py b/tests/sources/helpers/rest_client/test_client.py index 488d7ef525..36fe009b93 100644 --- a/tests/sources/helpers/rest_client/test_client.py +++ b/tests/sources/helpers/rest_client/test_client.py @@ -7,6 +7,7 @@ from requests import PreparedRequest, Request, Response from requests.auth import AuthBase from requests.exceptions import HTTPError +import requests_mock from dlt.common import logger from dlt.common.typing import TSecretStrValue @@ -512,3 +513,24 @@ def test_request_kwargs(self, mocker) -> None: "timeout": 432, "allow_redirects": False, } + + @requests_mock.Mocker(kw="mock") + def test_overwrite_path(self, mocker, **kwargs) -> None: + expected = {"foo": "bar"} + kwargs["mock"].get("https://completely.different/endpoint", json=expected) + rest_client = RESTClient( + base_url="https://api.example.com", + ) + response = rest_client.get("https://completely.different/endpoint") + assert response.json() == expected + + @requests_mock.Mocker(kw="mock") + def test_overwrite_path_ignores_different_protocol(self, mocker, **kwargs) -> None: + expected = {"foo": "bar"} + base_url = "https://api.example.com" + kwargs["mock"].get(f"{base_url}/my://protocol", json=expected) + rest_client = RESTClient( + base_url=base_url, + ) + response = rest_client.get("my://protocol") + assert response.json() == expected diff --git a/tests/sources/rest_api/configurations/source_configs.py b/tests/sources/rest_api/configurations/source_configs.py index 705a42637c..ff58fee0fb 100644 --- a/tests/sources/rest_api/configurations/source_configs.py +++ b/tests/sources/rest_api/configurations/source_configs.py @@ -395,6 +395,18 @@ def repositories(): repositories(), ], }, + { + "client": {"base_url": "https://github.com/api/v2"}, + "resources": [ + { + "name": "issues", + "endpoint": { + "path": "user/repos", + "auth": HttpBasicAuth("", "BASIC_AUTH_TOKEN"), + }, + } + ], + }, ] diff --git a/tests/sources/rest_api/integration/test_response_actions.py b/tests/sources/rest_api/integration/test_response_actions.py index 1ec8058a86..e4fcc32f8f 100644 --- a/tests/sources/rest_api/integration/test_response_actions.py +++ b/tests/sources/rest_api/integration/test_response_actions.py @@ -1,3 +1,4 @@ +import base64 import pytest from dlt.common import json from dlt.sources.helpers.requests import Response @@ -316,3 +317,43 @@ def add_field(response: Response, *args, **kwargs) -> Response: mock_response_hook_2.assert_called_once() assert all(record["custom_field"] == "foobar" for record in data) + + +def test_auth_overwrites_for_specific_endpoints(mock_api_server, mocker): + def custom_hook(response: Response, *args, **kwargs) -> Response: + assert ( + response.request.headers["Authorization"] + == f"Basic {base64.b64encode(b'U:P').decode('ascii')}" + ) + return response + + mock_response_hook = mocker.Mock(side_effect=custom_hook) + mock_source = rest_api_source( + { + "client": { + "base_url": "https://api.example.com", + "auth": { + "type": "bearer", + "token": "T", + }, + }, + "resources": [ + { + "name": "posts", + "endpoint": { + "auth": { + "type": "http_basic", + "username": "U", + "password": "P", + }, + "response_actions": [ + mock_response_hook, + ], + }, + }, + ], + } + ) + + list(mock_source.with_resources("posts").add_limit(1)) + mock_response_hook.assert_called_once() diff --git a/tests/sources/sql_database/test_schema_types.py b/tests/sources/sql_database/test_schema_types.py index 993a6f8955..d616b93a25 100644 --- a/tests/sources/sql_database/test_schema_types.py +++ b/tests/sources/sql_database/test_schema_types.py @@ -68,7 +68,7 @@ def test_get_table_references() -> None: refs = get_table_references(child) refs = sorted(refs, key=lambda x: x["referenced_table"]) assert refs[0]["referenced_table"] == "parent" - # Sqla aonstraints are not in fixed order + # Sqla constraints are not in fixed order assert set(refs[0]["columns"]) == {"parent_id", "parent_country"} assert set(refs[0]["referenced_columns"]) == {"id", "country"} # Ensure columns and referenced columns are the same order