diff --git a/.github/workflows/test_destination_clickhouse.yml b/.github/workflows/test_destination_clickhouse.yml new file mode 100644 index 0000000000..d834df6b28 --- /dev/null +++ b/.github/workflows/test_destination_clickhouse.yml @@ -0,0 +1,78 @@ + +name: test | clickhouse + +on: + pull_request: + branches: + - master + - devel + workflow_dispatch: + schedule: + - cron: '0 2 * * *' + +concurrency: + group: ${{ github.workflow }}-${{ github.event.pull_request.number || github.ref }} + cancel-in-progress: true + +env: + RUNTIME__SENTRY_DSN: https://6f6f7b6f8e0f458a89be4187603b55fe@o1061158.ingest.sentry.io/4504819859914752 + RUNTIME__LOG_LEVEL: ERROR + DLT_SECRETS_TOML: ${{ secrets.DLT_SECRETS_TOML }} + + ACTIVE_DESTINATIONS: "[\"clickhouse\"]" + ALL_FILESYSTEM_DRIVERS: "[\"memory\"]" + +jobs: + get_docs_changes: + name: docs changes + uses: ./.github/workflows/get_docs_changes.yml + if: ${{ !github.event.pull_request.head.repo.fork || contains(github.event.pull_request.labels.*.name, 'ci from fork')}} + + run_loader: + name: test | clickhouse tests + needs: get_docs_changes + if: needs.get_docs_changes.outputs.changes_outside_docs == 'true' + defaults: + run: + shell: bash + runs-on: "ubuntu-latest" + + steps: + + - name: Check out + uses: actions/checkout@master + + - name: Setup Python + uses: actions/setup-python@v4 + with: + python-version: "3.10.x" + + - name: Install Poetry + uses: snok/install-poetry@v1.3.2 + with: + virtualenvs-create: true + virtualenvs-in-project: true + installer-parallel: true + + - name: Load cached venv + id: cached-poetry-dependencies + uses: actions/cache@v3 + with: + path: .venv + key: venv-${{ runner.os }}-${{ steps.setup-python.outputs.python-version }}-${{ hashFiles('**/poetry.lock') }}-gcp + + - name: Install dependencies + run: poetry install --no-interaction -E clickhouse --with providers -E parquet --with sentry-sdk --with pipeline + + - name: create secrets.toml + run: pwd && echo "$DLT_SECRETS_TOML" > tests/.dlt/secrets.toml + + - run: | + poetry run pytest tests/load -m "essential" + name: Run essential tests Linux + if: ${{ ! (contains(github.event.pull_request.labels.*.name, 'ci full') || github.event_name == 'schedule')}} + + - run: | + poetry run pytest tests/load + name: Run all tests Linux + if: ${{ contains(github.event.pull_request.labels.*.name, 'ci full') || github.event_name == 'schedule'}} diff --git a/dlt/common/configuration/providers/airflow.py b/dlt/common/configuration/providers/airflow.py index 99edf258d2..edd02c3487 100644 --- a/dlt/common/configuration/providers/airflow.py +++ b/dlt/common/configuration/providers/airflow.py @@ -14,7 +14,6 @@ def name(self) -> str: def _look_vault(self, full_key: str, hint: type) -> str: """Get Airflow Variable with given `full_key`, return None if not found""" - from airflow.models import Variable with contextlib.redirect_stdout(io.StringIO()), contextlib.redirect_stderr(io.StringIO()): diff --git a/dlt/common/configuration/specs/base_configuration.py b/dlt/common/configuration/specs/base_configuration.py index 06fb97fcdd..1329feae6c 100644 --- a/dlt/common/configuration/specs/base_configuration.py +++ b/dlt/common/configuration/specs/base_configuration.py @@ -197,7 +197,7 @@ def default_factory(att_value=att_value): # type: ignore[no-untyped-def] synth_init = init and ((not base_params or base_params.init) and has_default_init) if synth_init != init and has_default_init: warnings.warn( - f"__init__ method will not be generated on {cls.__name__} because bas class didn't" + f"__init__ method will not be generated on {cls.__name__} because base class didn't" " synthesize __init__. Please correct `init` flag in confispec decorator. You are" " probably receiving incorrect __init__ signature for type checking" ) diff --git a/dlt/common/data_writers/escape.py b/dlt/common/data_writers/escape.py index e047778e3d..cdbb378faa 100644 --- a/dlt/common/data_writers/escape.py +++ b/dlt/common/data_writers/escape.py @@ -150,10 +150,47 @@ def escape_databricks_literal(v: Any) -> Any: return _escape_extended(json.dumps(v), prefix="'", escape_dict=DATABRICKS_ESCAPE_DICT) if isinstance(v, bytes): return f"X'{v.hex()}'" - if v is None: - return "NULL" + return "NULL" if v is None else str(v) - return str(v) + +# https://github.com/ClickHouse/ClickHouse/blob/master/docs/en/sql-reference/syntax.md#string +CLICKHOUSE_ESCAPE_DICT = { + "'": "''", + "\\": "\\\\", + "\n": "\\n", + "\t": "\\t", + "\b": "\\b", + "\f": "\\f", + "\r": "\\r", + "\0": "\\0", + "\a": "\\a", + "\v": "\\v", +} + +CLICKHOUSE_ESCAPE_RE = _make_sql_escape_re(CLICKHOUSE_ESCAPE_DICT) + + +def escape_clickhouse_literal(v: Any) -> Any: + if isinstance(v, str): + return _escape_extended( + v, prefix="'", escape_dict=CLICKHOUSE_ESCAPE_DICT, escape_re=CLICKHOUSE_ESCAPE_RE + ) + if isinstance(v, (datetime, date, time)): + return f"'{v.isoformat()}'" + if isinstance(v, (list, dict)): + return _escape_extended( + json.dumps(v), + prefix="'", + escape_dict=CLICKHOUSE_ESCAPE_DICT, + escape_re=CLICKHOUSE_ESCAPE_RE, + ) + if isinstance(v, bytes): + return f"'{v.hex()}'" + return "NULL" if v is None else str(v) + + +def escape_clickhouse_identifier(v: str) -> str: + return "`" + v.replace("`", "``").replace("\\", "\\\\") + "`" def format_datetime_literal(v: pendulum.DateTime, precision: int = 6, no_tz: bool = False) -> str: @@ -176,3 +213,11 @@ def format_bigquery_datetime_literal( """Returns BigQuery-adjusted datetime literal by prefixing required `TIMESTAMP` indicator.""" # https://cloud.google.com/bigquery/docs/reference/standard-sql/lexical#timestamp_literals return "TIMESTAMP " + format_datetime_literal(v, precision, no_tz) + + +def format_clickhouse_datetime_literal( + v: pendulum.DateTime, precision: int = 6, no_tz: bool = False +) -> str: + """Returns clickhouse compatibel function""" + datetime = format_datetime_literal(v, precision, True) + return f"toDateTime64({datetime}, {precision}, '{v.tzinfo}')" diff --git a/dlt/common/destination/capabilities.py b/dlt/common/destination/capabilities.py index 8432f8b544..e74f5a980d 100644 --- a/dlt/common/destination/capabilities.py +++ b/dlt/common/destination/capabilities.py @@ -53,6 +53,7 @@ class DestinationCapabilitiesContext(ContainerInjectableContext): insert_values_writer_type: str = "default" supports_multiple_statements: bool = True supports_clone_table: bool = False + """Destination supports CREATE TABLE ... CLONE ... statements""" max_table_nesting: Optional[int] = None # destination can overwrite max table nesting diff --git a/dlt/common/time.py b/dlt/common/time.py index b7be589b67..733b51b980 100644 --- a/dlt/common/time.py +++ b/dlt/common/time.py @@ -133,7 +133,6 @@ def ensure_pendulum_time(value: Union[str, datetime.time]) -> pendulum.Time: Returns: A pendulum.Time object """ - if isinstance(value, datetime.time): if isinstance(value, pendulum.Time): return value diff --git a/dlt/destinations/__init__.py b/dlt/destinations/__init__.py index 13b7f7ed99..302de24a6b 100644 --- a/dlt/destinations/__init__.py +++ b/dlt/destinations/__init__.py @@ -14,6 +14,7 @@ from dlt.destinations.impl.synapse.factory import synapse from dlt.destinations.impl.databricks.factory import databricks from dlt.destinations.impl.dremio.factory import dremio +from dlt.destinations.impl.clickhouse.factory import clickhouse __all__ = [ @@ -32,5 +33,6 @@ "synapse", "databricks", "dremio", + "clickhouse", "destination", ] diff --git a/dlt/destinations/adapters.py b/dlt/destinations/adapters.py index a143934116..554bd88924 100644 --- a/dlt/destinations/adapters.py +++ b/dlt/destinations/adapters.py @@ -4,5 +4,12 @@ from dlt.destinations.impl.qdrant import qdrant_adapter from dlt.destinations.impl.bigquery import bigquery_adapter from dlt.destinations.impl.synapse import synapse_adapter +from dlt.destinations.impl.clickhouse import clickhouse_adapter -__all__ = ["weaviate_adapter", "qdrant_adapter", "bigquery_adapter", "synapse_adapter"] +__all__ = [ + "weaviate_adapter", + "qdrant_adapter", + "bigquery_adapter", + "synapse_adapter", + "clickhouse_adapter", +] diff --git a/dlt/destinations/impl/clickhouse/__init__.py b/dlt/destinations/impl/clickhouse/__init__.py new file mode 100644 index 0000000000..bead136828 --- /dev/null +++ b/dlt/destinations/impl/clickhouse/__init__.py @@ -0,0 +1,53 @@ +import sys + +from dlt.common.pendulum import pendulum +from dlt.common.arithmetics import DEFAULT_NUMERIC_PRECISION, DEFAULT_NUMERIC_SCALE +from dlt.common.data_writers.escape import ( + escape_clickhouse_identifier, + escape_clickhouse_literal, + format_clickhouse_datetime_literal, +) +from dlt.common.destination import DestinationCapabilitiesContext +from dlt.destinations.impl.clickhouse.clickhouse_adapter import clickhouse_adapter + + +def capabilities() -> DestinationCapabilitiesContext: + caps = DestinationCapabilitiesContext() + caps.preferred_loader_file_format = "jsonl" + caps.supported_loader_file_formats = ["parquet", "jsonl"] + caps.preferred_staging_file_format = "jsonl" + caps.supported_staging_file_formats = ["parquet", "jsonl"] + + caps.format_datetime_literal = format_clickhouse_datetime_literal + caps.escape_identifier = escape_clickhouse_identifier + caps.escape_literal = escape_clickhouse_literal + + # https://stackoverflow.com/questions/68358686/what-is-the-maximum-length-of-a-column-in-clickhouse-can-it-be-modified + caps.max_identifier_length = 255 + caps.max_column_identifier_length = 255 + + # ClickHouse has no max `String` type length. + caps.max_text_data_type_length = sys.maxsize + + caps.schema_supports_numeric_precision = True + # Use 'Decimal128' with these defaults. + # https://clickhouse.com/docs/en/sql-reference/data-types/decimal + caps.decimal_precision = (DEFAULT_NUMERIC_PRECISION, DEFAULT_NUMERIC_SCALE) + # Use 'Decimal256' with these defaults. + caps.wei_precision = (76, 0) + caps.timestamp_precision = 6 + + # https://clickhouse.com/docs/en/operations/settings/settings#max_query_size + caps.is_max_query_length_in_bytes = True + caps.max_query_length = 262144 + + # ClickHouse has limited support for transactional semantics, especially for `ReplicatedMergeTree`, + # the default ClickHouse Cloud engine. It does, however, provide atomicity for individual DDL operations like `ALTER TABLE`. + # https://clickhouse-driver.readthedocs.io/en/latest/dbapi.html#clickhouse_driver.dbapi.connection.Connection.commit + # https://clickhouse.com/docs/en/guides/developer/transactional#transactions-commit-and-rollback + caps.supports_transactions = False + caps.supports_ddl_transactions = False + + caps.supports_truncate_command = True + + return caps diff --git a/dlt/destinations/impl/clickhouse/clickhouse.py b/dlt/destinations/impl/clickhouse/clickhouse.py new file mode 100644 index 0000000000..e2c1f827bc --- /dev/null +++ b/dlt/destinations/impl/clickhouse/clickhouse.py @@ -0,0 +1,408 @@ +import os +import re +from copy import deepcopy +from typing import ClassVar, Optional, Dict, List, Sequence, cast, Tuple +from urllib.parse import urlparse + +import clickhouse_connect +from clickhouse_connect.driver.tools import insert_file + +import dlt +from dlt import config +from dlt.common.configuration.specs import ( + CredentialsConfiguration, + AzureCredentialsWithoutDefaults, + GcpCredentials, + AwsCredentialsWithoutDefaults, +) +from dlt.destinations.exceptions import DestinationTransientException +from dlt.common.destination import DestinationCapabilitiesContext +from dlt.common.destination.reference import ( + SupportsStagingDestination, + TLoadJobState, + FollowupJob, + LoadJob, + NewLoadJob, +) +from dlt.common.schema import Schema, TColumnSchema +from dlt.common.schema.typing import ( + TTableFormat, + TTableSchema, + TColumnHint, + TColumnType, + TTableSchemaColumns, + TColumnSchemaBase, +) +from dlt.common.storages import FileStorage +from dlt.destinations.exceptions import LoadJobTerminalException +from dlt.destinations.impl.clickhouse import capabilities +from dlt.destinations.impl.clickhouse.clickhouse_adapter import ( + TTableEngineType, + TABLE_ENGINE_TYPE_HINT, +) +from dlt.destinations.impl.clickhouse.configuration import ( + ClickHouseClientConfiguration, +) +from dlt.destinations.impl.clickhouse.sql_client import ClickHouseSqlClient +from dlt.destinations.impl.clickhouse.utils import ( + convert_storage_to_http_scheme, + FILE_FORMAT_TO_TABLE_FUNCTION_MAPPING, + SUPPORTED_FILE_FORMATS, +) +from dlt.destinations.job_client_impl import ( + SqlJobClientBase, + SqlJobClientWithStaging, +) +from dlt.destinations.job_impl import NewReferenceJob, EmptyLoadJob +from dlt.destinations.sql_jobs import SqlMergeJob +from dlt.destinations.type_mapping import TypeMapper + + +HINT_TO_CLICKHOUSE_ATTR: Dict[TColumnHint, str] = { + "primary_key": "PRIMARY KEY", + "unique": "", # No unique constraints available in ClickHouse. + "foreign_key": "", # No foreign key constraints support in ClickHouse. +} + +TABLE_ENGINE_TYPE_TO_CLICKHOUSE_ATTR: Dict[TTableEngineType, str] = { + "merge_tree": "MergeTree", + "replicated_merge_tree": "ReplicatedMergeTree", +} + + +class ClickHouseTypeMapper(TypeMapper): + sct_to_unbound_dbt = { + "complex": "String", + "text": "String", + "double": "Float64", + "bool": "Boolean", + "date": "Date", + "timestamp": "DateTime64(6,'UTC')", + "time": "String", + "bigint": "Int64", + "binary": "String", + "wei": "Decimal", + } + + sct_to_dbt = { + "decimal": "Decimal(%i,%i)", + "wei": "Decimal(%i,%i)", + "timestamp": "DateTime64(%i,'UTC')", + } + + dbt_to_sct = { + "String": "text", + "Float64": "double", + "Bool": "bool", + "Date": "date", + "DateTime": "timestamp", + "DateTime64": "timestamp", + "Time": "timestamp", + "Int64": "bigint", + "Object('json')": "complex", + "Decimal": "decimal", + } + + def from_db_type( + self, db_type: str, precision: Optional[int] = None, scale: Optional[int] = None + ) -> TColumnType: + # Remove "Nullable" wrapper. + db_type = re.sub(r"^Nullable\((?P.+)\)$", r"\g", db_type) + + # Remove timezone details. + if db_type == "DateTime('UTC')": + db_type = "DateTime" + if datetime_match := re.match( + r"DateTime64(?:\((?P\d+)(?:,?\s*'(?PUTC)')?\))?", db_type + ): + if datetime_match["precision"]: + precision = int(datetime_match["precision"]) + else: + precision = None + db_type = "DateTime64" + + # Extract precision and scale, parameters and remove from string. + if decimal_match := re.match( + r"Decimal\((?P\d+)\s*(?:,\s*(?P\d+))?\)", db_type + ): + precision, scale = decimal_match.groups() # type: ignore[assignment] + precision = int(precision) + scale = int(scale) if scale else 0 + db_type = "Decimal" + + if db_type == "Decimal" and (precision, scale) == self.capabilities.wei_precision: + return dict(data_type="wei") + + return super().from_db_type(db_type, precision, scale) + + +class ClickHouseLoadJob(LoadJob, FollowupJob): + def __init__( + self, + file_path: str, + table_name: str, + client: ClickHouseSqlClient, + staging_credentials: Optional[CredentialsConfiguration] = None, + ) -> None: + file_name = FileStorage.get_file_name_from_file_path(file_path) + super().__init__(file_name) + + qualified_table_name = client.make_qualified_table_name(table_name) + bucket_path = None + + if NewReferenceJob.is_reference_job(file_path): + bucket_path = NewReferenceJob.resolve_reference(file_path) + file_name = FileStorage.get_file_name_from_file_path(bucket_path) + bucket_url = urlparse(bucket_path) + bucket_scheme = bucket_url.scheme + + ext = cast(SUPPORTED_FILE_FORMATS, os.path.splitext(file_name)[1][1:].lower()) + clickhouse_format: str = FILE_FORMAT_TO_TABLE_FUNCTION_MAPPING[ext] + + compression = "auto" + + # Don't use dbapi driver for local files. + if not bucket_path: + # Local filesystem. + if ext == "jsonl": + compression = "gz" if FileStorage.is_gzipped(file_path) else "none" + try: + with clickhouse_connect.create_client( + host=client.credentials.host, + port=client.credentials.http_port, + database=client.credentials.database, + user_name=client.credentials.username, + password=client.credentials.password, + secure=bool(client.credentials.secure), + ) as clickhouse_connect_client: + insert_file( + clickhouse_connect_client, + qualified_table_name, + file_path, + fmt=clickhouse_format, + settings={ + "allow_experimental_lightweight_delete": 1, + # "allow_experimental_object_type": 1, + "enable_http_compression": 1, + }, + compression=compression, + ) + except clickhouse_connect.driver.exceptions.Error as e: + raise LoadJobTerminalException( + file_path, + f"ClickHouse connection failed due to {e}.", + ) from e + return + + # Auto does not work for jsonl, get info from config for buckets + # NOTE: we should not really be accessing the config this way, but for + # now it is ok... + if ext == "jsonl": + compression = "none" if config.get("data_writer.disable_compression") else "gz" + + if bucket_scheme in ("s3", "gs", "gcs"): + # get auth and bucket url + bucket_http_url = convert_storage_to_http_scheme(bucket_url) + access_key_id: str = None + secret_access_key: str = None + if isinstance(staging_credentials, AwsCredentialsWithoutDefaults): + access_key_id = staging_credentials.aws_access_key_id + secret_access_key = staging_credentials.aws_secret_access_key + elif isinstance(staging_credentials, GcpCredentials): + access_key_id = client.credentials.gcp_access_key_id + secret_access_key = client.credentials.gcp_secret_access_key + if not access_key_id or not secret_access_key: + raise DestinationTransientException( + "You have tried loading from gcs with clickhouse. Please provide valid" + " 'gcp_access_key_id' and 'gcp_secret_access_key' to connect to gcs as" + " outlined in the dlthub docs." + ) + + auth = "NOSIGN" + if access_key_id and secret_access_key: + auth = f"'{access_key_id}','{secret_access_key}'" + + table_function = ( + f"s3('{bucket_http_url}',{auth},'{clickhouse_format}','auto','{compression}')" + ) + + elif bucket_scheme in ("az", "abfs"): + if not isinstance(staging_credentials, AzureCredentialsWithoutDefaults): + raise LoadJobTerminalException( + file_path, + "Unsigned Azure Blob Storage access from ClickHouse isn't supported as yet.", + ) + + # Authenticated access. + account_name = staging_credentials.azure_storage_account_name + storage_account_url = ( + f"https://{staging_credentials.azure_storage_account_name}.blob.core.windows.net" + ) + account_key = staging_credentials.azure_storage_account_key + + # build table func + table_function = f"azureBlobStorage('{storage_account_url}','{bucket_url.netloc}','{bucket_url.path}','{account_name}','{account_key}','{clickhouse_format}','{compression}')" + else: + raise LoadJobTerminalException( + file_path, + f"ClickHouse loader does not support '{bucket_scheme}' filesystem.", + ) + + statement = f"INSERT INTO {qualified_table_name} SELECT * FROM {table_function}" + with client.begin_transaction(): + client.execute_sql(statement) + + def state(self) -> TLoadJobState: + return "completed" + + def exception(self) -> str: + raise NotImplementedError() + + +class ClickHouseMergeJob(SqlMergeJob): + @classmethod + def _to_temp_table(cls, select_sql: str, temp_table_name: str) -> str: + return f"CREATE TABLE {temp_table_name} ENGINE = Memory AS {select_sql};" + + @classmethod + def gen_key_table_clauses( + cls, + root_table_name: str, + staging_root_table_name: str, + key_clauses: Sequence[str], + for_delete: bool, + ) -> List[str]: + join_conditions = " AND ".join([c.format(d="d", s="s") for c in key_clauses]) + return [ + f"FROM {root_table_name} AS d JOIN {staging_root_table_name} AS s ON {join_conditions}" + ] + + @classmethod + def gen_update_table_prefix(cls, table_name: str) -> str: + return f"ALTER TABLE {table_name} UPDATE" + + @classmethod + def requires_temp_table_for_delete(cls) -> bool: + return True + + +class ClickHouseClient(SqlJobClientWithStaging, SupportsStagingDestination): + capabilities: ClassVar[DestinationCapabilitiesContext] = capabilities() + + def __init__( + self, + schema: Schema, + config: ClickHouseClientConfiguration, + ) -> None: + self.sql_client: ClickHouseSqlClient = ClickHouseSqlClient( + config.normalize_dataset_name(schema), config.credentials + ) + super().__init__(schema, config, self.sql_client) + self.config: ClickHouseClientConfiguration = config + self.active_hints = deepcopy(HINT_TO_CLICKHOUSE_ATTR) + self.type_mapper = ClickHouseTypeMapper(self.capabilities) + + def _create_merge_followup_jobs(self, table_chain: Sequence[TTableSchema]) -> List[NewLoadJob]: + return [ClickHouseMergeJob.from_table_chain(table_chain, self.sql_client)] + + def _get_column_def_sql(self, c: TColumnSchema, table_format: TTableFormat = None) -> str: + # Build column definition. + # The primary key and sort order definition is defined outside column specification. + hints_str = " ".join( + self.active_hints.get(hint) + for hint in self.active_hints.keys() + if c.get(hint, False) is True + and hint not in ("primary_key", "sort") + and hint in self.active_hints + ) + + # Alter table statements only accept `Nullable` modifiers. + # JSON type isn't nullable in ClickHouse. + type_with_nullability_modifier = ( + f"Nullable({self.type_mapper.to_db_type(c)})" + if c.get("nullable", True) + else self.type_mapper.to_db_type(c) + ) + + return ( + f"{self.capabilities.escape_identifier(c['name'])} {type_with_nullability_modifier} {hints_str}" + .strip() + ) + + def start_file_load(self, table: TTableSchema, file_path: str, load_id: str) -> LoadJob: + return super().start_file_load(table, file_path, load_id) or ClickHouseLoadJob( + file_path, + table["name"], + self.sql_client, + staging_credentials=( + self.config.staging_config.credentials if self.config.staging_config else None + ), + ) + + def _get_table_update_sql( + self, table_name: str, new_columns: Sequence[TColumnSchema], generate_alter: bool + ) -> List[str]: + table: TTableSchema = self.prepare_load_table(table_name, self.in_staging_mode) + sql = SqlJobClientBase._get_table_update_sql(self, table_name, new_columns, generate_alter) + + if generate_alter: + return sql + + # Default to 'ReplicatedMergeTree' if user didn't explicitly set a table engine hint. + table_type = cast( + TTableEngineType, table.get(TABLE_ENGINE_TYPE_HINT, "replicated_merge_tree") + ) + sql[0] = f"{sql[0]}\nENGINE = {TABLE_ENGINE_TYPE_TO_CLICKHOUSE_ATTR.get(table_type)}" + + if primary_key_list := [ + self.capabilities.escape_identifier(c["name"]) + for c in new_columns + if c.get("primary_key") + ]: + sql[0] += "\nPRIMARY KEY (" + ", ".join(primary_key_list) + ")" + else: + sql[0] += "\nPRIMARY KEY tuple()" + + return sql + + def get_storage_table(self, table_name: str) -> Tuple[bool, TTableSchemaColumns]: + fields = self._get_storage_table_query_columns() + db_params = self.sql_client.make_qualified_table_name(table_name, escape=False).split( + ".", 3 + ) + query = f'SELECT {",".join(fields)} FROM INFORMATION_SCHEMA.COLUMNS WHERE ' + if len(db_params) == 3: + query += "table_catalog = %s AND " + query += "table_schema = %s AND table_name = %s ORDER BY ordinal_position;" + rows = self.sql_client.execute_sql(query, *db_params) + + # If no rows we assume that table does not exist. + schema_table: TTableSchemaColumns = {} + if len(rows) == 0: + return False, schema_table + for c in rows: + numeric_precision = ( + c[3] if self.capabilities.schema_supports_numeric_precision else None + ) + numeric_scale = c[4] if self.capabilities.schema_supports_numeric_precision else None + schema_c: TColumnSchemaBase = { + "name": c[0], + "nullable": bool(c[2]), + **self._from_db_type(c[1], numeric_precision, numeric_scale), + } + schema_table[c[0]] = schema_c # type: ignore + return True, schema_table + + @staticmethod + def _gen_not_null(v: bool) -> str: + # ClickHouse fields are not nullable by default. + # We use the `Nullable` modifier instead of NULL / NOT NULL modifiers to cater for ALTER statement. + pass + + def _from_db_type( + self, ch_t: str, precision: Optional[int], scale: Optional[int] + ) -> TColumnType: + return self.type_mapper.from_db_type(ch_t, precision, scale) + + def restore_file_load(self, file_path: str) -> LoadJob: + return EmptyLoadJob.from_file_path(file_path, "completed") diff --git a/dlt/destinations/impl/clickhouse/clickhouse_adapter.py b/dlt/destinations/impl/clickhouse/clickhouse_adapter.py new file mode 100644 index 0000000000..1bbde8e45d --- /dev/null +++ b/dlt/destinations/impl/clickhouse/clickhouse_adapter.py @@ -0,0 +1,60 @@ +from typing import Any, Literal, Set, get_args, Dict + +from dlt.destinations.utils import ensure_resource +from dlt.extract import DltResource +from dlt.extract.items import TTableHintTemplate + + +TTableEngineType = Literal["merge_tree", "replicated_merge_tree"] + +""" +The table engine (type of table) determines: + +- How and where data is stored, where to write it to, and where to read it from. +- Which queries are supported, and how. +- Concurrent data access. +- Use of indexes, if present. +- Whether multithread request execution is possible. +- Data replication parameters. + +See https://clickhouse.com/docs/en/engines/table-engines. +""" +TABLE_ENGINE_TYPES: Set[TTableEngineType] = set(get_args(TTableEngineType)) +TABLE_ENGINE_TYPE_HINT: Literal["x-table-engine-type"] = "x-table-engine-type" + + +def clickhouse_adapter(data: Any, table_engine_type: TTableEngineType = None) -> DltResource: + """Prepares data for the ClickHouse destination by specifying which table engine type + that should be used. + + 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. + table_engine_type (TTableEngineType, optional): The table index type used when creating + the Synapse table. + + Returns: + DltResource: A resource with applied Synapse-specific hints. + + Raises: + ValueError: If input for `table_engine_type` is invalid. + + Examples: + >>> data = [{"name": "Alice", "description": "Software Developer"}] + >>> clickhouse_adapter(data, table_engine_type="merge_tree") + [DltResource with hints applied] + """ + resource = ensure_resource(data) + + additional_table_hints: Dict[str, TTableHintTemplate[Any]] = {} + if table_engine_type is not None: + if table_engine_type not in TABLE_ENGINE_TYPES: + allowed_types = ", ".join(TABLE_ENGINE_TYPES) + raise ValueError( + f"Table engine type {table_engine_type} is invalid. Allowed table engine types are:" + f" {allowed_types}." + ) + additional_table_hints[TABLE_ENGINE_TYPE_HINT] = table_engine_type + resource.apply_hints(additional_table_hints=additional_table_hints) + return resource diff --git a/dlt/destinations/impl/clickhouse/configuration.py b/dlt/destinations/impl/clickhouse/configuration.py new file mode 100644 index 0000000000..bbff6e0a9c --- /dev/null +++ b/dlt/destinations/impl/clickhouse/configuration.py @@ -0,0 +1,95 @@ +import dataclasses +from typing import ClassVar, List, Any, Final, Literal, cast, Optional + +from dlt.common.configuration import configspec +from dlt.common.configuration.specs import ConnectionStringCredentials +from dlt.common.destination.reference import ( + DestinationClientDwhWithStagingConfiguration, +) +from dlt.common.libs.sql_alchemy import URL +from dlt.common.utils import digest128 + + +TSecureConnection = Literal[0, 1] + + +@configspec(init=False) +class ClickHouseCredentials(ConnectionStringCredentials): + drivername: str = "clickhouse" + host: str = None + """Host with running ClickHouse server.""" + port: int = 9440 + """Native port ClickHouse server is bound to. Defaults to 9440.""" + http_port: int = 8443 + """HTTP Port to connect to ClickHouse server's HTTP interface. + The HTTP port is needed for non-staging pipelines. + Defaults to 8123.""" + username: str = "default" + """Database user. Defaults to 'default'.""" + database: str = "default" + """database connect to. Defaults to 'default'.""" + secure: TSecureConnection = 1 + """Enables TLS encryption when connecting to ClickHouse Server. 0 means no encryption, 1 means encrypted.""" + connect_timeout: int = 15 + """Timeout for establishing connection. Defaults to 10 seconds.""" + send_receive_timeout: int = 300 + """Timeout for sending and receiving data. Defaults to 300 seconds.""" + dataset_table_separator: str = "___" + """Separator for dataset table names, defaults to '___', i.e. 'database.dataset___table'.""" + dataset_sentinel_table_name: str = "dlt_sentinel_table" + """Special table to mark dataset as existing""" + gcp_access_key_id: Optional[str] = None + """When loading from a gcp bucket, you need to provide gcp interoperable keys""" + gcp_secret_access_key: Optional[str] = None + """When loading from a gcp bucket, you need to provide gcp interoperable keys""" + + __config_gen_annotations__: ClassVar[List[str]] = [ + "host", + "port", + "http_port", + "database", + "username", + "password", + ] + + def parse_native_representation(self, native_value: Any) -> None: + super().parse_native_representation(native_value) + self.connect_timeout = int(self.query.get("connect_timeout", self.connect_timeout)) + self.send_receive_timeout = int( + self.query.get("send_receive_timeout", self.send_receive_timeout) + ) + self.secure = cast(TSecureConnection, int(self.query.get("secure", self.secure))) + if not self.is_partial(): + self.resolve() + + def to_url(self) -> URL: + url = super().to_url() + url = url.update_query_pairs( + [ + ("connect_timeout", str(self.connect_timeout)), + ("send_receive_timeout", str(self.send_receive_timeout)), + ("secure", str(1) if self.secure else str(0)), + # Toggle experimental settings. These are necessary for certain datatypes and not optional. + ("allow_experimental_lightweight_delete", "1"), + # ("allow_experimental_object_type", "1"), + ("enable_http_compression", "1"), + ] + ) + return url + + +@configspec +class ClickHouseClientConfiguration(DestinationClientDwhWithStagingConfiguration): + destination_type: Final[str] = dataclasses.field(default="clickhouse", init=False, repr=False, compare=False) # type: ignore[misc] + credentials: ClickHouseCredentials = None + + # Primary key columns are used to build a sparse primary index which allows for efficient data retrieval, + # but they do not enforce uniqueness constraints. It permits duplicate values even for the primary key + # columns within the same granule. + # See: https://clickhouse.com/docs/en/optimize/sparse-primary-indexes + + def fingerprint(self) -> str: + """Returns a fingerprint of host part of a connection string.""" + if self.credentials and self.credentials.host: + return digest128(self.credentials.host) + return "" diff --git a/dlt/destinations/impl/clickhouse/factory.py b/dlt/destinations/impl/clickhouse/factory.py new file mode 100644 index 0000000000..e5b8fc0e6a --- /dev/null +++ b/dlt/destinations/impl/clickhouse/factory.py @@ -0,0 +1,53 @@ +import typing as t + +from dlt.common.destination import Destination, DestinationCapabilitiesContext +from dlt.destinations.impl.clickhouse import capabilities +from dlt.destinations.impl.clickhouse.configuration import ( + ClickHouseClientConfiguration, + ClickHouseCredentials, +) + + +if t.TYPE_CHECKING: + from dlt.destinations.impl.clickhouse.clickhouse import ClickHouseClient + from clickhouse_driver.dbapi import Connection # type: ignore[import-untyped] + + +class clickhouse(Destination[ClickHouseClientConfiguration, "ClickHouseClient"]): + spec = ClickHouseClientConfiguration + + def capabilities(self) -> DestinationCapabilitiesContext: + return capabilities() + + @property + def client_class(self) -> t.Type["ClickHouseClient"]: + from dlt.destinations.impl.clickhouse.clickhouse import ClickHouseClient + + return ClickHouseClient + + def __init__( + self, + credentials: t.Union[ + ClickHouseCredentials, str, t.Dict[str, t.Any], t.Type["Connection"] + ] = None, + destination_name: str = None, + environment: str = None, + **kwargs: t.Any, + ) -> None: + """Configure the ClickHouse destination to use in a pipeline. + + All arguments provided here supersede other configuration sources such as environment + variables and dlt config files. + + Args: + credentials: Credentials to connect to the clickhouse database. + Can be an instance of `ClickHouseCredentials`, or a connection string + in the format `clickhouse://user:password@host:port/database`. + **kwargs: Additional arguments passed to the destination config. + """ + super().__init__( + credentials=credentials, + destination_name=destination_name, + environment=environment, + **kwargs, + ) diff --git a/dlt/destinations/impl/clickhouse/sql_client.py b/dlt/destinations/impl/clickhouse/sql_client.py new file mode 100644 index 0000000000..8fb89c90cd --- /dev/null +++ b/dlt/destinations/impl/clickhouse/sql_client.py @@ -0,0 +1,203 @@ +from contextlib import contextmanager +from typing import ( + Iterator, + AnyStr, + Any, + List, + Optional, + Sequence, + ClassVar, +) + +import clickhouse_driver # type: ignore[import-untyped] +import clickhouse_driver.errors # type: ignore[import-untyped] +from clickhouse_driver.dbapi import OperationalError # type: ignore[import-untyped] +from clickhouse_driver.dbapi.extras import DictCursor # type: ignore[import-untyped] + +from dlt.common.destination import DestinationCapabilitiesContext +from dlt.destinations.exceptions import ( + DatabaseUndefinedRelation, + DatabaseTransientException, + DatabaseTerminalException, +) +from dlt.destinations.impl.clickhouse import capabilities +from dlt.destinations.impl.clickhouse.configuration import ClickHouseCredentials +from dlt.destinations.sql_client import ( + DBApiCursorImpl, + SqlClientBase, + raise_database_error, + raise_open_connection_error, +) +from dlt.destinations.typing import DBTransaction, DBApi +from dlt.destinations.utils import _convert_to_old_pyformat + + +TRANSACTIONS_UNSUPPORTED_WARNING_MESSAGE = ( + "ClickHouse does not support transactions! Each statement is auto-committed separately." +) + + +class ClickHouseDBApiCursorImpl(DBApiCursorImpl): + native_cursor: DictCursor + + +class ClickHouseSqlClient( + SqlClientBase[clickhouse_driver.dbapi.connection.Connection], DBTransaction +): + dbapi: ClassVar[DBApi] = clickhouse_driver.dbapi + capabilities: ClassVar[DestinationCapabilitiesContext] = capabilities() + + def __init__(self, dataset_name: str, credentials: ClickHouseCredentials) -> None: + super().__init__(credentials.database, dataset_name) + self._conn: clickhouse_driver.dbapi.connection = None + self.credentials = credentials + self.database_name = credentials.database + + def has_dataset(self) -> bool: + sentinel_table = self.credentials.dataset_sentinel_table_name + return sentinel_table in [ + t.split(self.credentials.dataset_table_separator)[1] for t in self._list_tables() + ] + + def open_connection(self) -> clickhouse_driver.dbapi.connection.Connection: + self._conn = clickhouse_driver.connect(dsn=self.credentials.to_native_representation()) + return self._conn + + @raise_open_connection_error + def close_connection(self) -> None: + if self._conn: + self._conn.close() + self._conn = None + + @contextmanager + @raise_database_error + def begin_transaction(self) -> Iterator[DBTransaction]: + yield self + + @raise_database_error + def commit_transaction(self) -> None: + self._conn.commit() + + @raise_database_error + def rollback_transaction(self) -> None: + self._conn.rollback() + + @property + def native_connection(self) -> clickhouse_driver.dbapi.connection.Connection: + return self._conn + + def execute_sql( + self, sql: AnyStr, *args: Any, **kwargs: Any + ) -> Optional[Sequence[Sequence[Any]]]: + with self.execute_query(sql, *args, **kwargs) as curr: + return None if curr.description is None else curr.fetchall() + + def create_dataset(self) -> None: + # We create a sentinel table which defines wether we consider the dataset created + sentinel_table_name = self.make_qualified_table_name( + self.credentials.dataset_sentinel_table_name + ) + self.execute_sql( + f"""CREATE TABLE {sentinel_table_name} (_dlt_id String NOT NULL PRIMARY KEY) ENGINE=ReplicatedMergeTree COMMENT 'internal dlt sentinel table'""" + ) + + def drop_dataset(self) -> None: + # Since ClickHouse doesn't have schemas, we need to drop all tables in our virtual schema, + # or collection of tables, that has the `dataset_name` as a prefix. + to_drop_results = self._list_tables() + for table in to_drop_results: + # The "DROP TABLE" clause is discarded if we allow clickhouse_driver to handle parameter substitution. + # This is because the driver incorrectly substitutes the entire query string, causing the "DROP TABLE" keyword to be omitted. + # To resolve this, we are forced to provide the full query string here. + self.execute_sql( + f"""DROP TABLE {self.capabilities.escape_identifier(self.database_name)}.{self.capabilities.escape_identifier(table)} SYNC""" + ) + + def _list_tables(self) -> List[str]: + rows = self.execute_sql( + """ + SELECT name + FROM system.tables + WHERE database = %s + AND name LIKE %s + """, + ( + self.database_name, + f"{self.dataset_name}{self.credentials.dataset_table_separator}%", + ), + ) + return [row[0] for row in rows] + + @contextmanager + @raise_database_error + def execute_query( + self, query: AnyStr, *args: Any, **kwargs: Any + ) -> Iterator[ClickHouseDBApiCursorImpl]: + assert isinstance(query, str), "Query must be a string." + + db_args = kwargs.copy() + + if args: + query, db_args = _convert_to_old_pyformat(query, args, OperationalError) + db_args.update(kwargs) + + with self._conn.cursor() as cursor: + for query_line in query.split(";"): + if query_line := query_line.strip(): + try: + cursor.execute(query_line, db_args) + except KeyError as e: + raise DatabaseTransientException(OperationalError()) from e + + yield ClickHouseDBApiCursorImpl(cursor) # type: ignore[abstract] + + def fully_qualified_dataset_name(self, escape: bool = True) -> str: + database_name = self.database_name + dataset_name = self.dataset_name + if escape: + database_name = self.capabilities.escape_identifier(database_name) + dataset_name = self.capabilities.escape_identifier(dataset_name) + return f"{database_name}.{dataset_name}" + + def make_qualified_table_name(self, table_name: str, escape: bool = True) -> str: + database_name = self.database_name + table_name = f"{self.dataset_name}{self.credentials.dataset_table_separator}{table_name}" + if escape: + database_name = self.capabilities.escape_identifier(database_name) + table_name = self.capabilities.escape_identifier(table_name) + return f"{database_name}.{table_name}" + + @classmethod + def _make_database_exception(cls, ex: Exception) -> Exception: + if isinstance(ex, clickhouse_driver.dbapi.errors.OperationalError): + if "Code: 57." in str(ex) or "Code: 82." in str(ex) or "Code: 47." in str(ex): + return DatabaseTerminalException(ex) + elif "Code: 60." in str(ex) or "Code: 81." in str(ex): + return DatabaseUndefinedRelation(ex) + else: + return DatabaseTransientException(ex) + elif isinstance( + ex, + ( + clickhouse_driver.dbapi.errors.OperationalError, + clickhouse_driver.dbapi.errors.InternalError, + ), + ): + return DatabaseTransientException(ex) + elif isinstance( + ex, + ( + clickhouse_driver.dbapi.errors.DataError, + clickhouse_driver.dbapi.errors.ProgrammingError, + clickhouse_driver.dbapi.errors.IntegrityError, + ), + ): + return DatabaseTerminalException(ex) + elif cls.is_dbapi_exception(ex): + return DatabaseTransientException(ex) + else: + return ex + + @staticmethod + def is_dbapi_exception(ex: Exception) -> bool: + return isinstance(ex, clickhouse_driver.dbapi.Error) diff --git a/dlt/destinations/impl/clickhouse/utils.py b/dlt/destinations/impl/clickhouse/utils.py new file mode 100644 index 0000000000..b0b06909f9 --- /dev/null +++ b/dlt/destinations/impl/clickhouse/utils.py @@ -0,0 +1,42 @@ +from typing import Union, Literal, Dict +from urllib.parse import urlparse, ParseResult + + +SUPPORTED_FILE_FORMATS = Literal["jsonl", "parquet"] +FILE_FORMAT_TO_TABLE_FUNCTION_MAPPING: Dict[SUPPORTED_FILE_FORMATS, str] = { + "jsonl": "JSONEachRow", + "parquet": "Parquet", +} + + +def convert_storage_to_http_scheme( + url: Union[str, ParseResult], use_https: bool = False, endpoint: str = None, region: str = None +) -> str: + try: + if isinstance(url, str): + parsed_url = urlparse(url) + elif isinstance(url, ParseResult): + parsed_url = url + else: + raise TypeError("Invalid URL type. Expected str or ParseResult.") + + bucket_name = parsed_url.netloc + object_key = parsed_url.path.lstrip("/") + protocol = "https" if use_https else "http" + + if endpoint: + domain = endpoint + elif region and parsed_url.scheme == "s3": + domain = f"s3-{region}.amazonaws.com" + else: + # TODO: Incorporate dlt.config endpoint. + storage_domains = { + "s3": "s3.amazonaws.com", + "gs": "storage.googleapis.com", + "gcs": "storage.googleapis.com", + } + domain = storage_domains[parsed_url.scheme] + + return f"{protocol}://{bucket_name}.{domain}/{object_key}" + except Exception as e: + raise Exception(f"Error converting storage URL to HTTP protocol: '{url}'") from e diff --git a/dlt/destinations/impl/databricks/sql_client.py b/dlt/destinations/impl/databricks/sql_client.py index 7e2487593d..7f0ee2b5e6 100644 --- a/dlt/destinations/impl/databricks/sql_client.py +++ b/dlt/destinations/impl/databricks/sql_client.py @@ -142,12 +142,6 @@ def _make_database_exception(ex: Exception) -> Exception: else: return DatabaseTransientException(ex) - @staticmethod - def _maybe_make_terminal_exception_from_data_error( - databricks_ex: databricks_lib.DatabaseError, - ) -> Optional[Exception]: - return None - @staticmethod def is_dbapi_exception(ex: Exception) -> bool: return isinstance(ex, databricks_lib.DatabaseError) diff --git a/dlt/destinations/impl/duckdb/sql_client.py b/dlt/destinations/impl/duckdb/sql_client.py index 2863d4943e..bb85b5825b 100644 --- a/dlt/destinations/impl/duckdb/sql_client.py +++ b/dlt/destinations/impl/duckdb/sql_client.py @@ -168,11 +168,7 @@ def _make_database_exception(cls, ex: Exception) -> Exception: duckdb.ParserException, ), ): - term = cls._maybe_make_terminal_exception_from_data_error(ex) - if term: - return term - else: - return DatabaseTransientException(ex) + return DatabaseTransientException(ex) elif isinstance(ex, (duckdb.DataError, duckdb.ProgrammingError, duckdb.IntegrityError)): return DatabaseTerminalException(ex) elif cls.is_dbapi_exception(ex): @@ -180,10 +176,6 @@ def _make_database_exception(cls, ex: Exception) -> Exception: else: return ex - @staticmethod - def _maybe_make_terminal_exception_from_data_error(pg_ex: duckdb.Error) -> Optional[Exception]: - return None - @staticmethod def is_dbapi_exception(ex: Exception) -> bool: return isinstance(ex, duckdb.Error) diff --git a/dlt/destinations/impl/snowflake/sql_client.py b/dlt/destinations/impl/snowflake/sql_client.py index ba932277df..4a602ce0e8 100644 --- a/dlt/destinations/impl/snowflake/sql_client.py +++ b/dlt/destinations/impl/snowflake/sql_client.py @@ -148,11 +148,7 @@ def _make_database_exception(cls, ex: Exception) -> Exception: elif isinstance(ex, snowflake_lib.errors.IntegrityError): raise DatabaseTerminalException(ex) elif isinstance(ex, snowflake_lib.errors.DatabaseError): - term = cls._maybe_make_terminal_exception_from_data_error(ex) - if term: - return term - else: - return DatabaseTransientException(ex) + return DatabaseTransientException(ex) elif isinstance(ex, TypeError): # snowflake raises TypeError on malformed query parameters return DatabaseTransientException(snowflake_lib.errors.ProgrammingError(str(ex))) @@ -161,12 +157,6 @@ def _make_database_exception(cls, ex: Exception) -> Exception: else: return ex - @staticmethod - def _maybe_make_terminal_exception_from_data_error( - snowflake_ex: snowflake_lib.DatabaseError, - ) -> Optional[Exception]: - return None - @staticmethod def is_dbapi_exception(ex: Exception) -> bool: return isinstance(ex, snowflake_lib.DatabaseError) diff --git a/dlt/destinations/sql_jobs.py b/dlt/destinations/sql_jobs.py index ad91a76890..ff458870a1 100644 --- a/dlt/destinations/sql_jobs.py +++ b/dlt/destinations/sql_jobs.py @@ -3,7 +3,6 @@ import yaml from dlt.common.logger import pretty_format_exception -from dlt.common.pendulum import pendulum from dlt.common.schema.typing import ( TTableSchema, TSortOrder, @@ -393,7 +392,7 @@ def gen_merge_sql( unique_column: str = None root_key_column: str = None - if len(table_chain) == 1: + if len(table_chain) == 1 and not cls.requires_temp_table_for_delete(): key_table_clauses = cls.gen_key_table_clauses( root_table_name, staging_root_table_name, key_clauses, for_delete=True ) @@ -555,7 +554,7 @@ def gen_scd2_sql( # retire updated and deleted records sql.append(f""" - UPDATE {root_table_name} SET {to} = {boundary_ts} + {cls.gen_update_table_prefix(root_table_name)} {to} = {boundary_ts} WHERE {is_active_clause} AND {hash_} NOT IN (SELECT {hash_} FROM {staging_root_table_name}); """) @@ -597,7 +596,20 @@ def gen_scd2_sql( sql.append(f""" INSERT INTO {table_name} SELECT * - FROM {staging_table_name} AS s - WHERE NOT EXISTS (SELECT 1 FROM {table_name} AS f WHERE f.{unique_column} = s.{unique_column}); + FROM {staging_table_name} + WHERE {unique_column} NOT IN (SELECT {unique_column} FROM {table_name}); """) + return sql + + @classmethod + def gen_update_table_prefix(cls, table_name: str) -> str: + return f"UPDATE {table_name} SET" + + @classmethod + def requires_temp_table_for_delete(cls) -> bool: + """this could also be a capabitiy, but probably it is better stored here + this identifies destinations that can have a simplified method for merging single + table table chains + """ + return False diff --git a/dlt/destinations/utils.py b/dlt/destinations/utils.py index d4b945a840..c02460fe58 100644 --- a/dlt/destinations/utils.py +++ b/dlt/destinations/utils.py @@ -1,16 +1,48 @@ -from typing import Any +import re +from typing import Any, cast, Tuple, Dict, Type +from dlt.destinations.exceptions import DatabaseTransientException from dlt.extract import DltResource, resource as make_resource def ensure_resource(data: Any) -> DltResource: """Wraps `data` in a DltResource if it's not a DltResource already.""" - resource: DltResource - if not isinstance(data, DltResource): - resource_name: str = None - if not hasattr(data, "__name__"): - resource_name = "content" - resource = make_resource(data, name=resource_name) - else: - resource = data - return resource + if isinstance(data, DltResource): + return data + resource_name = None if hasattr(data, "__name__") else "content" + return cast(DltResource, make_resource(data, name=resource_name)) + + +def _convert_to_old_pyformat( + new_style_string: str, args: Tuple[Any, ...], operational_error_cls: Type[Exception] +) -> Tuple[str, Dict[str, Any]]: + """Converts a query string from the new pyformat style to the old pyformat style. + + The new pyformat style uses placeholders like %s, while the old pyformat style + uses placeholders like %(arg0)s, where the number corresponds to the index of + the argument in the args tuple. + + Args: + new_style_string (str): The query string in the new pyformat style. + args (Tuple[Any, ...]): The arguments to be inserted into the query string. + operational_error_cls (Type[Exception]): The specific OperationalError class to be raised + in case of a mismatch between placeholders and arguments. This should be the + OperationalError class provided by the DBAPI2-compliant driver being used. + + Returns: + Tuple[str, Dict[str, Any]]: A tuple containing the converted query string + in the old pyformat style, and a dictionary mapping argument keys to values. + + Raises: + DatabaseTransientException: If there is a mismatch between the number of + placeholders in the query string, and the number of arguments provided. + """ + if len(args) == 1 and isinstance(args[0], tuple): + args = args[0] + + keys = [f"arg{str(i)}" for i, _ in enumerate(args)] + old_style_string, count = re.subn(r"%s", lambda _: f"%({keys.pop(0)})s", new_style_string) + mapping = dict(zip([f"arg{str(i)}" for i, _ in enumerate(args)], args)) + if count != len(args): + raise DatabaseTransientException(operational_error_cls()) + return old_style_string, mapping diff --git a/dlt/helpers/dbt/profiles.yml b/dlt/helpers/dbt/profiles.yml index 8f2ad22585..d82eb0f2fa 100644 --- a/dlt/helpers/dbt/profiles.yml +++ b/dlt/helpers/dbt/profiles.yml @@ -171,3 +171,16 @@ databricks: http_path: "{{ env_var('DLT__CREDENTIALS__HTTP_PATH') }}" token: "{{ env_var('DLT__CREDENTIALS__ACCESS_TOKEN') }}" threads: 4 + + +clickhouse: + target: analytics + outputs: + analytics: + type: clickhouse + schema: "{{ env_var('DLT__CREDENTIALS__DATABASE') }}" + host: "{{ env_var('DLT__CREDENTIALS__HOST') }}" + user: "{{ env_var('DLT__CREDENTIALS__USERNAME') }}" + password: "{{ env_var('DLT__CREDENTIALS__PASSWORD') }}" + port: "{{ env_var('DLT__CREDENTIALS__PORT', 8443) | as_number }}" + secure: True \ No newline at end of file diff --git a/docs/website/docs/dlt-ecosystem/destinations/clickhouse.md b/docs/website/docs/dlt-ecosystem/destinations/clickhouse.md new file mode 100644 index 0000000000..ea187e54eb --- /dev/null +++ b/docs/website/docs/dlt-ecosystem/destinations/clickhouse.md @@ -0,0 +1,230 @@ +--- +title: ClickHouse +description: ClickHouse `dlt` destination +keywords: [ clickhouse, destination, data warehouse ] +--- + +# ClickHouse + +## Install dlt with ClickHouse + +**To install the DLT library with ClickHouse dependencies:** + +```sh +pip install dlt[clickhouse] +``` + +## Setup Guide + +### 1. Initialize the dlt project + +Let's start by initializing a new `dlt` project as follows: + +```sh +dlt init chess clickhouse +``` + +> 💡 This command will initialize your pipeline with chess as the source and ClickHouse as the destination. + +The above command generates several files and directories, including `.dlt/secrets.toml` and a requirements file for ClickHouse. You can install the necessary dependencies specified in the +requirements file by executing it as follows: + +```sh +pip install -r requirements.txt +``` + +or with `pip install dlt[clickhouse]`, which installs the `dlt` library and the necessary dependencies for working with ClickHouse as a destination. + +### 2. Setup ClickHouse database + +To load data into ClickHouse, you need to create a ClickHouse database. While we recommend asking our GPT-4 assistant for details, we have provided a general outline of the process below: + +1. You can use an existing ClickHouse database or create a new one. + +2. To create a new database, connect to your ClickHouse server using the `clickhouse-client` command line tool or a SQL client of your choice. + +3. Run the following SQL commands to create a new database, user and grant the necessary permissions: + + ```sql + CREATE DATABASE IF NOT EXISTS dlt; + CREATE USER dlt IDENTIFIED WITH sha256_password BY 'Dlt*12345789234567'; + GRANT CREATE, ALTER, SELECT, DELETE, DROP, TRUNCATE, OPTIMIZE, SHOW, INSERT, dictGet ON dlt.* TO dlt; + GRANT SELECT ON INFORMATION_SCHEMA.COLUMNS TO dlt; + GRANT CREATE TEMPORARY TABLE, S3 ON *.* TO dlt; + ``` + +### 3. Add credentials + +1. Next, set up the ClickHouse credentials in the `.dlt/secrets.toml` file as shown below: + + ```toml + [destination.clickhouse.credentials] + database = "dlt" # The database name you created + username = "dlt" # ClickHouse username, default is usually "default" + password = "Dlt*12345789234567" # ClickHouse password if any + host = "localhost" # ClickHouse server host + port = 9000 # ClickHouse HTTP port, default is 9000 + http_port = 8443 # HTTP Port to connect to ClickHouse server's HTTP interface. Defaults to 8443. + secure = 1 # Set to 1 if using HTTPS, else 0. + dataset_table_separator = "___" # Separator for dataset table names from dataset. + ``` + + :::info http_port + The `http_port` parameter specifies the port number to use when connecting to the ClickHouse server's HTTP interface. This is different from default port 9000, which is used for the native TCP + protocol. + + You must set `http_port` if you are not using external staging (i.e. you don't set the staging parameter in your pipeline). This is because dlt's built-in ClickHouse local storage staging uses the + [clickhouse-connect](https://github.com/ClickHouse/clickhouse-connect) library, which communicates with ClickHouse over HTTP. + + Make sure your ClickHouse server is configured to accept HTTP connections on the port specified by `http_port`. For example, if you set `http_port = 8443`, then ClickHouse should be listening for + HTTP + requests on port 8443. If you are using external staging, you can omit the `http_port` parameter, since clickhouse-connect will not be used in this case. + ::: + +2. You can pass a database connection string similar to the one used by the `clickhouse-driver` library. The credentials above will look like this: + + ```toml + # keep it at the top of your toml file, before any section starts. + destination.clickhouse.credentials="clickhouse://dlt:Dlt*12345789234567@localhost:9000/dlt?secure=1" + ``` + +## Write disposition + +All [write dispositions](../../general-usage/incremental-loading#choosing-a-write-disposition) are supported. + +## Data loading + +Data is loaded into ClickHouse using the most efficient method depending on the data source: + +- For local files, the `clickhouse-connect` library is used to directly load files into ClickHouse tables using the `INSERT` command. +- For files in remote storage like S3, Google Cloud Storage, or Azure Blob Storage, ClickHouse table functions like `s3`, `gcs` and `azureBlobStorage` are used to read the files and insert the data + into tables. + +## Datasets + +`Clickhouse` does not support multiple datasets in one database, dlt relies on datasets to exist for multiple reasons. +To make `clickhouse` work with `dlt`, tables generated by `dlt` in your `clickhouse` database will have their name prefixed with the dataset name separated by +the configurable `dataset_table_separator`. Additionally, a special sentinel table that does not contain any data will be created, so dlt knows which virtual datasets already exist in a +clickhouse +destination. + +## Supported file formats + +- [jsonl](../file-formats/jsonl.md) is the preferred format for both direct loading and staging. +- [parquet](../file-formats/parquet.md) is supported for both direct loading and staging. + +The `clickhouse` destination has a few specific deviations from the default sql destinations: + +1. `Clickhouse` has an experimental `object` datatype, but we have found it to be a bit unpredictable, so the dlt clickhouse destination will load the complex datatype to a `text` column. If you need this feature, get in touch with our Slack community, and we will consider adding it. +2. `Clickhouse` does not support the `time` datatype. Time will be loaded to a `text` column. +3. `Clickhouse` does not support the `binary` datatype. Binary will be loaded to a `text` column. When loading from `jsonl`, this will be a base64 string, when loading from parquet this will be + the `binary` object converted to `text`. +4. `Clickhouse` accepts adding columns to a populated table that are not null. +5. `Clickhouse` can produce rounding errors under certain conditions when using the float / double datatype. Make sure to use decimal if you cannot afford to have rounding errors. Loading the value 12.7001 to a double column with the loader file format jsonl set will predictbly produce a rounding error for example. + +## Supported column hints + +ClickHouse supports the following [column hints](../../general-usage/schema#tables-and-columns): + +- `primary_key` - marks the column as part of the primary key. Multiple columns can have this hint to create a composite primary key. + +## Table Engine + +By default, tables are created using the `ReplicatedMergeTree` table engine in ClickHouse. You can specify an alternate table engine using the `table_engine_type` with the clickhouse adapter: + +```py +from dlt.destinations.adapters import clickhouse_adapter + + +@dlt.resource() +def my_resource(): + ... + + +clickhouse_adapter(my_resource, table_engine_type="merge_tree") + +``` + +Supported values are: + +- `merge_tree` - creates tables using the `MergeTree` engine +- `replicated_merge_tree` (default) - creates tables using the `ReplicatedMergeTree` engine + +## Staging support + +ClickHouse supports Amazon S3, Google Cloud Storage and Azure Blob Storage as file staging destinations. + +`dlt` will upload Parquet or JSONL files to the staging location and use ClickHouse table functions to load the data directly from the staged files. + +Please refer to the filesystem documentation to learn how to configure credentials for the staging destinations: + +- [Amazon S3](./filesystem.md#aws-s3) +- [Google Cloud Storage](./filesystem.md#google-storage) +- [Azure Blob Storage](./filesystem.md#azure-blob-storage) + +To run a pipeline with staging enabled: + +```py +pipeline = dlt.pipeline( + pipeline_name='chess_pipeline', + destination='clickhouse', + staging='filesystem', # add this to activate staging + dataset_name='chess_data' +) +``` + +### Using Google Cloud Storage as a Staging Area + +dlt supports using Google Cloud Storage (GCS) as a staging area when loading data into ClickHouse. This is handled automatically by +ClickHouse's [GCS table function](https://clickhouse.com/docs/en/sql-reference/table-functions/gcs) which dlt uses under the hood. + +The clickhouse GCS table function only supports authentication using Hash-based Message Authentication Code (HMAC) keys. To enable this, GCS provides an S3 compatibility mode that emulates +the Amazon S3 +API. ClickHouse takes advantage of this to allow accessing GCS buckets via its S3 integration. + +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 as well as the `client_email`, `project_id` and `private_key` for your service account in your dlt project's ClickHouse destination settings in `config.toml`: + +```toml +[destination.filesystem] +bucket_url = "gs://dlt-ci" + +[destination.filesystem.credentials] +project_id = "a-cool-project" +client_email = "my-service-account@a-cool-project.iam.gserviceaccount.com" +private_key = "-----BEGIN PRIVATE KEY-----\nMIIEvQIBADANBgkaslkdjflasjnkdcopauihj...wEiEx7y+mx\nNffxQBqVVej2n/D93xY99pM=\n-----END PRIVATE KEY-----\n" + +[destination.clickhouse.credentials] +database = "dlt" +username = "dlt" +password = "Dlt*12345789234567" +host = "localhost" +port = 9440 +secure = 1 +gcp_access_key_id = "JFJ$$*f2058024835jFffsadf" +gcp_secret_access_key = "DFJdwslf2hf57)%$02jaflsedjfasoi" +``` + +Note: In addition to the HMAC keys (`gcp_access_key_id` and `gcp_secret_access_key`), you now need to provide the `client_email`, `project_id` and `private_key` for your service account +under `[destination.filesystem.credentials]`. +This is because the GCS staging support is now implemented as a temporary workaround and is still unoptimized. + +dlt will pass these credentials to ClickHouse which will handle the authentication and GCS access. + +There is active work in progress to simplify and improve the GCS staging setup for the ClickHouse dlt destination in the future. Proper GCS staging support is being tracked in these GitHub issues: + +- [Make filesystem destination work with gcs in s3 compatibility mode](https://github.com/dlt-hub/dlt/issues/1272) +- [GCS staging area support](https://github.com/dlt-hub/dlt/issues/1181) + +### dbt support + +Integration with [dbt](../transformations/dbt/dbt.md) is generally supported via dbt-clickhouse, but not tested by us. + +### Syncing of `dlt` state + +This destination fully supports [dlt state sync](../../general-usage/state#syncing-state-with-destination). + + \ No newline at end of file diff --git a/docs/website/sidebars.js b/docs/website/sidebars.js index a4666ab787..05dbce8943 100644 --- a/docs/website/sidebars.js +++ b/docs/website/sidebars.js @@ -105,6 +105,7 @@ const sidebars = { 'dlt-ecosystem/destinations/duckdb', 'dlt-ecosystem/destinations/mssql', 'dlt-ecosystem/destinations/synapse', + 'dlt-ecosystem/destinations/clickhouse', 'dlt-ecosystem/destinations/filesystem', 'dlt-ecosystem/destinations/postgres', 'dlt-ecosystem/destinations/redshift', diff --git a/poetry.lock b/poetry.lock index bb3001d25f..d6e78f4144 100644 --- a/poetry.lock +++ b/poetry.lock @@ -1,4 +1,4 @@ -# This file is automatically @generated by Poetry 1.7.1 and should not be changed by hand. +# This file is automatically @generated by Poetry 1.8.2 and should not be changed by hand. [[package]] name = "about-time" @@ -1670,6 +1670,218 @@ files = [ click = ">=4.0" PyYAML = ">=3.11" +[[package]] +name = "clickhouse-connect" +version = "0.7.8" +description = "ClickHouse Database Core Driver for Python, Pandas, and Superset" +optional = true +python-versions = "~=3.8" +files = [ + {file = "clickhouse-connect-0.7.8.tar.gz", hash = "sha256:dad10ba90eabfe215dfb1fef59f2821a95c752988e66f1093ca8590a51539b8f"}, + {file = "clickhouse_connect-0.7.8-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:5319d4a6f5a3484d20ba28954aaf417a3c37aa874877ed64fb635632025b4987"}, + {file = "clickhouse_connect-0.7.8-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:7e326370b0930dffb77d302f9ffbf418779f91c76b8386906e945b7986b625cf"}, + {file = "clickhouse_connect-0.7.8-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:cbb77a32f7cb5178f625cecef3c7924a8896e1273bb3c1c0819c64b60ab96347"}, + {file = "clickhouse_connect-0.7.8-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fb302c45d79411ae862bf894a3bbc8fd2808c016bfe491db2c957e2803f71d62"}, + {file = "clickhouse_connect-0.7.8-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1d5fd5f860ab28aa7096f79a866082251627218e5d994800dda6b4b860a5efc1"}, + {file = "clickhouse_connect-0.7.8-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:1b3df18dee46483cc982043f0ddccd301cd7eb1fc01a7f292ba62f41640f8d4f"}, + {file = "clickhouse_connect-0.7.8-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:b09dfcf48926b834090408b94d8be361c42b54fa892c29174b5883b34859577e"}, + {file = "clickhouse_connect-0.7.8-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:b9629122594f628aaa1188c6924298d3d7c36534de4241a58a68716da433ef07"}, + {file = "clickhouse_connect-0.7.8-cp310-cp310-win32.whl", hash = "sha256:7982e8c35a80c49e13c33889e7944ba265baf0e95e3b66db3d01efe5e1de4408"}, + {file = "clickhouse_connect-0.7.8-cp310-cp310-win_amd64.whl", hash = "sha256:38024602cfe9c8889e14d75f99f51e70352ddefbe3db5bee15300e6888a7397f"}, + {file = "clickhouse_connect-0.7.8-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:073c550b8433e31955a3468013147f339ad03650929467b73b120c8a788aa64a"}, + {file = "clickhouse_connect-0.7.8-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:f34829a5539b4643247e4c870ee04f9f03423ff35fecb22e0e5723d724f50986"}, + {file = "clickhouse_connect-0.7.8-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:959c3f324666ab97ccc324a0cf8b1c9eb2ecfcae32d05b3afc43f85b22aa486f"}, + {file = "clickhouse_connect-0.7.8-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:20905d467c61b334fe20af9432f06ff8ddd706eb9fba4596a7f629ae58897a36"}, + {file = "clickhouse_connect-0.7.8-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0f4193160c1a3bdd9569d407ec00d31c2c2c7f3748c97721005da496f609579b"}, + {file = "clickhouse_connect-0.7.8-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:f77052d16947eb30b694b3627d8b3fd313976c26692a80ab45e2fcf9b7dfe156"}, + {file = "clickhouse_connect-0.7.8-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:6e18df3466158bfaba9d5f52297701b31fa3d589a039392482a9a6be5341da17"}, + {file = "clickhouse_connect-0.7.8-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:6a2cfe42fca283c5f2e2d20cf79808c097c576d5a1f37e4b2870dcb39cea16c2"}, + {file = "clickhouse_connect-0.7.8-cp311-cp311-win32.whl", hash = "sha256:d964dc6e270f1f0d38cf2a780608329b5140877fb556f4c45bdf349900f5ebc8"}, + {file = "clickhouse_connect-0.7.8-cp311-cp311-win_amd64.whl", hash = "sha256:344500947233ddc429fb4653407b063d3d4efae52a4d508ff38118ec7c8b9afb"}, + {file = "clickhouse_connect-0.7.8-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:36ee57a5ab2d755f1ce0b05c1e1170c0cd23a22222b7af3394d1069552d33b01"}, + {file = "clickhouse_connect-0.7.8-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:02792590218e2b106a0dc87b0d21ab895c1643e9a62993b0b9922ac6a97ed0f5"}, + {file = "clickhouse_connect-0.7.8-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5983630c818ad87fba29cfd800880673263c1f5cfa8e199f6617d1ad5d041169"}, + {file = "clickhouse_connect-0.7.8-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a48eecc60fd522a76d03737063b14affb8b5e77a8446222da838857a91fce467"}, + {file = "clickhouse_connect-0.7.8-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5e65271cc53099ca10cb0fa0ba24c6e35b82865ae58915e3d503584c13620ed0"}, + {file = "clickhouse_connect-0.7.8-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:29c6a031beeba8b5bdaaec06abf061a5a2ca60652f2c7d7f3dad99b4cfd9ccde"}, + {file = "clickhouse_connect-0.7.8-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:10c7780e8bae624f6fc5aa25cccf44c0d56a91d5c02511fedfc02befa3a4cba1"}, + {file = "clickhouse_connect-0.7.8-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:04cc78a342511798cd24a8e9304930a19057d775c6b91cd83972b3cab66d28a2"}, + {file = "clickhouse_connect-0.7.8-cp312-cp312-win32.whl", hash = "sha256:69ff583442b832a54919e5d475b3392f580e949b461e0b60eb993609075c1a90"}, + {file = "clickhouse_connect-0.7.8-cp312-cp312-win_amd64.whl", hash = "sha256:31662144b8c75db19789825effe18b1e380fa5c479846bc3b81690ecf28fd35b"}, + {file = "clickhouse_connect-0.7.8-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:fc30ac90e5c660688f84da35434a2007fd7d846600e3c4039e4c38616c05575a"}, + {file = "clickhouse_connect-0.7.8-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:a993bb9b0be4ba2e847bdf4ea1851fe9e5d0a2124a3bfd0bd97d1d376154d081"}, + {file = "clickhouse_connect-0.7.8-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0102bf3c5d84ef5cda05cc5c4de05992d0e5e46a992188199a08bdc36fbe94a3"}, + {file = "clickhouse_connect-0.7.8-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c08c09842873851b9b02df27f056da76f49aa777b1bd76108e386cd2c75fc0fa"}, + {file = "clickhouse_connect-0.7.8-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b5a867e0a6556a7e6e7ef63c9c31561a47f5e2067758a89f583fbe03d093bd11"}, + {file = "clickhouse_connect-0.7.8-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:ee9c165fe9149dbeb07a735bce4fbd928485dcf25ff0401b687fe9c64e1f4c38"}, + {file = "clickhouse_connect-0.7.8-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:7e9fbb25ec59b1a8b306dbd448c458c722fab6f718adcfd2434113f1d2d12823"}, + {file = "clickhouse_connect-0.7.8-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:c985706917d870164b385e932041dd7a396acf41fc79ffd79cf04fe5153667d6"}, + {file = "clickhouse_connect-0.7.8-cp38-cp38-win32.whl", hash = "sha256:7f6a1ffd05f18b599432d763d8f94b9e27023d4dd6a45419d4bce9bb94a8b7a8"}, + {file = "clickhouse_connect-0.7.8-cp38-cp38-win_amd64.whl", hash = "sha256:9fc821567221eea914b0bc79cd46d25b95687585e35514bd4ab20e10ac688e67"}, + {file = "clickhouse_connect-0.7.8-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:a2ba42da85cabf8077f84f06761009fb99e374eb72a9f76e5d5e3e5b796a53ac"}, + {file = "clickhouse_connect-0.7.8-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:15cc99ab4dcbc850dd541796a8310c273a9d18f8ebfc6dedd71884101c4a20a1"}, + {file = "clickhouse_connect-0.7.8-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:20e9832d3a6a2052cd1531463c492ac5175d17ff7c0e2129f0ca5525cc4f58a6"}, + {file = "clickhouse_connect-0.7.8-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a74905ada295fe4ef8788b8fd8ebfa92277da0519535d3d379a8a6a4f3f6fc86"}, + {file = "clickhouse_connect-0.7.8-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:28eca36b4b399e3e419b159e015f21a64545b75cf9a2515e5e5d8ef44fff1dba"}, + {file = "clickhouse_connect-0.7.8-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:0015aeaecffef53c725a76963cfd60f74947dce82b7df525fba9bd5c8f096182"}, + {file = "clickhouse_connect-0.7.8-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:46f90b12dd9d289fba9dcca0c49f38dc4ab34135b63f93e549cd859c3f59acf1"}, + {file = "clickhouse_connect-0.7.8-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:460d56e33d66e1cca036b4adbba0008c897852cd21f6a3b4c32fcce8739d68f8"}, + {file = "clickhouse_connect-0.7.8-cp39-cp39-win32.whl", hash = "sha256:f945c97d6adaa9bfa12ce707bab91601a3c92d6548d0d6c0e3473413094d2e90"}, + {file = "clickhouse_connect-0.7.8-cp39-cp39-win_amd64.whl", hash = "sha256:9000dbf2d3130d008488d41cfa72c9b91c14a95eed28efc142ff8dd02d753042"}, + {file = "clickhouse_connect-0.7.8-pp310-pypy310_pp73-macosx_10_9_x86_64.whl", hash = "sha256:1c0467f4b7726389c908cc8b1c700aa86c888b5cad51715285122ff40526a16c"}, + {file = "clickhouse_connect-0.7.8-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:cb7c302ff0f8224347ceb9ed047ee0ef0d264dd338b096e7432972015751aab1"}, + {file = "clickhouse_connect-0.7.8-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f41c91b3a4342050232a1e513ab673e12d8b46105a3d0ab9a588c718d5108a65"}, + {file = "clickhouse_connect-0.7.8-pp310-pypy310_pp73-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:92dba0b57d7e56be04f4906668af8852c1fbd1c73abac5a72bba46d496ead6c9"}, + {file = "clickhouse_connect-0.7.8-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:9bfd180121d4fd18d35e00134c8006cfbbbd73f52cf7709e25e64c0e3e0f18cc"}, + {file = "clickhouse_connect-0.7.8-pp38-pypy38_pp73-macosx_10_9_x86_64.whl", hash = "sha256:bd0caa30d5d2ecf64337d43086064ccf6abd21d66e2bc62b41233e6f603598ac"}, + {file = "clickhouse_connect-0.7.8-pp38-pypy38_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:cdd5babe7e6b6815afe48c162b1b84b365eb8e35eb6371df0880a71b55d18aa2"}, + {file = "clickhouse_connect-0.7.8-pp38-pypy38_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2ac22b8615db9a028d7be0afadf2655f377a243922b381d69595b3a4f6ebd830"}, + {file = "clickhouse_connect-0.7.8-pp38-pypy38_pp73-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3a2584336f70f06e50bfd274827bdac9da1a94c98e90ae2f49f612d22bcfd202"}, + {file = "clickhouse_connect-0.7.8-pp38-pypy38_pp73-win_amd64.whl", hash = "sha256:668ff0a87e06f18cc45d8fa3dcf2868cfa9d108f695fd2222c6b7efb23d66eba"}, + {file = "clickhouse_connect-0.7.8-pp39-pypy39_pp73-macosx_10_9_x86_64.whl", hash = "sha256:e3ec29fd00421a0832ad03b876d33982df87d3f4d15858d0d0bd5cd366301c4c"}, + {file = "clickhouse_connect-0.7.8-pp39-pypy39_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:09d1f7ed3d24e6b6ee26b8e3a47fccc3f86da366287728acaf1e38cfe587a78d"}, + {file = "clickhouse_connect-0.7.8-pp39-pypy39_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8467d9f9dab8869d7ca7d670b27abbb47393f69c0074060833e9fc01ed939ee5"}, + {file = "clickhouse_connect-0.7.8-pp39-pypy39_pp73-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:7282b19857f154ffbe2216c702a60b9a6aca5ae275ed6476fa4d12e18fba0941"}, + {file = "clickhouse_connect-0.7.8-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:42880228bd4c1e499543d29c667159e7eccc4011ed6ceea7a32289d2a3b981a0"}, +] + +[package.dependencies] +certifi = "*" +lz4 = "*" +pytz = "*" +urllib3 = ">=1.26" +zstandard = "*" + +[package.extras] +arrow = ["pyarrow"] +numpy = ["numpy"] +orjson = ["orjson"] +pandas = ["pandas"] +sqlalchemy = ["sqlalchemy (>1.3.21,<2.0)"] +tzlocal = ["tzlocal"] + +[[package]] +name = "clickhouse-driver" +version = "0.2.7" +description = "Python driver with native interface for ClickHouse" +optional = true +python-versions = ">=3.7, <4" +files = [ + {file = "clickhouse-driver-0.2.7.tar.gz", hash = "sha256:299cfbe6d561955d88eeab6e09f3de31e2f6daccc6fdd904a59e46357d2d28d9"}, + {file = "clickhouse_driver-0.2.7-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:c44fefc2fd44f432d5b162bfe34ad76840137c34167d46a18c554a7c7c6e3566"}, + {file = "clickhouse_driver-0.2.7-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:e018452a7bf8d8c0adf958afbc5b0d29e402fc09a1fb34e9186293eae57f3b4e"}, + {file = "clickhouse_driver-0.2.7-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ff8b09f8b13df28d2f91ee3d0d2edd9589cbda76b74acf60669112219cea8c9d"}, + {file = "clickhouse_driver-0.2.7-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:54aa91c9512fd5a73f038cae4f67ca2ff0b2f8a84de846179a31530936ef4e20"}, + {file = "clickhouse_driver-0.2.7-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:b8342a7ba31ccb393ee31dfd61173aa84c995b4ac0b44d404adc8463534233d5"}, + {file = "clickhouse_driver-0.2.7-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:199000f8adf38fade0b5a52c273a396168105539de741a18ba3e68d7fc06e0e6"}, + {file = "clickhouse_driver-0.2.7-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f60a2a40602b207506e505cfb184a81cd4b752bde17153bc0b32c3931ddb792f"}, + {file = "clickhouse_driver-0.2.7-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:5db3a26b18146b2b0b06d3f32ce588af5afaa38c719daf6f9606981514228a8b"}, + {file = "clickhouse_driver-0.2.7-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:5579a31da1f3cf49630e43fbbb11cab891b78161abdcb33908b79820b7cd3a23"}, + {file = "clickhouse_driver-0.2.7-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = "sha256:cc39f0fb761aed96917b0f55679174a50f9591afc0e696e745cd698ef822661f"}, + {file = "clickhouse_driver-0.2.7-cp310-cp310-musllinux_1_1_s390x.whl", hash = "sha256:9aa0f7c740e4e61886c6d388792c5d1a2084d4b5462e6dcfc24e30ca7e7f8e68"}, + {file = "clickhouse_driver-0.2.7-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:2caee88b6eec7b33ddbccd24501ad99ff8ff2b0a6a4471945cbfb28947a9a791"}, + {file = "clickhouse_driver-0.2.7-cp310-cp310-win32.whl", hash = "sha256:a4aef432cc7120a971eebb7ca2fddac4472e810b57e403d3a371b0c69cbb2bb0"}, + {file = "clickhouse_driver-0.2.7-cp310-cp310-win_amd64.whl", hash = "sha256:f307de7df6bc23ad5ec8a1ba1db157f4d14de673ddd4798f37790f23255605b0"}, + {file = "clickhouse_driver-0.2.7-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:cbf3ca8919bf856ca6588669a863065fb732a32a6387095f64d19038fd99db9f"}, + {file = "clickhouse_driver-0.2.7-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:ab68b3d9b9d1386adfd3a57edd47b62858a145bf7ccc7f11b31d308195d966e5"}, + {file = "clickhouse_driver-0.2.7-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:985a9d60044c5ad39c6e018b852c7105ec4ebfdf4c3abe23183b4867454e570a"}, + {file = "clickhouse_driver-0.2.7-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:6c94330054c8d92d2286898906f843f26e2f96fc2aa11a9a96a7b5593d299bf0"}, + {file = "clickhouse_driver-0.2.7-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:92938f55c8f797e50e624a4b96e685178d043cdf0ede306a7fd4e7dda19b8dfd"}, + {file = "clickhouse_driver-0.2.7-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:05bd53e9bf49c3013d06f9e6d2812872d44b150f7a2d1cf18e1498257d42330e"}, + {file = "clickhouse_driver-0.2.7-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0f1f8ed5404e283a9ded499c33eade2423fdc15e31f8a711d75e91f890d0f70b"}, + {file = "clickhouse_driver-0.2.7-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:a398085e4a1766d907ac32c282d4172db38a44243bde303372396208d1cbf4bb"}, + {file = "clickhouse_driver-0.2.7-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:fa1808593123b6056f93808f0afbc7938f06a8149cb4e381aa7b1a234c1d3c18"}, + {file = "clickhouse_driver-0.2.7-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = "sha256:0512d54ae23bd4a69278e04f42b651d7c71b63ba6043e2c6bd97b11329692f99"}, + {file = "clickhouse_driver-0.2.7-cp311-cp311-musllinux_1_1_s390x.whl", hash = "sha256:5bc2b67e7e68f74ccebf95a8b3a13f13a7c34b89b32c9813103221de14c06c8b"}, + {file = "clickhouse_driver-0.2.7-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:04a37cdafc671cb796af3e566cef0aeb39111d82aebeecd9106a049434953b26"}, + {file = "clickhouse_driver-0.2.7-cp311-cp311-win32.whl", hash = "sha256:019538c7c23e976538e5081dd2f77a8a40bf663c638a62d857ff05f42b0c9052"}, + {file = "clickhouse_driver-0.2.7-cp311-cp311-win_amd64.whl", hash = "sha256:5166643683584bc53fcadda73c65f6a9077feb472f3d167ecef1a1a7024973aa"}, + {file = "clickhouse_driver-0.2.7-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:59affab7b5a3c4aab5b6a730f606575efdefea213458de2eb14927ee4e0640f4"}, + {file = "clickhouse_driver-0.2.7-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:dcb93dd07fe65ac4f1a2bc0b8967911d4ad2152dbee000f025ea5cb575da5ecb"}, + {file = "clickhouse_driver-0.2.7-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:55a48019b79181ae1ca90e980e74c5d413c3f8829f6744e2b056646c2d435a1a"}, + {file = "clickhouse_driver-0.2.7-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:507463c9157240fd7c3246781e8c30df8db3c80bf68925b36ff3ad4a80c4b924"}, + {file = "clickhouse_driver-0.2.7-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:1e2d8d2295ee9e0cfab8ad77cb635a05da2160334b4f16ed8c3d00fbf39a2343"}, + {file = "clickhouse_driver-0.2.7-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e38c44546dcdb956b5ab0944cb3d51e8c98f816e75bab1a2254c478865bc6e7b"}, + {file = "clickhouse_driver-0.2.7-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e6690a2bdd9e7531fe50b53193279f8b35cbcd5c5ee36c0fcc112518a7d24f16"}, + {file = "clickhouse_driver-0.2.7-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:bc6b4ba0a6467fd09021aa1d87a44fb4589600d61b010fca41e0dfffd0dee322"}, + {file = "clickhouse_driver-0.2.7-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:254bbd400eb87ff547a08755bc714f712e11f7a6d3ebbbb7aaa1dd454fb16d44"}, + {file = "clickhouse_driver-0.2.7-cp312-cp312-musllinux_1_1_ppc64le.whl", hash = "sha256:7bbbe3f8b87fc1489bc15fa9c88cc9fac9d4d7d683d076f058c2c83e6ee422fd"}, + {file = "clickhouse_driver-0.2.7-cp312-cp312-musllinux_1_1_s390x.whl", hash = "sha256:745e5b18f0957d932151527f1523d0e516c199de8c589638e5f55ab2559886f3"}, + {file = "clickhouse_driver-0.2.7-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:0fa0357fb5f26149e3df86a117d3678329b85d8827b78a5a09bbf224d8dd4541"}, + {file = "clickhouse_driver-0.2.7-cp312-cp312-win32.whl", hash = "sha256:ace652af7ca94ba3cb3a04a5c363e135dc5009f31d8201903e21db9d5daf2358"}, + {file = "clickhouse_driver-0.2.7-cp312-cp312-win_amd64.whl", hash = "sha256:c0ba68489544df89e4138a14b0ec3e1e5eb102d5d3283a91d9b837c420c0ab97"}, + {file = "clickhouse_driver-0.2.7-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:66267e4ba21fa66c97ce784a5de2202d3b7d4db3e50bfcdde92830a68f6fae30"}, + {file = "clickhouse_driver-0.2.7-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6cf55c285b75c178487407721baef4980b3c6515c9c0c1a6c1ea8b001afe658e"}, + {file = "clickhouse_driver-0.2.7-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:deeb66bb56490db2157f199c6d9aa2c53f046677be430cc834fc1e74eec6e654"}, + {file = "clickhouse_driver-0.2.7-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:dfe5b4020939abeeb407b4eead598c954b1573d2d2b4f174f793b196d378b9d9"}, + {file = "clickhouse_driver-0.2.7-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:84d39506b5f8d86a1195ebde1c66aba168f34ebce6ebd828888f0625cac54774"}, + {file = "clickhouse_driver-0.2.7-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f93a27db2dcbbd3ecad36e8df4395d047cb7410e2dc69f6d037674e15442f4ee"}, + {file = "clickhouse_driver-0.2.7-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:ebc29e501e47ecbfd44c89c0e5c87b2a722049d38b9e93fdd4bea510a82e16ac"}, + {file = "clickhouse_driver-0.2.7-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:f9cc8c186fea09a94d89e5c9c4e8d05ec3a80e2f6d25673c48efec8117a13cfc"}, + {file = "clickhouse_driver-0.2.7-cp37-cp37m-musllinux_1_1_ppc64le.whl", hash = "sha256:0757dfde5410c42230b24825ea3ab904a78160520e5ceb953482e133e368733b"}, + {file = "clickhouse_driver-0.2.7-cp37-cp37m-musllinux_1_1_s390x.whl", hash = "sha256:c9f88818cf411f928c29ba295c677cd95773bd256b8490f5655fb489e0c6658c"}, + {file = "clickhouse_driver-0.2.7-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:e19952f158ebe274c65ffeb294ba378d75048a48f31b77573948d606bed019d5"}, + {file = "clickhouse_driver-0.2.7-cp37-cp37m-win32.whl", hash = "sha256:008b1f32c7c68564de8051482b72a5289b6933bca9d9b1ad1474dd448d6768ba"}, + {file = "clickhouse_driver-0.2.7-cp37-cp37m-win_amd64.whl", hash = "sha256:622933cc9834c39f03de5d43a12f13fc7133d31d6d2597e67866d4a549ca9e60"}, + {file = "clickhouse_driver-0.2.7-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:92540581e5b5f36d915f14d05c30244870fb123c74b38c645fa47663053c5471"}, + {file = "clickhouse_driver-0.2.7-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:02dfadc6111b64e01c20b8c11266cab97d4f06685a392a183af437f2f1afb990"}, + {file = "clickhouse_driver-0.2.7-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e3ca17fece86fe85d97705024bec881978271931b3d00db273c9d63244f7d606"}, + {file = "clickhouse_driver-0.2.7-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:76474f1315ca3ab484ae28ad085b8f756c8b9a755882f93912b2149290482033"}, + {file = "clickhouse_driver-0.2.7-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f5c0ff12368b34aaf58dd948b0819e5b54d261911de334d3f048328dc9354013"}, + {file = "clickhouse_driver-0.2.7-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8cd441b17294e90e313b08fabf84fcc782c191d2b9b2a924f163928202db6fcc"}, + {file = "clickhouse_driver-0.2.7-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:62aa158f61d7d84c58e8cd75b3b8340b28607e5a70132395078f578d518aaae3"}, + {file = "clickhouse_driver-0.2.7-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:bcb2a39a1fef8bf1b581f06125c2a84a5b92c939b079d1a95126e3207b05dc77"}, + {file = "clickhouse_driver-0.2.7-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:1f29cc641a65e89a51a15f6d195f565ad2761d1bd653408c6b4046c987c5fb99"}, + {file = "clickhouse_driver-0.2.7-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = "sha256:ac1a43690696bda46c9a23fc6fd79b6fe22d428a18e880bdbdf5e6aeb31008c5"}, + {file = "clickhouse_driver-0.2.7-cp38-cp38-musllinux_1_1_s390x.whl", hash = "sha256:1dd5ea4584c42f85d96ddfa7d07da2abb35a797c45e4d3a66ace149ee4977cad"}, + {file = "clickhouse_driver-0.2.7-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:a736c0af858a3c83af03848b18754ab18dc594cc7f3bf6be0b1fac682def182c"}, + {file = "clickhouse_driver-0.2.7-cp38-cp38-win32.whl", hash = "sha256:6cb8ca47f5818c1bc5814b9ff775e383f3c50059b1fd28a02cb9be1b666929f8"}, + {file = "clickhouse_driver-0.2.7-cp38-cp38-win_amd64.whl", hash = "sha256:a90e7dc92985669a5e6569356bb3028d9d475f95006d4487cb0789aa53f9489c"}, + {file = "clickhouse_driver-0.2.7-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:04b77cd6c583da9135db4a62c5a7999ae248c2dbfc0cb8e8a3d8a853b1fbfa11"}, + {file = "clickhouse_driver-0.2.7-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:c7671f8c0e8960d766b2e0eaefcae3088fccdd3920e9cd3dee8e344cfd0a6929"}, + {file = "clickhouse_driver-0.2.7-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:502d7cd28522b95a399e993ffd48487e8c12c50ce2d4e89b77b938f945304405"}, + {file = "clickhouse_driver-0.2.7-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:969739279f4010e7b5b6b2c9d2ab56a463aed11fdaed5e02424c1b3915f144f8"}, + {file = "clickhouse_driver-0.2.7-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:ed34b60f741eeb02407ea72180d77cbfc368c1be6fc2f2ff8319d1856ce67e10"}, + {file = "clickhouse_driver-0.2.7-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a667b48927f4420eb8c03fa33369edfbdf359a788897a01ac945263a2a611461"}, + {file = "clickhouse_driver-0.2.7-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1f93aa3a90f3847872d7464ec9076482b2e812c4e7d61682daedffdf3471be00"}, + {file = "clickhouse_driver-0.2.7-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:190890667215691fdf2155c3b233b39146054ab1cd854c7d91221e6ed633d71e"}, + {file = "clickhouse_driver-0.2.7-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:ff280aeac5e96c764cd31ba1077c95601337b9a97fb0b9ed4d24c64431f2c322"}, + {file = "clickhouse_driver-0.2.7-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = "sha256:01e63e35d2ab55b8eb48facf6e951968c80d27ee6703aa6c91c73d9d0a4d0efe"}, + {file = "clickhouse_driver-0.2.7-cp39-cp39-musllinux_1_1_s390x.whl", hash = "sha256:a29fb24b910dafc8c11ba882797d13ec0323a97dce80a57673116fa893d1b669"}, + {file = "clickhouse_driver-0.2.7-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:5f229a7853fc767e63143ea69889d49f6fd5623adc2f7b0f7eb360117d7e91a5"}, + {file = "clickhouse_driver-0.2.7-cp39-cp39-win32.whl", hash = "sha256:b7f34ad2ed509f48f8ed1f9b96e89765173a7b35d286c7350aa85934a11c0f49"}, + {file = "clickhouse_driver-0.2.7-cp39-cp39-win_amd64.whl", hash = "sha256:78b166597afbe490cc0cdac44fed8c8b81668f87125601dda17b154f237eef5d"}, + {file = "clickhouse_driver-0.2.7-pp310-pypy310_pp73-macosx_10_9_x86_64.whl", hash = "sha256:16ab64beb8d079cb9b3200539539a35168f524eedf890c9acefb719e25bdc96e"}, + {file = "clickhouse_driver-0.2.7-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:03e28fd50fc7c54874bf8e638a2ea87f73ae35bfbbf90123fdb395f38d62f159"}, + {file = "clickhouse_driver-0.2.7-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0677b8350acd8d186b6acd0026b62dd262d6fee428a5fa3ad9561908d4b02c39"}, + {file = "clickhouse_driver-0.2.7-pp310-pypy310_pp73-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a2f3c9e2182809131701bb28a606dec90525c7ab20490714714a4b3eb015454b"}, + {file = "clickhouse_driver-0.2.7-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:e03a1a1b30cc58c9bd2cbe25bf5e40b1f1d16d52d44ddefb3af50435d1ed613c"}, + {file = "clickhouse_driver-0.2.7-pp37-pypy37_pp73-macosx_10_9_x86_64.whl", hash = "sha256:a1be8081306a4beb12444ed8e3208e1eb6c01ed207c471b33009c13504c88139"}, + {file = "clickhouse_driver-0.2.7-pp37-pypy37_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:933b40722cbca9b1123a5bb2fb4bafafd234deae0f3481125cb6b6fa1d39aa84"}, + {file = "clickhouse_driver-0.2.7-pp37-pypy37_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3054b5022f9bf15a5f4663a7cd190f466e70a2d7b8d45429d8742c515b556c10"}, + {file = "clickhouse_driver-0.2.7-pp37-pypy37_pp73-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:61744760ee046c9a268cb801ca21bfe44c4873db9901a7cd0f3ca8830205feff"}, + {file = "clickhouse_driver-0.2.7-pp37-pypy37_pp73-win_amd64.whl", hash = "sha256:5e28427e05a72e7a4c3672e36703a2d80107ee0b3ab537e3380d726c96b07821"}, + {file = "clickhouse_driver-0.2.7-pp38-pypy38_pp73-macosx_10_9_x86_64.whl", hash = "sha256:c483f5ec836ae87803478f2a7b9daf15343078edd6a8be7364dd9db64905bbd0"}, + {file = "clickhouse_driver-0.2.7-pp38-pypy38_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:28220b794874e68bc2f06dbfff5748f1c5a3236922f59e127abd58d44ae20a3f"}, + {file = "clickhouse_driver-0.2.7-pp38-pypy38_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8c09877b59b34d5b3043ad70ec31543173cac8b64b4a8afaa89416b22fb28da5"}, + {file = "clickhouse_driver-0.2.7-pp38-pypy38_pp73-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3580f78db27119f7380627873214ae1342066f1ecb35700c1d7bf418dd70ae73"}, + {file = "clickhouse_driver-0.2.7-pp38-pypy38_pp73-win_amd64.whl", hash = "sha256:0842ac1b2f7a9ca46dac2027849b241bccd8eb8ff1c59cb0a5874042b267b733"}, + {file = "clickhouse_driver-0.2.7-pp39-pypy39_pp73-macosx_10_9_x86_64.whl", hash = "sha256:7a3fb585e2d3514196258a4a3b0267510c03477f3c2380239ade4c056ba689a7"}, + {file = "clickhouse_driver-0.2.7-pp39-pypy39_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:48ea25287566d45efbaee0857ad25e8b33ffd7fd73e89424d79fe7f532962915"}, + {file = "clickhouse_driver-0.2.7-pp39-pypy39_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ee4a4935667b59b4816a5ca77300f5dbe5a7416860551d17376426b8fefc1175"}, + {file = "clickhouse_driver-0.2.7-pp39-pypy39_pp73-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:358058cfceea9b43c4af9de81842563746f16984b34525a15b41eacf8fc2bed2"}, + {file = "clickhouse_driver-0.2.7-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:ae760fb843dec0b5c398536ca8dfaf243f494ba8fc68132ae1bd62004b0c396a"}, +] + +[package.dependencies] +pytz = "*" +tzlocal = "*" + +[package.extras] +lz4 = ["clickhouse-cityhash (>=1.0.2.1)", "lz4", "lz4 (<=3.0.1)"] +numpy = ["numpy (>=1.12.0)", "pandas (>=0.24.0)"] +zstd = ["clickhouse-cityhash (>=1.0.2.1)", "zstd"] + [[package]] name = "colorama" version = "0.4.6" @@ -7107,6 +7319,7 @@ files = [ {file = "PyYAML-6.0.1-cp311-cp311-win_amd64.whl", hash = "sha256:bf07ee2fef7014951eeb99f56f39c9bb4af143d8aa3c21b1677805985307da34"}, {file = "PyYAML-6.0.1-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:855fb52b0dc35af121542a76b9a84f8d1cd886ea97c84703eaa6d88e37a2ad28"}, {file = "PyYAML-6.0.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:40df9b996c2b73138957fe23a16a4f0ba614f4c0efce1e9406a184b6d07fa3a9"}, + {file = "PyYAML-6.0.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a08c6f0fe150303c1c6b71ebcd7213c2858041a7e01975da3a99aed1e7a378ef"}, {file = "PyYAML-6.0.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6c22bec3fbe2524cde73d7ada88f6566758a8f7227bfbf93a408a9d86bcc12a0"}, {file = "PyYAML-6.0.1-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:8d4e9c88387b0f5c7d5f281e55304de64cf7f9c0021a3525bd3b1c542da3b0e4"}, {file = "PyYAML-6.0.1-cp312-cp312-win32.whl", hash = "sha256:d483d2cdf104e7c9fa60c544d92981f12ad66a457afae824d146093b8c294c54"}, @@ -8635,15 +8848,33 @@ files = [ [[package]] name = "tzdata" -version = "2023.3" +version = "2024.1" description = "Provider of IANA time zone data" optional = false python-versions = ">=2" files = [ - {file = "tzdata-2023.3-py2.py3-none-any.whl", hash = "sha256:7e65763eef3120314099b6939b5546db7adce1e7d6f2e179e3df563c70511eda"}, - {file = "tzdata-2023.3.tar.gz", hash = "sha256:11ef1e08e54acb0d4f95bdb1be05da659673de4acbd21bf9c69e94cc5e907a3a"}, + {file = "tzdata-2024.1-py2.py3-none-any.whl", hash = "sha256:9068bc196136463f5245e51efda838afa15aaeca9903f49050dfa2679db4d252"}, + {file = "tzdata-2024.1.tar.gz", hash = "sha256:2674120f8d891909751c38abcdfd386ac0a5a1127954fbc332af6b5ceae07efd"}, +] + +[[package]] +name = "tzlocal" +version = "5.2" +description = "tzinfo object for the local timezone" +optional = true +python-versions = ">=3.8" +files = [ + {file = "tzlocal-5.2-py3-none-any.whl", hash = "sha256:49816ef2fe65ea8ac19d19aa7a1ae0551c834303d5014c6d5a62e4cbda8047b8"}, + {file = "tzlocal-5.2.tar.gz", hash = "sha256:8d399205578f1a9342816409cc1e46a93ebd5755e39ea2d85334bea911bf0e6e"}, ] +[package.dependencies] +"backports.zoneinfo" = {version = "*", markers = "python_version < \"3.9\""} +tzdata = {version = "*", markers = "platform_system == \"Windows\""} + +[package.extras] +devenv = ["check-manifest", "pytest (>=4.3)", "pytest-cov", "pytest-mock (>=3.3)", "zest.releaser"] + [[package]] name = "uc-micro-py" version = "1.0.2" @@ -9064,11 +9295,73 @@ files = [ docs = ["furo", "jaraco.packaging (>=9.3)", "jaraco.tidelift (>=1.4)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-lint"] testing = ["big-O", "jaraco.functools", "jaraco.itertools", "more-itertools", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=2.2)", "pytest-ignore-flaky", "pytest-mypy (>=0.9.1)", "pytest-ruff"] +[[package]] +name = "zstandard" +version = "0.22.0" +description = "Zstandard bindings for Python" +optional = true +python-versions = ">=3.8" +files = [ + {file = "zstandard-0.22.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:275df437ab03f8c033b8a2c181e51716c32d831082d93ce48002a5227ec93019"}, + {file = "zstandard-0.22.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:2ac9957bc6d2403c4772c890916bf181b2653640da98f32e04b96e4d6fb3252a"}, + {file = "zstandard-0.22.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fe3390c538f12437b859d815040763abc728955a52ca6ff9c5d4ac707c4ad98e"}, + {file = "zstandard-0.22.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1958100b8a1cc3f27fa21071a55cb2ed32e9e5df4c3c6e661c193437f171cba2"}, + {file = "zstandard-0.22.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:93e1856c8313bc688d5df069e106a4bc962eef3d13372020cc6e3ebf5e045202"}, + {file = "zstandard-0.22.0-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:1a90ba9a4c9c884bb876a14be2b1d216609385efb180393df40e5172e7ecf356"}, + {file = "zstandard-0.22.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:3db41c5e49ef73641d5111554e1d1d3af106410a6c1fb52cf68912ba7a343a0d"}, + {file = "zstandard-0.22.0-cp310-cp310-win32.whl", hash = "sha256:d8593f8464fb64d58e8cb0b905b272d40184eac9a18d83cf8c10749c3eafcd7e"}, + {file = "zstandard-0.22.0-cp310-cp310-win_amd64.whl", hash = "sha256:f1a4b358947a65b94e2501ce3e078bbc929b039ede4679ddb0460829b12f7375"}, + {file = "zstandard-0.22.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:589402548251056878d2e7c8859286eb91bd841af117dbe4ab000e6450987e08"}, + {file = "zstandard-0.22.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:a97079b955b00b732c6f280d5023e0eefe359045e8b83b08cf0333af9ec78f26"}, + {file = "zstandard-0.22.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:445b47bc32de69d990ad0f34da0e20f535914623d1e506e74d6bc5c9dc40bb09"}, + {file = "zstandard-0.22.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:33591d59f4956c9812f8063eff2e2c0065bc02050837f152574069f5f9f17775"}, + {file = "zstandard-0.22.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:888196c9c8893a1e8ff5e89b8f894e7f4f0e64a5af4d8f3c410f0319128bb2f8"}, + {file = "zstandard-0.22.0-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:53866a9d8ab363271c9e80c7c2e9441814961d47f88c9bc3b248142c32141d94"}, + {file = "zstandard-0.22.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:4ac59d5d6910b220141c1737b79d4a5aa9e57466e7469a012ed42ce2d3995e88"}, + {file = "zstandard-0.22.0-cp311-cp311-win32.whl", hash = "sha256:2b11ea433db22e720758cba584c9d661077121fcf60ab43351950ded20283440"}, + {file = "zstandard-0.22.0-cp311-cp311-win_amd64.whl", hash = "sha256:11f0d1aab9516a497137b41e3d3ed4bbf7b2ee2abc79e5c8b010ad286d7464bd"}, + {file = "zstandard-0.22.0-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:6c25b8eb733d4e741246151d895dd0308137532737f337411160ff69ca24f93a"}, + {file = "zstandard-0.22.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:f9b2cde1cd1b2a10246dbc143ba49d942d14fb3d2b4bccf4618d475c65464912"}, + {file = "zstandard-0.22.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a88b7df61a292603e7cd662d92565d915796b094ffb3d206579aaebac6b85d5f"}, + {file = "zstandard-0.22.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:466e6ad8caefb589ed281c076deb6f0cd330e8bc13c5035854ffb9c2014b118c"}, + {file = "zstandard-0.22.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a1d67d0d53d2a138f9e29d8acdabe11310c185e36f0a848efa104d4e40b808e4"}, + {file = "zstandard-0.22.0-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:39b2853efc9403927f9065cc48c9980649462acbdf81cd4f0cb773af2fd734bc"}, + {file = "zstandard-0.22.0-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:8a1b2effa96a5f019e72874969394edd393e2fbd6414a8208fea363a22803b45"}, + {file = "zstandard-0.22.0-cp312-cp312-win32.whl", hash = "sha256:88c5b4b47a8a138338a07fc94e2ba3b1535f69247670abfe422de4e0b344aae2"}, + {file = "zstandard-0.22.0-cp312-cp312-win_amd64.whl", hash = "sha256:de20a212ef3d00d609d0b22eb7cc798d5a69035e81839f549b538eff4105d01c"}, + {file = "zstandard-0.22.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:d75f693bb4e92c335e0645e8845e553cd09dc91616412d1d4650da835b5449df"}, + {file = "zstandard-0.22.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:36a47636c3de227cd765e25a21dc5dace00539b82ddd99ee36abae38178eff9e"}, + {file = "zstandard-0.22.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:68953dc84b244b053c0d5f137a21ae8287ecf51b20872eccf8eaac0302d3e3b0"}, + {file = "zstandard-0.22.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2612e9bb4977381184bb2463150336d0f7e014d6bb5d4a370f9a372d21916f69"}, + {file = "zstandard-0.22.0-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:23d2b3c2b8e7e5a6cb7922f7c27d73a9a615f0a5ab5d0e03dd533c477de23004"}, + {file = "zstandard-0.22.0-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:1d43501f5f31e22baf822720d82b5547f8a08f5386a883b32584a185675c8fbf"}, + {file = "zstandard-0.22.0-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:a493d470183ee620a3df1e6e55b3e4de8143c0ba1b16f3ded83208ea8ddfd91d"}, + {file = "zstandard-0.22.0-cp38-cp38-win32.whl", hash = "sha256:7034d381789f45576ec3f1fa0e15d741828146439228dc3f7c59856c5bcd3292"}, + {file = "zstandard-0.22.0-cp38-cp38-win_amd64.whl", hash = "sha256:d8fff0f0c1d8bc5d866762ae95bd99d53282337af1be9dc0d88506b340e74b73"}, + {file = "zstandard-0.22.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:2fdd53b806786bd6112d97c1f1e7841e5e4daa06810ab4b284026a1a0e484c0b"}, + {file = "zstandard-0.22.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:73a1d6bd01961e9fd447162e137ed949c01bdb830dfca487c4a14e9742dccc93"}, + {file = "zstandard-0.22.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9501f36fac6b875c124243a379267d879262480bf85b1dbda61f5ad4d01b75a3"}, + {file = "zstandard-0.22.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:48f260e4c7294ef275744210a4010f116048e0c95857befb7462e033f09442fe"}, + {file = "zstandard-0.22.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:959665072bd60f45c5b6b5d711f15bdefc9849dd5da9fb6c873e35f5d34d8cfb"}, + {file = "zstandard-0.22.0-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:d22fdef58976457c65e2796e6730a3ea4a254f3ba83777ecfc8592ff8d77d303"}, + {file = "zstandard-0.22.0-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:a7ccf5825fd71d4542c8ab28d4d482aace885f5ebe4b40faaa290eed8e095a4c"}, + {file = "zstandard-0.22.0-cp39-cp39-win32.whl", hash = "sha256:f058a77ef0ece4e210bb0450e68408d4223f728b109764676e1a13537d056bb0"}, + {file = "zstandard-0.22.0-cp39-cp39-win_amd64.whl", hash = "sha256:e9e9d4e2e336c529d4c435baad846a181e39a982f823f7e4495ec0b0ec8538d2"}, + {file = "zstandard-0.22.0.tar.gz", hash = "sha256:8226a33c542bcb54cd6bd0a366067b610b41713b64c9abec1bc4533d69f51e70"}, +] + +[package.dependencies] +cffi = {version = ">=1.11", markers = "platform_python_implementation == \"PyPy\""} + +[package.extras] +cffi = ["cffi (>=1.11)"] + [extras] athena = ["botocore", "pyarrow", "pyathena", "s3fs"] az = ["adlfs"] bigquery = ["gcsfs", "google-cloud-bigquery", "grpcio", "pyarrow"] cli = ["cron-descriptor", "pipdeptree"] +clickhouse = ["adlfs", "clickhouse-connect", "clickhouse-driver", "gcsfs", "pyarrow", "s3fs"] databricks = ["databricks-sql-connector"] dbt = ["dbt-athena-community", "dbt-bigquery", "dbt-core", "dbt-databricks", "dbt-duckdb", "dbt-redshift", "dbt-snowflake"] dremio = ["pyarrow"] @@ -9090,4 +9383,4 @@ weaviate = ["weaviate-client"] [metadata] lock-version = "2.0" python-versions = ">=3.8.1,<3.13" -content-hash = "dfd9c83255cedff494fa28475473f232cae56ec49451f770c6940f0cb3e2b33e" +content-hash = "85921eb9304af4af4126161624e3d5fb6945c1d5877b7d70903199d92639180d" diff --git a/pyproject.toml b/pyproject.toml index fbac23d7ca..d337ba75fb 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -80,8 +80,11 @@ pyodbc = {version = "^4.0.39", optional = true} qdrant-client = {version = "^1.6.4", optional = true, extras = ["fastembed"]} databricks-sql-connector = {version = ">=2.9.3,<3.0.0", optional = true} dbt-databricks = {version = "^1.7.3", optional = true} +clickhouse-driver = { version = "^0.2.7", optional = true } +clickhouse-connect = { version = "^0.7.7", optional = true } black = "23.9.1" + [tool.poetry.extras] dbt = ["dbt-core", "dbt-redshift", "dbt-bigquery", "dbt-duckdb", "dbt-snowflake", "dbt-athena-community", "dbt-databricks"] gcp = ["grpcio", "google-cloud-bigquery", "db-dtypes", "gcsfs"] @@ -104,6 +107,7 @@ mssql = ["pyodbc"] synapse = ["pyodbc", "adlfs", "pyarrow"] qdrant = ["qdrant-client"] databricks = ["databricks-sql-connector"] +clickhouse = ["clickhouse-driver", "clickhouse-connect", "s3fs", "gcsfs", "adlfs", "pyarrow"] dremio = ["pyarrow"] [tool.poetry.scripts] @@ -152,7 +156,7 @@ pyjwt = "^2.8.0" pytest-mock = "^3.14.0" [tool.poetry.group.pipeline] -optional=true +optional = true [tool.poetry.group.pipeline.dependencies] google-auth-oauthlib = "^1.0.0" @@ -195,9 +199,9 @@ SQLAlchemy = ">=1.4.0" pymysql = "^1.1.0" pypdf2 = "^3.0.1" pydoc-markdown = "^4.8.2" -connectorx="0.3.2" -dbt-core=">=1.2.0" -dbt-duckdb=">=1.2.0" +connectorx = "0.3.2" +dbt-core = ">=1.2.0" +dbt-duckdb = ">=1.2.0" pymongo = ">=4.3.3" pandas = ">2" diff --git a/tests/cases.py b/tests/cases.py index 83814845a7..2b655fdc8b 100644 --- a/tests/cases.py +++ b/tests/cases.py @@ -197,6 +197,7 @@ def assert_all_data_types_row( timestamp_precision: int = 6, schema: TTableSchemaColumns = None, expect_filtered_null_columns=False, + allow_string_binary: bool = False, ) -> None: # content must equal # print(db_row) @@ -245,9 +246,14 @@ def assert_all_data_types_row( db_mapping[binary_col] ) # redshift returns binary as hex string except ValueError: - if not allow_base64_binary: + if allow_string_binary: + db_mapping[binary_col] = db_mapping[binary_col].encode("utf-8") + elif allow_base64_binary: + db_mapping[binary_col] = base64.b64decode( + db_mapping[binary_col], validate=True + ) + else: raise - db_mapping[binary_col] = base64.b64decode(db_mapping[binary_col], validate=True) else: db_mapping[binary_col] = bytes(db_mapping[binary_col]) @@ -305,7 +311,7 @@ def arrow_table_all_data_types( "datetime": pd.date_range("2021-01-01T01:02:03.1234", periods=num_rows, tz=tz, unit="us"), "bool": [random.choice([True, False]) for _ in range(num_rows)], "string_null": [random.choice(ascii_lowercase) for _ in range(num_rows - 1)] + [None], - "float_null": [round(random.uniform(0, 100), 5) for _ in range(num_rows - 1)] + [ + "float_null": [round(random.uniform(0, 100), 4) for _ in range(num_rows - 1)] + [ None ], # decrease precision "null": pd.Series([None for _ in range(num_rows)]), diff --git a/tests/load/clickhouse/__init__.py b/tests/load/clickhouse/__init__.py new file mode 100644 index 0000000000..440cb317fd --- /dev/null +++ b/tests/load/clickhouse/__init__.py @@ -0,0 +1,3 @@ +from tests.utils import skip_if_not_active + +skip_if_not_active("clickhouse") diff --git a/tests/load/clickhouse/test_clickhouse_adapter.py b/tests/load/clickhouse/test_clickhouse_adapter.py new file mode 100644 index 0000000000..36d3ac07f7 --- /dev/null +++ b/tests/load/clickhouse/test_clickhouse_adapter.py @@ -0,0 +1,61 @@ +import dlt +from dlt.destinations.adapters import clickhouse_adapter +from tests.pipeline.utils import assert_load_info + + +def test_clickhouse_adapter() -> None: + @dlt.resource + def merge_tree_resource(): + yield {"field1": 1, "field2": 2} + + @dlt.resource + def replicated_merge_tree_resource(): + yield {"field1": 1, "field2": 2} + + @dlt.resource + def not_annotated_resource(): + yield {"field1": 1, "field2": 2} + + clickhouse_adapter(merge_tree_resource, table_engine_type="merge_tree") + clickhouse_adapter(replicated_merge_tree_resource, table_engine_type="replicated_merge_tree") + + pipe = dlt.pipeline(pipeline_name="adapter_test", destination="clickhouse", full_refresh=True) + pack = pipe.run([merge_tree_resource, replicated_merge_tree_resource, not_annotated_resource]) + + assert_load_info(pack) + + with pipe.sql_client() as client: + # get map of table names to full table names + tables = {} + for table in client._list_tables(): + if "resource" in table: + tables[table.split("___")[1]] = table + assert (len(tables.keys())) == 3 + + # check content + for full_table_name in tables.values(): + with client.execute_query(f"SELECT * FROM {full_table_name};") as cursor: + res = cursor.fetchall() + assert tuple(res[0])[:2] == (1, 2) + + # check table format + # fails now, because we do not have a cluster (I think), it will fall back to SharedMergeTree + for full_table_name in tables.values(): + with client.execute_query( + "SELECT database, name, engine, engine_full FROM system.tables WHERE name =" + f" '{full_table_name}';" + ) as cursor: + res = cursor.fetchall() + # this should test that two tables should be replicatedmergetree tables + assert tuple(res[0])[2] == "SharedMergeTree" + + # we can check the gen table sql though + with pipe.destination_client() as dest_client: + for table in tables.keys(): + sql = dest_client._get_table_update_sql( # type: ignore[attr-defined] + table, pipe.default_schema.tables[table]["columns"].values(), generate_alter=False + ) + if table == "merge_tree_resource": + assert "ENGINE = MergeTree" in sql[0] + else: + assert "ENGINE = ReplicatedMergeTree" in sql[0] diff --git a/tests/load/clickhouse/test_clickhouse_configuration.py b/tests/load/clickhouse/test_clickhouse_configuration.py new file mode 100644 index 0000000000..eb02155406 --- /dev/null +++ b/tests/load/clickhouse/test_clickhouse_configuration.py @@ -0,0 +1,76 @@ +from typing import Any, Iterator + +import pytest + +import dlt +from dlt.common.configuration.resolve import resolve_configuration +from dlt.common.libs.sql_alchemy import make_url +from dlt.common.utils import digest128 +from dlt.destinations.impl.clickhouse.clickhouse import ClickHouseClient +from dlt.destinations.impl.clickhouse.configuration import ( + ClickHouseCredentials, + ClickHouseClientConfiguration, +) +from dlt.destinations.impl.snowflake.configuration import ( + SnowflakeClientConfiguration, + SnowflakeCredentials, +) +from tests.common.configuration.utils import environment +from tests.load.utils import yield_client_with_storage + + +@pytest.fixture(scope="function") +def client() -> Iterator[ClickHouseClient]: + yield from yield_client_with_storage("clickhouse") # type: ignore + + +def test_clickhouse_connection_string_with_all_params() -> None: + url = ( + "clickhouse://user1:pass1@host1:9000/testdb?allow_experimental_lightweight_delete=1&" + "allow_experimental_object_type=1&connect_timeout=230&enable_http_compression=1&secure=0" + "&send_receive_timeout=1000" + ) + + creds = ClickHouseCredentials() + creds.parse_native_representation(url) + + assert creds.database == "testdb" + assert creds.username == "user1" + assert creds.password == "pass1" + assert creds.host == "host1" + assert creds.port == 9000 + assert creds.secure == 0 + assert creds.connect_timeout == 230 + assert creds.send_receive_timeout == 1000 + + expected = make_url(url) + + # Test URL components regardless of query param order. + assert make_url(creds.to_native_representation()) == expected + + +def test_clickhouse_configuration() -> None: + # def empty fingerprint + assert ClickHouseClientConfiguration().fingerprint() == "" + # based on host + c = resolve_configuration( + SnowflakeCredentials(), + explicit_value="clickhouse://user1:pass1@host1:9000/db1", + ) + assert SnowflakeClientConfiguration(credentials=c).fingerprint() == digest128("host1") + + +def test_clickhouse_connection_settings(client: ClickHouseClient) -> None: + """Test experimental settings are set correctly for session.""" + conn = client.sql_client.open_connection() + cursor1 = conn.cursor() + cursor2 = conn.cursor() + + cursors = [cursor1, cursor2] + + for cursor in cursors: + cursor.execute("SELECT name, value FROM system.settings") + res = cursor.fetchall() + + assert ("allow_experimental_lightweight_delete", "1") in res + assert ("enable_http_compression", "1") in res diff --git a/tests/load/clickhouse/test_clickhouse_table_builder.py b/tests/load/clickhouse/test_clickhouse_table_builder.py new file mode 100644 index 0000000000..fd3bf50907 --- /dev/null +++ b/tests/load/clickhouse/test_clickhouse_table_builder.py @@ -0,0 +1,172 @@ +from copy import deepcopy + +import pytest + +from dlt.common.configuration import resolve_configuration +from dlt.common.schema import Schema +from dlt.common.utils import custom_environ, digest128 +from dlt.common.utils import uniq_id +from dlt.destinations.impl.clickhouse.clickhouse import ClickHouseClient +from dlt.destinations.impl.clickhouse.configuration import ( + ClickHouseCredentials, + ClickHouseClientConfiguration, +) +from tests.load.utils import TABLE_UPDATE, empty_schema + + +@pytest.fixture +def clickhouse_client(empty_schema: Schema) -> ClickHouseClient: + # Return a client without opening connection. + creds = ClickHouseCredentials() + return ClickHouseClient( + empty_schema, + ClickHouseClientConfiguration(credentials=creds)._bind_dataset_name(f"test_{uniq_id()}"), + ) + + +def test_clickhouse_configuration() -> None: + # Check names normalized. + with custom_environ( + { + "DESTINATION__CLICKHOUSE__CREDENTIALS__USERNAME": "username", + "DESTINATION__CLICKHOUSE__CREDENTIALS__HOST": "host", + "DESTINATION__CLICKHOUSE__CREDENTIALS__DATABASE": "mydb", + "DESTINATION__CLICKHOUSE__CREDENTIALS__PASSWORD": "fuss_do_rah", + } + ): + C = resolve_configuration(ClickHouseCredentials(), sections=("destination", "clickhouse")) + assert C.database == "mydb" + assert C.password == "fuss_do_rah" + + # Check fingerprint. + assert ClickHouseClientConfiguration().fingerprint() == "" + # Based on host. + c = resolve_configuration( + ClickHouseCredentials(), + explicit_value="clickhouse://user1:pass@host1/db1", + ) + assert ClickHouseClientConfiguration(credentials=c).fingerprint() == digest128("host1") + + +def test_clickhouse_create_table(clickhouse_client: ClickHouseClient) -> None: + statements = clickhouse_client._get_table_update_sql("event_test_table", TABLE_UPDATE, False) + assert len(statements) == 1 + sql = statements[0] + + # sqlfluff struggles with clickhouse's backtick escape characters. + # sqlfluff.parse(sql, dialect="clickhouse") + + assert sql.strip().startswith("CREATE TABLE") + assert "event_test_table" in sql + assert "`col1` Int64" in sql + assert "`col2` Float64" in sql + assert "`col3` Boolean" in sql + assert "`col4` DateTime64(6,'UTC')" in sql + assert "`col5` String" in sql + assert "`col6` Decimal(38,9)" in sql + assert "`col7` String" in sql + assert "`col8` Decimal(76,0)" in sql + assert "`col9` String" in sql + assert "`col10` Date" in sql + assert "`col11` String" in sql + assert "`col1_null` Nullable(Int64)" in sql + assert "`col2_null` Nullable(Float64)" in sql + assert "`col3_null` Nullable(Boolean)" in sql + assert "`col4_null` Nullable(DateTime64(6,'UTC'))" in sql + assert "`col5_null` Nullable(String)" in sql + assert "`col6_null` Nullable(Decimal(38,9))" in sql + assert "`col7_null` Nullable(String)" in sql + assert "`col8_null` Nullable(Decimal(76,0))" in sql + assert "`col9_null` Nullable(String)" in sql + assert "`col10_null` Nullable(Date)" in sql + assert "`col11_null` Nullable(String)" in sql + assert "`col1_precision` Int64" in sql + assert "`col4_precision` DateTime64(3,'UTC')" in sql + assert "`col5_precision` String" in sql + assert "`col6_precision` Decimal(6,2)" in sql + assert "`col7_precision` String" in sql + assert "`col11_precision` String" in sql + + +def test_clickhouse_alter_table(clickhouse_client: ClickHouseClient) -> None: + statements = clickhouse_client._get_table_update_sql("event_test_table", TABLE_UPDATE, True) + assert len(statements) == 1 + sql = statements[0] + + # sqlfluff struggles with clickhouse's backtick escape characters. + # sqlfluff.parse(sql, dialect="clickhouse") + + # Alter table statements only accept `Nullable` modifiers. + assert sql.startswith("ALTER TABLE") + assert sql.count("ALTER TABLE") == 1 + assert "event_test_table" in sql + assert "`col1` Int64" in sql + assert "`col2` Float64" in sql + assert "`col3` Boolean" in sql + assert "`col4` DateTime64(6,'UTC')" in sql + assert "`col5` String" in sql + assert "`col6` Decimal(38,9)" in sql + assert "`col7` String" in sql + assert "`col8` Decimal(76,0)" in sql + assert "`col9` String" in sql + assert "`col10` Date" in sql + assert "`col11` String" in sql + assert "`col1_null` Nullable(Int64)" in sql + assert "`col2_null` Nullable(Float64)" in sql + assert "`col3_null` Nullable(Boolean)" in sql + assert "`col4_null` Nullable(DateTime64(6,'UTC'))" in sql + assert "`col5_null` Nullable(String)" in sql + assert "`col6_null` Nullable(Decimal(38,9))" in sql + assert "`col7_null` Nullable(String)" in sql + assert "`col8_null` Nullable(Decimal(76,0))" in sql + assert "`col9_null` Nullable(String)" in sql + assert "`col10_null` Nullable(Date)" in sql + assert "`col11_null` Nullable(String)" in sql + assert "`col1_precision` Int64" in sql + assert "`col4_precision` DateTime64(3,'UTC')" in sql + assert "`col5_precision` String" in sql + assert "`col6_precision` Decimal(6,2)" in sql + assert "`col7_precision` String" in sql + assert "`col11_precision` String" in sql + + mod_table = deepcopy(TABLE_UPDATE) + mod_table.pop(0) + sql = clickhouse_client._get_table_update_sql("event_test_table", mod_table, True)[0] + + assert "`col1`" not in sql + assert "`col2` Float64" in sql + + +@pytest.mark.usefixtures("empty_schema") +def test_clickhouse_create_table_with_primary_keys(clickhouse_client: ClickHouseClient) -> None: + mod_update = deepcopy(TABLE_UPDATE) + + mod_update[1]["primary_key"] = True + mod_update[4]["primary_key"] = True + statements = clickhouse_client._get_table_update_sql("event_test_table", mod_update, False) + assert len(statements) == 1 + sql = statements[0] + + assert sql.endswith("PRIMARY KEY (`col2`, `col5`)") + + +@pytest.mark.skip( + "Only `primary_key` hint has been implemented so far, which isn't specified inline with the" + " column definition." +) +def test_clickhouse_create_table_with_hints(client: ClickHouseClient) -> None: + mod_update = deepcopy(TABLE_UPDATE) + + mod_update[0]["primary_key"] = True + mod_update[0]["sort"] = True + mod_update[1]["cluster"] = True + mod_update[4]["cluster"] = True + + sql = client._get_table_update_sql("event_test_table", TABLE_UPDATE, True) + + assert "`col1` bigint SORTKEY NOT NULL" in sql + assert "`col2` double precision DISTKEY NOT NULL" in sql + assert "`col5` varchar(max) DISTKEY" in sql + # No hints. + assert "`col3` boolean NOT NULL" in sql + assert "`col4` timestamp with time zone NOT NULL" in sql diff --git a/tests/load/pipeline/test_arrow_loading.py b/tests/load/pipeline/test_arrow_loading.py index 2ba633504e..c5a37ee5bb 100644 --- a/tests/load/pipeline/test_arrow_loading.py +++ b/tests/load/pipeline/test_arrow_loading.py @@ -1,15 +1,15 @@ from datetime import datetime # noqa: I251 -from typing import Any, Union, List, Dict, Tuple, Literal import os import pytest import numpy as np import pyarrow as pa import pandas as pd +import base64 import dlt from dlt.common import pendulum -from dlt.common.time import reduce_pendulum_datetime_precision +from dlt.common.time import reduce_pendulum_datetime_precision, ensure_pendulum_datetime from dlt.common.utils import uniq_id from tests.load.utils import destinations_configs, DestinationTestConfiguration from tests.pipeline.utils import assert_load_info, select_data @@ -46,11 +46,13 @@ def test_load_arrow_item( "redshift", "databricks", "synapse", + "clickhouse", ) # athena/redshift can't load TIME columns include_binary = not ( destination_config.destination in ("redshift", "databricks") and destination_config.file_format == "jsonl" ) + include_decimal = not ( destination_config.destination == "databricks" and destination_config.file_format == "jsonl" ) @@ -106,15 +108,30 @@ def some_data(): row[i] = row[i].tobytes() if destination_config.destination == "redshift": - # Binary columns are hex formatted in results + # Redshift needs hex string for record in records: if "binary" in record: record["binary"] = record["binary"].hex() + if destination_config.destination == "clickhouse": + for record in records: + # Clickhouse needs base64 string for jsonl + if "binary" in record and destination_config.file_format == "jsonl": + record["binary"] = base64.b64encode(record["binary"]).decode("ascii") + if "binary" in record and destination_config.file_format == "parquet": + record["binary"] = record["binary"].decode("ascii") + for row in rows: for i in range(len(row)): if isinstance(row[i], datetime): row[i] = pendulum.instance(row[i]) + # clickhouse produces rounding errors on double with jsonl, so we round the result coming from there + if ( + destination_config.destination == "clickhouse" + and destination_config.file_format == "jsonl" + and isinstance(row[i], float) + ): + row[i] = round(row[i], 4) expected = sorted([list(r.values()) for r in records]) @@ -133,6 +150,7 @@ def some_data(): for row, expected_row in zip(rows, expected): # Compare without _dlt_id/_dlt_load_id columns + assert row[3] == expected_row[3] assert row[:-2] == expected_row # Load id and dlt_id are set assert row[-2] == load_id diff --git a/tests/load/pipeline/test_clickhouse.py b/tests/load/pipeline/test_clickhouse.py new file mode 100644 index 0000000000..2ba5cfdcb8 --- /dev/null +++ b/tests/load/pipeline/test_clickhouse.py @@ -0,0 +1,83 @@ +from typing import Iterator + +import pytest + +import dlt +from dlt.common.typing import TDataItem +from dlt.common.utils import uniq_id +from tests.load.pipeline.utils import ( + destinations_configs, + DestinationTestConfiguration, +) +from tests.pipeline.utils import load_table_counts + + +@pytest.mark.parametrize( + "destination_config", + destinations_configs(default_sql_configs=True, all_staging_configs=True, subset=["clickhouse"]), + ids=lambda x: x.name, +) +def test_clickhouse_destination_append(destination_config: DestinationTestConfiguration) -> None: + pipeline = destination_config.setup_pipeline(f"clickhouse_{uniq_id()}", full_refresh=True) + + try: + + @dlt.resource(name="items", write_disposition="append") + def items() -> Iterator[TDataItem]: + yield { + "id": 1, + "name": "item", + "sub_items": [ + {"id": 101, "name": "sub item 101"}, + {"id": 101, "name": "sub item 102"}, + ], + } + + pipeline.run( + items, + loader_file_format=destination_config.file_format, + staging=destination_config.staging, + ) + + table_counts = load_table_counts( + pipeline, *[t["name"] for t in pipeline.default_schema._schema_tables.values()] + ) + assert table_counts["items"] == 1 + assert table_counts["items__sub_items"] == 2 + assert table_counts["_dlt_loads"] == 1 + + # Load again with schema evolution. + @dlt.resource(name="items", write_disposition="append") + def items2() -> Iterator[TDataItem]: + yield { + "id": 1, + "name": "item", + "new_field": "hello", + "sub_items": [ + { + "id": 101, + "name": "sub item 101", + "other_new_field": "hello 101", + }, + { + "id": 101, + "name": "sub item 102", + "other_new_field": "hello 102", + }, + ], + } + + pipeline.run(items2) + table_counts = load_table_counts( + pipeline, *[t["name"] for t in pipeline.default_schema._schema_tables.values()] + ) + assert table_counts["items"] == 2 + assert table_counts["items__sub_items"] == 4 + assert table_counts["_dlt_loads"] == 2 + + except Exception as e: + raise e + + finally: + with pipeline.sql_client() as client: + client.drop_dataset() diff --git a/tests/load/pipeline/test_merge_disposition.py b/tests/load/pipeline/test_merge_disposition.py index 8fa018c6c3..14840896df 100644 --- a/tests/load/pipeline/test_merge_disposition.py +++ b/tests/load/pipeline/test_merge_disposition.py @@ -149,7 +149,8 @@ def load_issues(): with open( "tests/normalize/cases/github.issues.load_page_5_duck.json", "r", encoding="utf-8" ) as f: - yield from json.load(f) + for item in json.load(f): + yield item return load_issues diff --git a/tests/load/pipeline/test_pipelines.py b/tests/load/pipeline/test_pipelines.py index 03d7819c73..7c9f281426 100644 --- a/tests/load/pipeline/test_pipelines.py +++ b/tests/load/pipeline/test_pipelines.py @@ -797,8 +797,8 @@ def other_data(): data_types = deepcopy(TABLE_ROW_ALL_DATA_TYPES) column_schemas = deepcopy(TABLE_UPDATE_COLUMNS_SCHEMA) - # parquet on bigquery does not support JSON but we still want to run the test - if destination_config.destination == "bigquery": + # parquet on bigquery and clickhouse does not support JSON but we still want to run the test + if destination_config.destination in ["bigquery"]: column_schemas["col9_null"]["data_type"] = column_schemas["col9"]["data_type"] = "text" # duckdb 0.9.1 does not support TIME other than 6 @@ -811,7 +811,13 @@ def other_data(): column_schemas["col4_precision"]["precision"] = 6 # drop TIME from databases not supporting it via parquet - if destination_config.destination in ["redshift", "athena", "synapse", "databricks"]: + if destination_config.destination in [ + "redshift", + "athena", + "synapse", + "databricks", + "clickhouse", + ]: data_types.pop("col11") data_types.pop("col11_null") data_types.pop("col11_precision") @@ -868,6 +874,7 @@ def some_source(): schema=column_schemas, parse_complex_strings=destination_config.destination in ["snowflake", "bigquery", "redshift"], + allow_string_binary=destination_config.destination == "clickhouse", timestamp_precision=3 if destination_config.destination in ("athena", "dremio") else 6, ) @@ -906,7 +913,7 @@ def table_1(): yield {"id": 1} @dlt.resource( - columns=[{"name": "id", "data_type": "bigint", "nullable": True}], + columns=[{"name": "id", "data_type": "bigint", "nullable": True, "unique": True}], write_disposition="merge", ) def table_2(): diff --git a/tests/load/pipeline/test_scd2.py b/tests/load/pipeline/test_scd2.py index d5d76580d7..e8baa33ff3 100644 --- a/tests/load/pipeline/test_scd2.py +++ b/tests/load/pipeline/test_scd2.py @@ -81,7 +81,7 @@ def assert_records_as_set(actual: List[Dict[str, Any]], expected: List[Dict[str, for dconf in destinations_configs(default_sql_configs=True, supports_merge=True) ] + [ - (dconf, True, None, pendulum.DateTime(3234, 12, 31, 22, 2, 59)) # arbitrary timestamp + (dconf, True, None, pendulum.DateTime(2099, 12, 31, 22, 2, 59)) # arbitrary timestamp for dconf in destinations_configs(default_sql_configs=True, supports_merge=True) ] + [ # test nested columns and validity column name configuration only for postgres and duckdb diff --git a/tests/load/pipeline/test_stage_loading.py b/tests/load/pipeline/test_stage_loading.py index 3c0207c9bc..e0e2154b57 100644 --- a/tests/load/pipeline/test_stage_loading.py +++ b/tests/load/pipeline/test_stage_loading.py @@ -180,6 +180,7 @@ def test_all_data_types(destination_config: DestinationTestConfiguration) -> Non "redshift", "athena", "databricks", + "clickhouse", ) and destination_config.file_format in ("parquet", "jsonl"): # Redshift copy doesn't support TIME column exclude_types.append("time") @@ -203,9 +204,9 @@ def test_all_data_types(destination_config: DestinationTestConfiguration) -> Non exclude_types=exclude_types, exclude_columns=exclude_columns ) - # bigquery cannot load into JSON fields from parquet + # bigquery and clickhouse cannot load into JSON fields from parquet if destination_config.file_format == "parquet": - if destination_config.destination == "bigquery": + if destination_config.destination in ["bigquery"]: # change datatype to text and then allow for it in the assert (parse_complex_strings) column_schemas["col9_null"]["data_type"] = column_schemas["col9"]["data_type"] = "text" # redshift cannot load from json into VARBYTE @@ -241,13 +242,18 @@ def my_source(): ) allow_base64_binary = ( destination_config.file_format == "jsonl" - and destination_config.destination in ["redshift"] + and destination_config.destination in ["redshift", "clickhouse"] + ) + allow_string_binary = ( + destination_config.file_format == "parquet" + and destination_config.destination in ["clickhouse"] ) # content must equal assert_all_data_types_row( db_row[:-2], parse_complex_strings=parse_complex_strings, allow_base64_binary=allow_base64_binary, + allow_string_binary=allow_string_binary, timestamp_precision=sql_client.capabilities.timestamp_precision, schema=column_schemas, ) diff --git a/tests/load/pipeline/test_write_disposition_changes.py b/tests/load/pipeline/test_write_disposition_changes.py index 0af6263fb6..b9c2b35717 100644 --- a/tests/load/pipeline/test_write_disposition_changes.py +++ b/tests/load/pipeline/test_write_disposition_changes.py @@ -126,12 +126,18 @@ def source(): # schemaless destinations allow adding of root key without the pipeline failing # for now this is only the case for dremio # doing this will result in somewhat useless behavior - destination_allows_adding_root_key = destination_config.destination == "dremio" + destination_allows_adding_root_key = destination_config.destination in ["dremio", "clickhouse"] if destination_allows_adding_root_key and not with_root_key: + pipeline.run( + s, + table_name="items", + write_disposition="merge", + loader_file_format=destination_config.file_format, + ) return - # without a root key this will fail, it is expected + # without a root key this will fail, it is expected as adding non-nullable columns should not work if not with_root_key and destination_config.supports_merge: with pytest.raises(PipelineStepFailed): pipeline.run( diff --git a/tests/load/test_job_client.py b/tests/load/test_job_client.py index 2204d1facf..08b80af928 100644 --- a/tests/load/test_job_client.py +++ b/tests/load/test_job_client.py @@ -116,6 +116,7 @@ def test_get_update_basic_schema(client: SqlJobClientBase) -> None: # update in storage client._update_schema_in_storage(schema) + sleep(1) this_schema = client.get_stored_schema_by_hash(schema.version_hash) newest_schema = client.get_stored_schema() assert this_schema == newest_schema @@ -129,7 +130,7 @@ def test_get_update_basic_schema(client: SqlJobClientBase) -> None: first_schema._bump_version() assert first_schema.version == this_schema.version == 2 # wait to make load_newest_schema deterministic - sleep(0.1) + sleep(1) client._update_schema_in_storage(first_schema) this_schema = client.get_stored_schema_by_hash(first_schema.version_hash) newest_schema = client.get_stored_schema() @@ -388,11 +389,19 @@ def test_get_storage_table_with_all_types(client: SqlJobClientBase) -> None: "time", ): continue - # mssql and synapse have no native data type for the complex type. - if client.config.destination_type in ("mssql", "synapse") and c["data_type"] in ("complex"): + # mssql, clickhouse and synapse have no native data type for the complex type. + if client.config.destination_type in ("mssql", "synapse", "clickhouse") and c[ + "data_type" + ] in ("complex"): continue if client.config.destination_type == "databricks" and c["data_type"] in ("complex", "time"): continue + # ClickHouse has no active data type for binary or time type. + if client.config.destination_type == "clickhouse": + if c["data_type"] in ("binary", "time"): + continue + elif c["data_type"] == "complex" and c["nullable"]: + continue if client.config.destination_type == "dremio" and c["data_type"] == "complex": continue assert c["data_type"] == expected_c["data_type"] @@ -509,8 +518,11 @@ def test_load_with_all_types( pytest.skip("preferred loader file format not set, destination will only work with staging") table_name = "event_test_table" + uniq_id() column_schemas, data_types = table_update_and_row( - exclude_types=["time"] if client.config.destination_type == "databricks" else None, + exclude_types=( + ["time"] if client.config.destination_type in ["databricks", "clickhouse"] else None + ), ) + # we should have identical content with all disposition types client.schema.update_table( new_table( @@ -537,7 +549,11 @@ def test_load_with_all_types( expect_load_file(client, file_storage, query, table_name) db_row = list(client.sql_client.execute_sql(f"SELECT * FROM {canonical_name}")[0]) # content must equal - assert_all_data_types_row(db_row, schema=column_schemas) + assert_all_data_types_row( + db_row, + schema=column_schemas, + allow_base64_binary=client.config.destination_type in ["clickhouse"], + ) @pytest.mark.parametrize( @@ -772,18 +788,21 @@ def _load_something(_client: SqlJobClientBase, expected_rows: int) -> None: # 3 rows because we load to the same table _load_something(client, 3) - # adding new non null column will generate sync error + # adding new non null column will generate sync error, except for clickhouse, there it will work event_3_schema.tables["event_user"]["columns"]["mandatory_column"] = new_column( "mandatory_column", "text", nullable=False ) client.schema._bump_version() - with pytest.raises(DatabaseException) as py_ex: + if destination_config.destination == "clickhouse": client.update_stored_schema() - assert ( - "mandatory_column" in str(py_ex.value).lower() - or "NOT NULL" in str(py_ex.value) - or "Adding columns with constraints not yet supported" in str(py_ex.value) - ) + else: + with pytest.raises(DatabaseException) as py_ex: + client.update_stored_schema() + assert ( + "mandatory_column" in str(py_ex.value).lower() + or "NOT NULL" in str(py_ex.value) + or "Adding columns with constraints not yet supported" in str(py_ex.value) + ) def prepare_schema(client: SqlJobClientBase, case: str) -> Tuple[List[Dict[str, Any]], str]: diff --git a/tests/load/test_sql_client.py b/tests/load/test_sql_client.py index bd1ec5ba43..26d7884179 100644 --- a/tests/load/test_sql_client.py +++ b/tests/load/test_sql_client.py @@ -38,7 +38,9 @@ def client(request) -> Iterator[SqlJobClientBase]: @pytest.mark.parametrize( "client", - destinations_configs(default_sql_configs=True, exclude=["mssql", "synapse", "dremio"]), + destinations_configs( + default_sql_configs=True, exclude=["mssql", "synapse", "dremio", "clickhouse"] + ), indirect=True, ids=lambda x: x.name, ) @@ -369,7 +371,7 @@ def test_database_exceptions(client: SqlJobClientBase) -> None: with client.sql_client.execute_query(f"DELETE FROM {qualified_name} WHERE 1=1"): pass assert client.sql_client.is_dbapi_exception(term_ex.value.dbapi_exception) - if client.config.destination_type != "dremio": + if client.config.destination_type not in ["dremio", "clickhouse"]: with pytest.raises(DatabaseUndefinedRelation) as term_ex: with client.sql_client.execute_query("DROP SCHEMA UNKNOWN"): pass @@ -630,18 +632,21 @@ def assert_load_id(sql_client: SqlClientBase[TNativeConn], load_id: str) -> None def prepare_temp_table(client: SqlJobClientBase) -> str: uniq_suffix = uniq_id() table_name = f"tmp_{uniq_suffix}" - iceberg_table_suffix = "" + ddl_suffix = "" coltype = "numeric" if client.config.destination_type == "athena": - iceberg_table_suffix = ( + ddl_suffix = ( f"LOCATION '{AWS_BUCKET}/ci/{table_name}' TBLPROPERTIES ('table_type'='ICEBERG'," " 'format'='parquet');" ) coltype = "bigint" qualified_table_name = table_name + elif client.config.destination_type == "clickhouse": + ddl_suffix = "ENGINE = MergeTree() ORDER BY col" + qualified_table_name = client.sql_client.make_qualified_table_name(table_name) else: qualified_table_name = client.sql_client.make_qualified_table_name(table_name) client.sql_client.execute_sql( - f"CREATE TABLE {qualified_table_name} (col {coltype}) {iceberg_table_suffix};" + f"CREATE TABLE {qualified_table_name} (col {coltype}) {ddl_suffix};" ) return table_name diff --git a/tests/load/utils.py b/tests/load/utils.py index 2f20e91e69..74cd782a56 100644 --- a/tests/load/utils.py +++ b/tests/load/utils.py @@ -45,7 +45,7 @@ assert_all_data_types_row, ) -# bucket urls +# Bucket urls. AWS_BUCKET = dlt.config.get("tests.bucket_url_s3", str) GCS_BUCKET = dlt.config.get("tests.bucket_url_gs", str) AZ_BUCKET = dlt.config.get("tests.bucket_url_az", str) @@ -178,12 +178,13 @@ def destinations_configs( destination_configs += [ DestinationTestConfiguration(destination=destination) for destination in SQL_DESTINATIONS - if destination not in ("athena", "mssql", "synapse", "databricks", "dremio") + if destination + not in ("athena", "mssql", "synapse", "databricks", "dremio", "clickhouse") ] destination_configs += [ DestinationTestConfiguration(destination="duckdb", file_format="parquet") ] - # athena needs filesystem staging, which will be automatically set, we have to supply a bucket url though + # Athena needs filesystem staging, which will be automatically set; we have to supply a bucket url though. destination_configs += [ DestinationTestConfiguration( destination="athena", @@ -203,6 +204,11 @@ def destinations_configs( extra_info="iceberg", ) ] + destination_configs += [ + DestinationTestConfiguration( + destination="clickhouse", file_format="jsonl", supports_dbt=False + ) + ] destination_configs += [ DestinationTestConfiguration( destination="databricks", @@ -318,6 +324,49 @@ def destinations_configs( file_format="parquet", bucket_url=AZ_BUCKET, extra_info="az-authorization", + disable_compression=True, + ), + DestinationTestConfiguration( + destination="clickhouse", + staging="filesystem", + file_format="parquet", + bucket_url=GCS_BUCKET, + extra_info="gcs-authorization", + ), + DestinationTestConfiguration( + destination="clickhouse", + staging="filesystem", + file_format="parquet", + bucket_url=AWS_BUCKET, + extra_info="s3-authorization", + ), + DestinationTestConfiguration( + destination="clickhouse", + staging="filesystem", + file_format="parquet", + bucket_url=AZ_BUCKET, + extra_info="az-authorization", + ), + DestinationTestConfiguration( + destination="clickhouse", + staging="filesystem", + file_format="jsonl", + bucket_url=AZ_BUCKET, + extra_info="az-authorization", + ), + DestinationTestConfiguration( + destination="clickhouse", + staging="filesystem", + file_format="jsonl", + bucket_url=GCS_BUCKET, + extra_info="gcs-authorization", + ), + DestinationTestConfiguration( + destination="clickhouse", + staging="filesystem", + file_format="jsonl", + bucket_url=AWS_BUCKET, + extra_info="s3-authorization", ), DestinationTestConfiguration( destination="dremio", diff --git a/tests/utils.py b/tests/utils.py index 1ccb7fc5e4..580c040706 100644 --- a/tests/utils.py +++ b/tests/utils.py @@ -48,6 +48,7 @@ "destination", "synapse", "databricks", + "clickhouse", "dremio", } NON_SQL_DESTINATIONS = {"filesystem", "weaviate", "dummy", "motherduck", "qdrant", "destination"}