diff --git a/.github/workflows/test_destination_dremio.yml b/.github/workflows/test_destination_dremio.yml new file mode 100644 index 0000000000..342f547163 --- /dev/null +++ b/.github/workflows/test_destination_dremio.yml @@ -0,0 +1,89 @@ + +name: test | dremio + +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 + + ACTIVE_DESTINATIONS: "[\"dremio\"]" + 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 | dremio 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: Start dremio + run: docker-compose -f "tests/load/dremio/docker-compose.yml" up -d + + - 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 s3 -E gs -E az -E parquet --with sentry-sdk --with pipeline + + - run: | + poetry run pytest tests/load + if: runner.os != 'Windows' + name: Run tests Linux/MAC + env: + DESTINATION__DREMIO__CREDENTIALS: grpc://dremio:dremio123@localhost:32010/nas + DESTINATION__DREMIO__STAGING_DATA_SOURCE: minio + DESTINATION__FILESYSTEM__BUCKET_URL: s3://dlt-ci-test-bucket + DESTINATION__FILESYSTEM__CREDENTIALS__AWS_ACCESS_KEY_ID: minioadmin + DESTINATION__FILESYSTEM__CREDENTIALS__AWS_SECRET_ACCESS_KEY: minioadmin + DESTINATION__FILESYSTEM__CREDENTIALS__ENDPOINT_URL: http://127.0.0.1:9010 + + - run: | + poetry run pytest tests/load + if: runner.os == 'Windows' + name: Run tests Windows + shell: cmd + + - name: Stop dremio + if: always() + run: docker-compose -f "tests/load/dremio/docker-compose.yml" down -v diff --git a/.github/workflows/test_local_destinations.yml b/.github/workflows/test_local_destinations.yml index 0138e087ff..bb0a7a35f5 100644 --- a/.github/workflows/test_local_destinations.yml +++ b/.github/workflows/test_local_destinations.yml @@ -15,7 +15,8 @@ concurrency: cancel-in-progress: true env: - DLT_SECRETS_TOML: ${{ secrets.DLT_SECRETS_TOML }} + # NOTE: this workflow can't use github secrets! + # DLT_SECRETS_TOML: ${{ secrets.DLT_SECRETS_TOML }} RUNTIME__SENTRY_DSN: https://6f6f7b6f8e0f458a89be4187603b55fe@o1061158.ingest.sentry.io/4504819859914752 RUNTIME__LOG_LEVEL: ERROR @@ -23,6 +24,9 @@ env: ACTIVE_DESTINATIONS: "[\"duckdb\", \"postgres\", \"filesystem\", \"weaviate\"]" ALL_FILESYSTEM_DRIVERS: "[\"memory\", \"file\"]" + DESTINATION__WEAVIATE__VECTORIZER: text2vec-contextionary + DESTINATION__WEAVIATE__MODULE_CONFIG: "{\"text2vec-contextionary\": {\"vectorizeClassName\": false, \"vectorizePropertyName\": true}}" + jobs: get_docs_changes: name: docs changes diff --git a/dlt/common/data_writers/escape.py b/dlt/common/data_writers/escape.py index 5460657253..3200350f0b 100644 --- a/dlt/common/data_writers/escape.py +++ b/dlt/common/data_writers/escape.py @@ -119,6 +119,7 @@ def escape_redshift_identifier(v: str) -> str: escape_postgres_identifier = escape_redshift_identifier escape_athena_identifier = escape_postgres_identifier +escape_dremio_identifier = escape_postgres_identifier def escape_bigquery_identifier(v: str) -> str: diff --git a/dlt/destinations/__init__.py b/dlt/destinations/__init__.py index 4a10deffc0..13b7f7ed99 100644 --- a/dlt/destinations/__init__.py +++ b/dlt/destinations/__init__.py @@ -13,6 +13,7 @@ from dlt.destinations.impl.destination.factory import destination from dlt.destinations.impl.synapse.factory import synapse from dlt.destinations.impl.databricks.factory import databricks +from dlt.destinations.impl.dremio.factory import dremio __all__ = [ @@ -30,5 +31,6 @@ "weaviate", "synapse", "databricks", + "dremio", "destination", ] diff --git a/dlt/destinations/impl/dremio/__init__.py b/dlt/destinations/impl/dremio/__init__.py new file mode 100644 index 0000000000..b4bde2fe6d --- /dev/null +++ b/dlt/destinations/impl/dremio/__init__.py @@ -0,0 +1,27 @@ +from dlt.common.arithmetics import DEFAULT_NUMERIC_PRECISION, DEFAULT_NUMERIC_SCALE +from dlt.common.data_writers.escape import escape_dremio_identifier +from dlt.common.destination import DestinationCapabilitiesContext + + +def capabilities() -> DestinationCapabilitiesContext: + caps = DestinationCapabilitiesContext() + caps.preferred_loader_file_format = None + caps.supported_loader_file_formats = [] + caps.preferred_staging_file_format = "parquet" + caps.supported_staging_file_formats = ["jsonl", "parquet"] + caps.escape_identifier = escape_dremio_identifier + caps.decimal_precision = (DEFAULT_NUMERIC_PRECISION, DEFAULT_NUMERIC_SCALE) + caps.wei_precision = (DEFAULT_NUMERIC_PRECISION, 0) + caps.max_identifier_length = 255 + caps.max_column_identifier_length = 255 + caps.max_query_length = 2 * 1024 * 1024 + caps.is_max_query_length_in_bytes = True + caps.max_text_data_type_length = 16 * 1024 * 1024 + caps.is_max_text_data_type_length_in_bytes = True + caps.supports_transactions = False + caps.supports_ddl_transactions = False + caps.alter_add_multi_column = True + caps.supports_clone_table = False + caps.supports_multiple_statements = False + caps.timestamp_precision = 3 + return caps diff --git a/dlt/destinations/impl/dremio/configuration.py b/dlt/destinations/impl/dremio/configuration.py new file mode 100644 index 0000000000..9b1e52f292 --- /dev/null +++ b/dlt/destinations/impl/dremio/configuration.py @@ -0,0 +1,43 @@ +import dataclasses +from typing import Final, Optional, Any, Dict, ClassVar, List + +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.typing import TSecretStrValue +from dlt.common.utils import digest128 + + +@configspec(init=False) +class DremioCredentials(ConnectionStringCredentials): + drivername: str = "grpc" + username: str = None + password: TSecretStrValue = None + host: str = None + port: Optional[int] = 32010 + database: str = None + + __config_gen_annotations__: ClassVar[List[str]] = ["port"] + + def to_native_credentials(self) -> str: + return URL.create( + drivername=self.drivername, host=self.host, port=self.port + ).render_as_string(hide_password=False) + + def db_kwargs(self) -> Dict[str, Any]: + return dict(username=self.username, password=self.password) + + +@configspec +class DremioClientConfiguration(DestinationClientDwhWithStagingConfiguration): + destination_type: Final[str] = dataclasses.field(default="dremio", init=False, repr=False, compare=False) # type: ignore[misc] + credentials: DremioCredentials = None + staging_data_source: str = None + """The name of the staging data source""" + + 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/dremio/dremio.py b/dlt/destinations/impl/dremio/dremio.py new file mode 100644 index 0000000000..23bca0ad74 --- /dev/null +++ b/dlt/destinations/impl/dremio/dremio.py @@ -0,0 +1,241 @@ +from typing import ClassVar, Optional, Sequence, Tuple, List, Any +from urllib.parse import urlparse + +from dlt.common.destination import DestinationCapabilitiesContext +from dlt.common.destination.reference import ( + FollowupJob, + TLoadJobState, + LoadJob, + SupportsStagingDestination, + NewLoadJob, +) +from dlt.common.schema import TColumnSchema, Schema, TTableSchemaColumns +from dlt.common.schema.typing import TTableSchema, TColumnType, TTableFormat, TColumnSchemaBase +from dlt.common.storages.file_storage import FileStorage +from dlt.common.utils import uniq_id +from dlt.destinations.exceptions import LoadJobTerminalException +from dlt.destinations.impl.dremio import capabilities +from dlt.destinations.impl.dremio.configuration import DremioClientConfiguration +from dlt.destinations.impl.dremio.sql_client import DremioSqlClient +from dlt.destinations.job_client_impl import SqlJobClientWithStaging +from dlt.destinations.job_impl import EmptyLoadJob +from dlt.destinations.job_impl import NewReferenceJob +from dlt.destinations.sql_jobs import SqlMergeJob +from dlt.destinations.type_mapping import TypeMapper +from dlt.destinations.sql_client import SqlClientBase + + +class DremioTypeMapper(TypeMapper): + BIGINT_PRECISION = 19 + sct_to_unbound_dbt = { + "complex": "VARCHAR", + "text": "VARCHAR", + "double": "DOUBLE", + "bool": "BOOLEAN", + "date": "DATE", + "timestamp": "TIMESTAMP", + "bigint": "BIGINT", + "binary": "VARBINARY", + "time": "TIME", + } + + sct_to_dbt = { + "decimal": "DECIMAL(%i,%i)", + "wei": "DECIMAL(%i,%i)", + } + + dbt_to_sct = { + "VARCHAR": "text", + "DOUBLE": "double", + "FLOAT": "double", + "BOOLEAN": "bool", + "DATE": "date", + "TIMESTAMP": "timestamp", + "VARBINARY": "binary", + "BINARY": "binary", + "BINARY VARYING": "binary", + "VARIANT": "complex", + "TIME": "time", + "BIGINT": "bigint", + "DECIMAL": "decimal", + } + + def from_db_type( + self, db_type: str, precision: Optional[int] = None, scale: Optional[int] = None + ) -> TColumnType: + if db_type == "DECIMAL": + if (precision, scale) == self.capabilities.wei_precision: + return dict(data_type="wei") + return dict(data_type="decimal", precision=precision, scale=scale) + return super().from_db_type(db_type, precision, scale) + + +class DremioMergeJob(SqlMergeJob): + @classmethod + def _new_temp_table_name(cls, name_prefix: str, sql_client: SqlClientBase[Any]) -> str: + return sql_client.make_qualified_table_name(f"_temp_{name_prefix}_{uniq_id()}") + + @classmethod + def _to_temp_table(cls, select_sql: str, temp_table_name: str) -> str: + return f"CREATE TABLE {temp_table_name} AS {select_sql};" + + @classmethod + def default_order_by(cls) -> str: + return "NULL" + + +class DremioLoadJob(LoadJob, FollowupJob): + def __init__( + self, + file_path: str, + table_name: str, + client: DremioSqlClient, + stage_name: Optional[str] = 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) + + # extract and prepare some vars + bucket_path = ( + NewReferenceJob.resolve_reference(file_path) + if NewReferenceJob.is_reference_job(file_path) + else "" + ) + + if not bucket_path: + raise RuntimeError("Could not resolve bucket path.") + + file_name = ( + FileStorage.get_file_name_from_file_path(bucket_path) if bucket_path else file_name + ) + + bucket_url = urlparse(bucket_path) + bucket_scheme = bucket_url.scheme + if bucket_scheme == "s3" and stage_name: + from_clause = ( + f"FROM '@{stage_name}/{bucket_url.hostname}/{bucket_url.path.lstrip('/')}'" + ) + else: + raise LoadJobTerminalException( + file_path, "Only s3 staging currently supported in Dremio destination" + ) + + source_format = file_name.split(".")[-1] + + client.execute_sql(f"""COPY INTO {qualified_table_name} + {from_clause} + FILE_FORMAT '{source_format}' + """) + + def state(self) -> TLoadJobState: + return "completed" + + def exception(self) -> str: + raise NotImplementedError() + + +class DremioClient(SqlJobClientWithStaging, SupportsStagingDestination): + capabilities: ClassVar[DestinationCapabilitiesContext] = capabilities() + + def __init__(self, schema: Schema, config: DremioClientConfiguration) -> None: + sql_client = DremioSqlClient(config.normalize_dataset_name(schema), config.credentials) + super().__init__(schema, config, sql_client) + self.config: DremioClientConfiguration = config + self.sql_client: DremioSqlClient = sql_client # type: ignore + self.type_mapper = DremioTypeMapper(self.capabilities) + + def start_file_load(self, table: TTableSchema, file_path: str, load_id: str) -> LoadJob: + job = super().start_file_load(table, file_path, load_id) + + if not job: + job = DremioLoadJob( + file_path=file_path, + table_name=table["name"], + client=self.sql_client, + stage_name=self.config.staging_data_source, + ) + return job + + def restore_file_load(self, file_path: str) -> LoadJob: + return EmptyLoadJob.from_file_path(file_path, "completed") + + def _get_table_update_sql( + self, + table_name: str, + new_columns: Sequence[TColumnSchema], + generate_alter: bool, + separate_alters: bool = False, + ) -> List[str]: + sql = super()._get_table_update_sql(table_name, new_columns, generate_alter) + + if not generate_alter: + partition_list = [ + self.capabilities.escape_identifier(c["name"]) + for c in new_columns + if c.get("partition") + ] + if partition_list: + sql[0] += "\nPARTITION BY (" + ",".join(partition_list) + ")" + + sort_list = [ + self.capabilities.escape_identifier(c["name"]) for c in new_columns if c.get("sort") + ] + if sort_list: + sql[0] += "\nLOCALSORT BY (" + ",".join(sort_list) + ")" + + return sql + + def _from_db_type( + self, bq_t: str, precision: Optional[int], scale: Optional[int] + ) -> TColumnType: + return self.type_mapper.from_db_type(bq_t, precision, scale) + + def _get_column_def_sql(self, c: TColumnSchema, table_format: TTableFormat = None) -> str: + name = self.capabilities.escape_identifier(c["name"]) + return ( + f"{name} {self.type_mapper.to_db_type(c)} {self._gen_not_null(c.get('nullable', True))}" + ) + + def get_storage_table(self, table_name: str) -> Tuple[bool, TTableSchemaColumns]: + def _null_to_bool(v: str) -> bool: + if v == "NO": + return False + elif v == "YES": + return True + raise ValueError(v) + + fields = self._get_storage_table_query_columns() + table_schema = self.sql_client.fully_qualified_dataset_name(escape=False) + db_params = (table_schema, table_name) + query = f""" +SELECT {",".join(fields)} + FROM INFORMATION_SCHEMA.COLUMNS +WHERE + table_catalog = 'DREMIO' AND 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] + numeric_scale = c[4] + schema_c: TColumnSchemaBase = { + "name": c[0], + "nullable": _null_to_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 + + def _create_merge_followup_jobs(self, table_chain: Sequence[TTableSchema]) -> List[NewLoadJob]: + return [DremioMergeJob.from_table_chain(table_chain, self.sql_client)] + + def _make_add_column_sql( + self, new_columns: Sequence[TColumnSchema], table_format: TTableFormat = None + ) -> List[str]: + return ["ADD COLUMNS (" + ", ".join(self._get_column_def_sql(c) for c in new_columns) + ")"] diff --git a/dlt/destinations/impl/dremio/factory.py b/dlt/destinations/impl/dremio/factory.py new file mode 100644 index 0000000000..61895e4f90 --- /dev/null +++ b/dlt/destinations/impl/dremio/factory.py @@ -0,0 +1,49 @@ +import typing as t + +from dlt.destinations.impl.dremio.configuration import ( + DremioCredentials, + DremioClientConfiguration, +) +from dlt.destinations.impl.dremio import capabilities +from dlt.common.destination import Destination, DestinationCapabilitiesContext + +if t.TYPE_CHECKING: + from dlt.destinations.impl.dremio.dremio import DremioClient + + +class dremio(Destination[DremioClientConfiguration, "DremioClient"]): + spec = DremioClientConfiguration + + def capabilities(self) -> DestinationCapabilitiesContext: + return capabilities() + + @property + def client_class(self) -> t.Type["DremioClient"]: + from dlt.destinations.impl.dremio.dremio import DremioClient + + return DremioClient + + def __init__( + self, + credentials: t.Union[DremioCredentials, t.Dict[str, t.Any], str] = None, + staging_data_source: str = None, + destination_name: t.Optional[str] = None, + environment: t.Optional[str] = None, + **kwargs: t.Any, + ) -> None: + """Configure the Dremio 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 dremio database. Can be an instance of `DremioCredentials` or + a connection string in the format `dremio://user:password@host:port/database` + staging_data_source: The name of the "Object Storage" data source in Dremio containing the s3 bucket + """ + super().__init__( + credentials=credentials, + staging_data_source=staging_data_source, + destination_name=destination_name, + environment=environment, + **kwargs, + ) diff --git a/dlt/destinations/impl/dremio/pydremio.py b/dlt/destinations/impl/dremio/pydremio.py new file mode 100644 index 0000000000..b936278fbd --- /dev/null +++ b/dlt/destinations/impl/dremio/pydremio.py @@ -0,0 +1,287 @@ +""" + Copyright (C) 2017-2021 Dremio Corporation + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + +The code in this module was original from https://github.com/dremio-hub/arrow-flight-client-examples/tree/main/python. +The code has been modified and extended to provide a PEP 249 compatible interface. + +This implementation will eagerly gather the full result set after every query. +Eventually, this module should be replaced with ADBC Flight SQL client. +See: https://github.com/apache/arrow-adbc/issues/1559 +""" + +from dataclasses import dataclass, field +from datetime import datetime # noqa: I251 +from http.cookies import SimpleCookie +from typing import Any, List, Tuple, Optional, Mapping, Dict, AnyStr + +import pyarrow +import pytz +from pyarrow import flight + +apilevel = "2.0" +threadsafety = 2 +paramstyle = "format" + + +def connect( + uri: str, + db_kwargs: Optional[Mapping[str, Any]] = None, + conn_kwargs: Optional[Mapping[str, Any]] = None, +) -> "DremioConnection": + username = db_kwargs["username"] + password = db_kwargs["password"] + tls_root_certs = db_kwargs.get("tls_root_certs") + client = create_flight_client(location=uri, tls_root_certs=tls_root_certs) + options = create_flight_call_options( + username=username, + password=password, + client=client, + ) + return DremioConnection( + client=client, + options=options, + ) + + +def quote_string(string: str) -> str: + return "'" + string.strip("'") + "'" + + +def format_datetime(d: datetime) -> str: + return d.astimezone(pytz.UTC).replace(tzinfo=None).isoformat(sep=" ", timespec="milliseconds") + + +def format_parameter(param: Any) -> str: + if isinstance(param, str): + return quote_string(param) + elif isinstance(param, datetime): + return quote_string(format_datetime(param)) + else: + return str(param) + + +class MalformedQueryError(Exception): + pass + + +def parameterize_query(query: str, parameters: Optional[Tuple[Any, ...]]) -> str: + parameters = parameters or () + parameters = tuple(format_parameter(p) for p in parameters) + try: + return query % parameters + except TypeError as ex: + raise MalformedQueryError(*ex.args) + + +def execute_query(connection: "DremioConnection", query: str) -> pyarrow.Table: + flight_desc = flight.FlightDescriptor.for_command(query) + flight_info = connection.client.get_flight_info(flight_desc, connection.options) + return connection.client.do_get(flight_info.endpoints[0].ticket, connection.options).read_all() + + +def _any_str_to_str(string: AnyStr) -> str: + if isinstance(string, bytes): + return string.decode() + else: + return string + + +@dataclass +class DremioCursor: + connection: "DremioConnection" + table: pyarrow.Table = field(init=False, default_factory=lambda: pyarrow.table([])) + + @property + def description(self) -> List[Tuple[str, pyarrow.DataType, Any, Any, Any, Any, Any]]: + return [(fld.name, fld.type, None, None, None, None, None) for fld in self.table.schema] + + @property + def rowcount(self) -> int: + return len(self.table) + + def execute( + self, query: AnyStr, parameters: Optional[Tuple[Any, ...]] = None, *args: Any, **kwargs: Any + ) -> None: + query_str = _any_str_to_str(query) + parameterized_query = parameterize_query(query_str, parameters) + self.table = execute_query(self.connection, parameterized_query) + + def fetchall(self) -> List[Tuple[Any, ...]]: + return self.fetchmany() + + def fetchmany(self, size: Optional[int] = None) -> List[Tuple[Any, ...]]: + if size is None: + size = len(self.table) + pylist = self.table.to_pylist() + self.table = pyarrow.Table.from_pylist(pylist[size:]) + return [tuple(d.values()) for d in pylist[:size]] + + def fetchone(self) -> Optional[Tuple[Any, ...]]: + result = self.fetchmany(1) + return result[0] if result else None + + def fetch_arrow_table(self) -> pyarrow.Table: + table = self.table + self.table = pyarrow.table({col.name: [] for col in table.schema}, schema=table.schema) + return table + + def close(self) -> None: + pass + + def __enter__(self) -> "DremioCursor": + return self + + def __exit__(self, exc_type, exc_val, exc_tb) -> None: # type: ignore + self.close() + + +@dataclass(frozen=True) +class DremioConnection: + client: flight.FlightClient + options: flight.FlightCallOptions + + def close(self) -> None: + self.client.close() + + def cursor(self) -> DremioCursor: + return DremioCursor(self) + + def __enter__(self) -> "DremioConnection": + return self + + def __exit__(self, exc_type, exc_val, exc_tb) -> None: # type: ignore + self.close() + + +class DremioAuthError(Exception): + pass + + +class DremioClientAuthMiddlewareFactory(flight.ClientMiddlewareFactory): + """A factory that creates DremioClientAuthMiddleware(s).""" + + def __init__(self, *args: Any, **kwargs: Any): + super().__init__(*args, **kwargs) + self.call_credential: Optional[Tuple[bytes, bytes]] = None + + def start_call(self, info: flight.CallInfo) -> flight.ClientMiddleware: + return DremioClientAuthMiddleware(self) + + def set_call_credential(self, call_credential: Tuple[bytes, bytes]) -> None: + self.call_credential = call_credential + + +class DremioClientAuthMiddleware(flight.ClientMiddleware): + """ + A ClientMiddleware that extracts the bearer token from + the authorization header returned by the Dremio + Flight Server Endpoint. + + Parameters + ---------- + factory : ClientHeaderAuthMiddlewareFactory + The factory to set call credentials if an + authorization header with bearer token is + returned by the Dremio server. + """ + + def __init__(self, factory: flight.ClientMiddlewareFactory, *args: Any, **kwargs: Any): + super().__init__(*args, **kwargs) + self.factory = factory + + def received_headers(self, headers: Mapping[str, str]) -> None: + auth_header_key = "authorization" + authorization_header = None + for key in headers: + if key.lower() == auth_header_key: + authorization_header = headers.get(auth_header_key) + if authorization_header: + self.factory.set_call_credential( + (b"authorization", authorization_header[0].encode("utf-8")) + ) + + +class CookieMiddlewareFactory(flight.ClientMiddlewareFactory): + """A factory that creates CookieMiddleware(s).""" + + def __init__(self, *args: Any, **kwargs: Any): + super().__init__(*args, **kwargs) + self.cookies: Dict[str, Any] = {} + + def start_call(self, info: flight.CallInfo) -> flight.ClientMiddleware: + return CookieMiddleware(self) + + +class CookieMiddleware(flight.ClientMiddleware): + """ + A ClientMiddleware that receives and retransmits cookies. + For simplicity, this does not auto-expire cookies. + + Parameters + ---------- + factory : CookieMiddlewareFactory + The factory containing the currently cached cookies. + """ + + def __init__(self, factory: CookieMiddlewareFactory, *args: Any, **kwargs: Any): + super().__init__(*args, **kwargs) + self.factory = factory + + def received_headers(self, headers: Mapping[str, str]) -> None: + for key in headers: + if key.lower() == "set-cookie": + cookie = SimpleCookie() # type: ignore + for item in headers.get(key): + cookie.load(item) + + self.factory.cookies.update(cookie.items()) + + def sending_headers(self) -> Dict[bytes, bytes]: + if self.factory.cookies: + cookie_string = "; ".join( + "{!s}={!s}".format(key, val.value) for (key, val) in self.factory.cookies.items() + ) + return {b"cookie": cookie_string.encode("utf-8")} + return {} + + +# def tls_root_certs() -> bytes: +# with open("certs/ca-certificates.crt", "rb") as f: +# return f.read() + + +def create_flight_client( + location: str, tls_root_certs: Optional[bytes] = None, **kwargs: Any +) -> flight.FlightClient: + return flight.FlightClient( + location=location, + tls_root_certs=tls_root_certs, + middleware=[DremioClientAuthMiddlewareFactory(), CookieMiddlewareFactory()], + **kwargs, + ) + + +def create_flight_call_options( + username: str, password: str, client: flight.FlightClient +) -> flight.FlightCallOptions: + headers: List[Any] = [] + # Retrieve bearer token and append to the header for future calls. + bearer_token = client.authenticate_basic_token( + username, + password, + flight.FlightCallOptions(headers=headers), + ) + headers.append(bearer_token) + return flight.FlightCallOptions(headers=headers) diff --git a/dlt/destinations/impl/dremio/sql_client.py b/dlt/destinations/impl/dremio/sql_client.py new file mode 100644 index 0000000000..255c8acee0 --- /dev/null +++ b/dlt/destinations/impl/dremio/sql_client.py @@ -0,0 +1,154 @@ +from contextlib import contextmanager, suppress +from typing import Any, AnyStr, ClassVar, Iterator, Optional, Sequence, List + +import pyarrow + +from dlt.common import logger +from dlt.common.destination import DestinationCapabilitiesContext +from dlt.destinations.exceptions import ( + DatabaseTerminalException, + DatabaseUndefinedRelation, + DatabaseTransientException, +) +from dlt.destinations.impl.dremio import capabilities, pydremio +from dlt.destinations.impl.dremio.configuration import DremioCredentials +from dlt.destinations.sql_client import ( + DBApiCursorImpl, + SqlClientBase, + raise_database_error, + raise_open_connection_error, +) +from dlt.destinations.typing import DBApi, DBApiCursor, DBTransaction, DataFrame + + +class DremioCursorImpl(DBApiCursorImpl): + native_cursor: pydremio.DremioCursor # type: ignore[assignment] + + def df(self, chunk_size: int = None, **kwargs: Any) -> Optional[DataFrame]: + if chunk_size is None: + return self.native_cursor.fetch_arrow_table().to_pandas() + return super().df(chunk_size=chunk_size, **kwargs) + + +class DremioSqlClient(SqlClientBase[pydremio.DremioConnection]): + dbapi: ClassVar[DBApi] = pydremio + capabilities: ClassVar[DestinationCapabilitiesContext] = capabilities() + + def __init__(self, dataset_name: str, credentials: DremioCredentials) -> None: + super().__init__(credentials.database, dataset_name) + self._conn: Optional[pydremio.DremioConnection] = None + self.credentials = credentials + + def open_connection(self) -> pydremio.DremioConnection: + db_kwargs = self.credentials.db_kwargs() + self._conn = pydremio.connect( + uri=self.credentials.to_native_credentials(), db_kwargs=db_kwargs + ) + 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]: + logger.warning( + "Dremio does not support transactions! Each SQL statement is auto-committed separately." + ) + yield self + + @raise_database_error + def commit_transaction(self) -> None: + pass + + @raise_database_error + def rollback_transaction(self) -> None: + raise NotImplementedError("You cannot rollback Dremio SQL statements.") + + @property + def native_connection(self) -> "pydremio.DremioConnection": + return self._conn + + def drop_tables(self, *tables: str) -> None: + # Tables are drop with `IF EXISTS`, but dremio raises when the schema doesn't exist. + # Multi statement exec is safe and the error can be ignored since all tables are in the same schema. + with suppress(DatabaseUndefinedRelation): + super().drop_tables(*tables) + + def execute_sql( + self, sql: AnyStr, *args: Any, **kwargs: Any + ) -> Optional[Sequence[Sequence[Any]]]: + with self.execute_query(sql, *args, **kwargs) as curr: + if curr.description is None: + return None + else: + f = curr.fetchall() + return f + + @contextmanager + @raise_database_error + def execute_query(self, query: AnyStr, *args: Any, **kwargs: Any) -> Iterator[DBApiCursor]: + db_args = args if args else kwargs if kwargs else None + with self._conn.cursor() as curr: + try: + curr.execute(query, db_args) # type: ignore + except pydremio.MalformedQueryError as ex: + raise DatabaseTransientException(ex) + yield DremioCursorImpl(curr) # type: ignore + + def fully_qualified_dataset_name(self, escape: bool = True) -> str: + database_name = self.credentials.database + 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: + if escape: + table_name = self.capabilities.escape_identifier(table_name) + return f"{self.fully_qualified_dataset_name(escape=escape)}.{table_name}" + + @classmethod + def _make_database_exception(cls, ex: Exception) -> Exception: + if isinstance(ex, pyarrow.lib.ArrowInvalid): + msg = str(ex) + if "not found in any table" in msg: + return DatabaseTerminalException(ex) + elif "not found" in msg or "does not exist" in msg: + return DatabaseUndefinedRelation(ex) + elif "Non-query expression encountered in illegal context" in msg: + return DatabaseTransientException(ex) + else: + return DatabaseTerminalException(ex) + else: + return ex + + @staticmethod + def is_dbapi_exception(ex: Exception) -> bool: + return isinstance(ex, (pyarrow.lib.ArrowInvalid, pydremio.MalformedQueryError)) + + def create_dataset(self) -> None: + pass + + def _get_table_names(self) -> List[str]: + query = """ + SELECT TABLE_NAME + FROM INFORMATION_SCHEMA."TABLES" + WHERE TABLE_CATALOG = 'DREMIO' AND TABLE_SCHEMA = %s + """ + db_params = [self.fully_qualified_dataset_name(escape=False)] + tables = self.execute_sql(query, *db_params) or [] + return [table[0] for table in tables] + + def drop_dataset(self) -> None: + table_names = self._get_table_names() + for table_name in table_names: + full_table_name = self.make_qualified_table_name(table_name) + self.execute_sql("DROP TABLE IF EXISTS %s;" % full_table_name) + + def has_dataset(self) -> bool: + return len(self._get_table_names()) > 0 diff --git a/dlt/destinations/impl/mssql/mssql.py b/dlt/destinations/impl/mssql/mssql.py index 85e2b9e475..8de15e2bd9 100644 --- a/dlt/destinations/impl/mssql/mssql.py +++ b/dlt/destinations/impl/mssql/mssql.py @@ -140,8 +140,8 @@ def _to_temp_table(cls, select_sql: str, temp_table_name: str) -> str: return f"SELECT * INTO {temp_table_name} FROM ({select_sql}) as t;" @classmethod - def _new_temp_table_name(cls, name_prefix: str) -> str: - name = SqlMergeJob._new_temp_table_name(name_prefix) + def _new_temp_table_name(cls, name_prefix: str, sql_client: SqlClientBase[Any]) -> str: + name = SqlMergeJob._new_temp_table_name(name_prefix, sql_client) return "#" + name diff --git a/dlt/destinations/sql_jobs.py b/dlt/destinations/sql_jobs.py index 2f09414ef1..9c5a080278 100644 --- a/dlt/destinations/sql_jobs.py +++ b/dlt/destinations/sql_jobs.py @@ -195,14 +195,14 @@ def gen_key_table_clauses( @classmethod def gen_delete_temp_table_sql( - cls, unique_column: str, key_table_clauses: Sequence[str] + cls, unique_column: str, key_table_clauses: Sequence[str], sql_client: SqlClientBase[Any] ) -> Tuple[List[str], str]: """Generate sql that creates delete temp table and inserts `unique_column` from root table for all records to delete. May return several statements. Returns temp table name for cases where special names are required like SQLServer. """ sql: List[str] = [] - temp_table_name = cls._new_temp_table_name("delete") + temp_table_name = cls._new_temp_table_name("delete", sql_client) select_statement = f"SELECT d.{unique_column} {key_table_clauses[0]}" sql.append(cls._to_temp_table(select_statement, temp_table_name)) for clause in key_table_clauses[1:]: @@ -255,7 +255,7 @@ def gen_select_from_dedup_sql( 1) To select the values for an INSERT INTO statement. 2) To select the values for a temporary table used for inserts. """ - order_by = "(SELECT NULL)" + order_by = cls.default_order_by() if dedup_sort is not None: order_by = f"{dedup_sort[0]} {dedup_sort[1].upper()}" if condition is None: @@ -272,17 +272,22 @@ def gen_select_from_dedup_sql( ) AS _dlt_dedup_numbered WHERE _dlt_dedup_rn = 1 AND ({condition}) """ + @classmethod + def default_order_by(cls) -> str: + return "(SELECT NULL)" + @classmethod def gen_insert_temp_table_sql( cls, staging_root_table_name: str, + sql_client: SqlClientBase[Any], primary_keys: Sequence[str], unique_column: str, dedup_sort: Tuple[str, TSortOrder] = None, condition: str = None, condition_columns: Sequence[str] = None, ) -> Tuple[List[str], str]: - temp_table_name = cls._new_temp_table_name("insert") + temp_table_name = cls._new_temp_table_name("insert", sql_client) if len(primary_keys) > 0: # deduplicate select_sql = cls.gen_select_from_dedup_sql( @@ -314,7 +319,7 @@ def gen_delete_from_sql( """ @classmethod - def _new_temp_table_name(cls, name_prefix: str) -> str: + def _new_temp_table_name(cls, name_prefix: str, sql_client: SqlClientBase[Any]) -> str: return f"{name_prefix}_{uniq_id()}" @classmethod @@ -348,6 +353,7 @@ def gen_merge_sql( root_table_name = sql_client.make_qualified_table_name(root_table["name"]) with sql_client.with_staging_dataset(staging=True): staging_root_table_name = sql_client.make_qualified_table_name(root_table["name"]) + # get merge and primary keys from top level primary_keys = list( map( @@ -391,7 +397,7 @@ def gen_merge_sql( unique_column = escape_id(unique_columns[0]) # create temp table with unique identifier create_delete_temp_table_sql, delete_temp_table_name = cls.gen_delete_temp_table_sql( - unique_column, key_table_clauses + unique_column, key_table_clauses, sql_client ) sql.extend(create_delete_temp_table_sql) @@ -445,6 +451,7 @@ def gen_merge_sql( insert_temp_table_name, ) = cls.gen_insert_temp_table_sql( staging_root_table_name, + sql_client, primary_keys, unique_column, dedup_sort, diff --git a/docs/website/docs/dlt-ecosystem/destinations/dremio.md b/docs/website/docs/dlt-ecosystem/destinations/dremio.md new file mode 100644 index 0000000000..deb5947a06 --- /dev/null +++ b/docs/website/docs/dlt-ecosystem/destinations/dremio.md @@ -0,0 +1,121 @@ +--- +title: ๐Ÿงช Dremio +description: Dremio `dlt` destination +keywords: [dremio, iceberg, aws, glue catalog] +--- + +# Dremio + +## Install dlt with Dremio +**To install the DLT library with Dremio and s3 dependencies:** +```sh +pip install dlt[dremio,s3] +``` + +## Setup Guide +### 1. Initialize the dlt project + +Let's start by initializing a new dlt project as follows: + ```sh + dlt init chess dremio + ``` + > ๐Ÿ’ก This command will initialise your pipeline with chess as the source and aws dremio as the destination using the filesystem staging destination + + +### 2. Setup bucket storage and dremio credentials + +First install dependencies by running: +```sh +pip install -r requirements.txt +``` +or with `pip install dlt[dremio,s3]` which will install `s3fs`, `pyarrow`, and `botocore` packages. + +To edit the `dlt` credentials file with your secret info, open `.dlt/secrets.toml`. You will need to provide a `bucket_url` which holds the uploaded parquet files. + +The toml file looks like this: + +```toml +[destination.filesystem] +bucket_url = "s3://[your_bucket_name]" # replace with your bucket name, + +[destination.filesystem.credentials] +aws_access_key_id = "please set me up!" # copy the access key here +aws_secret_access_key = "please set me up!" # copy the secret access key here + +[destination.dremio] +staging_data_source = "" # the name of the "Object Storage" data source in Dremio containing the s3 bucket + +[destination.dremio.credentials] +username = "" # the dremio username +password = "" # dremio password or PAT token +database = "" # the name of the "data source" set up in Dremio where you want to load your data +host = "localhost" # the Dremio hostname +port = 32010 # the Dremio Arrow Flight grpc port +drivername="grpc" # either 'grpc' or 'grpc+tls' +``` + +You can also pass SqlAlchemy-like connection like below +```toml +[destination.dremio] +staging_data_source="s3_staging" +credentials="grpc://:@:/" +``` + +if you have your credentials stored in `~/.aws/credentials` just remove the **[destination.filesystem.credentials]** and **[destination.dremio.credentials]** section above and `dlt` will fall back to your **default** profile in local credentials. If you want to switch the profile, pass the profile name as follows (here: `dlt-ci-user`): +```toml +[destination.filesystem.credentials] +profile_name="dlt-ci-user" +``` + +## Write disposition + +`dremio` destination handles the write dispositions as follows: +- `append` +- `replace` +- `merge` + +> The `merge` write disposition uses the default DELETE/UPDATE/INSERT strategy to merge data into the destination. Be aware that Dremio does not support transactions so a partial pipeline failure can result in the destination table being in an inconsistent state. The `merge` write disposition will eventually be implemented using [MERGE INTO](https://docs.dremio.com/current/reference/sql/commands/apache-iceberg-tables/apache-iceberg-merge/) to resolve this issue. + +## Data loading + +Data loading happens by copying a staged parquet files from an object storage bucket to the destination table in Dremio using [COPY INTO](https://docs.dremio.com/cloud/reference/sql/commands/copy-into-table/) statements. The destination table format is specified by the storage format for the data source in Dremio. Typically, this will be Apache Iceberg. + +> โ— **Dremio cannot load `fixed_len_byte_array` columns from `parquet` files**. + +## Dataset Creation + +Dremio does not support `CREATE SCHEMA` DDL statements. + +Therefore, "Metastore" data sources, such as Hive or Glue, require that the dataset schema exists prior to running the DLT pipeline. `full_refresh=True` is unsupported for these data sources. + +"Object Storage" data sources do not have this limitation. + +## Staging support + +Using a staging destination is mandatory when using the dremio destination. If you do not set staging to `filesystem`, dlt will automatically do this for you. + +## Table Partitioning and Local Sort +Apache Iceberg table partitions and local sort properties can be configured as shown below: +```py +import dlt +from dlt.common.schema import TColumnSchema + +@dlt.resource( + table_name="my_table", + columns=dict( + foo=TColumnSchema(partition=True), + bar=TColumnSchema(partition=True), + baz=TColumnSchema(sort=True), + ), +) +def my_table_resource(): + ... +``` +This will result in `PARTITION BY ("foo","bar")` and `LOCALSORT BY ("baz")` clauses being added to the `CREATE TABLE` DML statement. + +> ***Note:*** Table partition migration is not implemented. The table will need to be dropped and recreated to alter partitions or localsort. + +### 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 9776de0818..418ac2efd6 100644 --- a/docs/website/sidebars.js +++ b/docs/website/sidebars.js @@ -112,6 +112,7 @@ const sidebars = { 'dlt-ecosystem/destinations/athena', 'dlt-ecosystem/destinations/weaviate', 'dlt-ecosystem/destinations/qdrant', + 'dlt-ecosystem/destinations/dremio', 'dlt-ecosystem/destinations/destination', 'dlt-ecosystem/destinations/motherduck' ] diff --git a/poetry.lock b/poetry.lock index 458614b792..a05cb28c1b 100644 --- a/poetry.lock +++ b/poetry.lock @@ -3357,164 +3357,6 @@ files = [ {file = "google_re2-1.1-1-cp39-cp39-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:9c6c9f64b9724ec38da8e514f404ac64e9a6a5e8b1d7031c2dadd05c1f4c16fd"}, {file = "google_re2-1.1-1-cp39-cp39-win32.whl", hash = "sha256:d1b751b9ab9f8e2ab2a36d72b909281ce65f328c9115a1685acae1a2d1afd7a4"}, {file = "google_re2-1.1-1-cp39-cp39-win_amd64.whl", hash = "sha256:ac775c75cec7069351d201da4e0fb0cae4c1c5ebecd08fa34e1be89740c1d80b"}, - {file = "google_re2-1.1-2-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:5eaefe4705b75ca5f78178a50104b689e9282f868e12f119b26b4cffc0c7ee6e"}, - {file = "google_re2-1.1-2-cp310-cp310-macosx_11_0_x86_64.whl", hash = "sha256:e35f2c8aabfaaa4ce6420b3cae86c0c29042b1b4f9937254347e9b985694a171"}, - {file = "google_re2-1.1-2-cp310-cp310-macosx_12_0_arm64.whl", hash = "sha256:35fd189cbaaaa39c9a6a8a00164c8d9c709bacd0c231c694936879609beff516"}, - {file = "google_re2-1.1-2-cp310-cp310-macosx_12_0_x86_64.whl", hash = "sha256:60475d222cebd066c80414831c8a42aa2449aab252084102ee05440896586e6a"}, - {file = "google_re2-1.1-2-cp310-cp310-macosx_13_0_arm64.whl", hash = "sha256:871cb85b9b0e1784c983b5c148156b3c5314cb29ca70432dff0d163c5c08d7e5"}, - {file = "google_re2-1.1-2-cp310-cp310-macosx_13_0_x86_64.whl", hash = "sha256:94f4e66e34bdb8de91ec6cdf20ba4fa9fea1dfdcfb77ff1f59700d01a0243664"}, - {file = "google_re2-1.1-2-cp310-cp310-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:1563577e2b720d267c4cffacc0f6a2b5c8480ea966ebdb1844fbea6602c7496f"}, - {file = "google_re2-1.1-2-cp310-cp310-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:49b7964532a801b96062d78c0222d155873968f823a546a3dbe63d73f25bb56f"}, - {file = "google_re2-1.1-2-cp310-cp310-manylinux_2_24_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:2362fd70eb639a75fd0187d28b4ba7b20b3088833d8ad7ffd8693d0ba159e1c2"}, - {file = "google_re2-1.1-2-cp310-cp310-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:86b80719636a4e21391e20a9adf18173ee6ae2ec956726fe2ff587417b5e8ba6"}, - {file = "google_re2-1.1-2-cp310-cp310-win32.whl", hash = "sha256:5456fba09df951fe8d1714474ed1ecda102a68ddffab0113e6c117d2e64e6f2b"}, - {file = "google_re2-1.1-2-cp310-cp310-win_amd64.whl", hash = "sha256:2ac6936a3a60d8d9de9563e90227b3aea27068f597274ca192c999a12d8baa8f"}, - {file = "google_re2-1.1-2-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:d5a87b436028ec9b0f02fe19d4cbc19ef30441085cdfcdf1cce8fbe5c4bd5e9a"}, - {file = "google_re2-1.1-2-cp311-cp311-macosx_11_0_x86_64.whl", hash = "sha256:fc0d4163de9ed2155a77e7a2d59d94c348a6bbab3cff88922fab9e0d3d24faec"}, - {file = "google_re2-1.1-2-cp311-cp311-macosx_12_0_arm64.whl", hash = "sha256:48b12d953bc796736e7831d67b36892fb6419a4cc44cb16521fe291e594bfe23"}, - {file = "google_re2-1.1-2-cp311-cp311-macosx_12_0_x86_64.whl", hash = "sha256:62c780c927cff98c1538439f0ff616f48a9b2e8837c676f53170d8ae5b9e83cb"}, - {file = "google_re2-1.1-2-cp311-cp311-macosx_13_0_arm64.whl", hash = "sha256:04b2aefd768aa4edeef8b273327806c9cb0b82e90ff52eacf5d11003ac7a0db2"}, - {file = "google_re2-1.1-2-cp311-cp311-macosx_13_0_x86_64.whl", hash = "sha256:9c90175992346519ee7546d9af9a64541c05b6b70346b0ddc54a48aa0d3b6554"}, - {file = "google_re2-1.1-2-cp311-cp311-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:22ad9ad9d125249d6386a2e80efb9de7af8260b703b6be7fa0ab069c1cf56ced"}, - {file = "google_re2-1.1-2-cp311-cp311-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:f70971f6ffe5254e476e71d449089917f50ebf9cf60f9cec80975ab1693777e2"}, - {file = "google_re2-1.1-2-cp311-cp311-manylinux_2_24_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:f267499529e64a4abed24c588f355ebe4700189d434d84a7367725f5a186e48d"}, - {file = "google_re2-1.1-2-cp311-cp311-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:b632eff5e4cd44545a9c0e52f2e1becd55831e25f4dd4e0d7ec8ee6ca50858c1"}, - {file = "google_re2-1.1-2-cp311-cp311-win32.whl", hash = "sha256:a42c733036e8f242ee4e5f0e27153ad4ca44ced9e4ce82f3972938ddee528db0"}, - {file = "google_re2-1.1-2-cp311-cp311-win_amd64.whl", hash = "sha256:64f8eed4ca96905d99b5286b3d14b5ca4f6a025ff3c1351626a7df2f93ad1ddd"}, - {file = "google_re2-1.1-2-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:5541efcca5b5faf7e0d882334a04fa479bad4e7433f94870f46272eec0672c4a"}, - {file = "google_re2-1.1-2-cp38-cp38-macosx_11_0_x86_64.whl", hash = "sha256:92309af35b6eb2d3b3dc57045cdd83a76370958ab3e0edd2cc4638f6d23f5b32"}, - {file = "google_re2-1.1-2-cp38-cp38-macosx_12_0_arm64.whl", hash = "sha256:197cd9bcaba96d18c5bf84d0c32fca7a26c234ea83b1d3083366f4392cb99f78"}, - {file = "google_re2-1.1-2-cp38-cp38-macosx_12_0_x86_64.whl", hash = "sha256:1b896f171d29b541256cf26e10dccc9103ac1894683914ed88828ca6facf8dca"}, - {file = "google_re2-1.1-2-cp38-cp38-macosx_13_0_arm64.whl", hash = "sha256:e022d3239b945014e916ca7120fee659b246ec26c301f9e0542f1a19b38a8744"}, - {file = "google_re2-1.1-2-cp38-cp38-macosx_13_0_x86_64.whl", hash = "sha256:2c73f8a9440873b68bee1198094377501065e85aaf6fcc0d2512c7589ffa06ca"}, - {file = "google_re2-1.1-2-cp38-cp38-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:901d86555bd7725506d651afaba7d71cd4abd13260aed6cfd7c641a45f76d4f6"}, - {file = "google_re2-1.1-2-cp38-cp38-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:ce4710ff636701cfb56eb91c19b775d53b03749a23b7d2a5071bbbf4342a9067"}, - {file = "google_re2-1.1-2-cp38-cp38-manylinux_2_24_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:76a20e5ebdf5bc5d430530197e42a2eeb562f729d3a3fb51f39168283d676e66"}, - {file = "google_re2-1.1-2-cp38-cp38-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:77c9f4d4bb1c8de9d2642d3c4b8b615858ba764df025b3b4f1310266f8def269"}, - {file = "google_re2-1.1-2-cp38-cp38-win32.whl", hash = "sha256:94bd60785bf37ef130a1613738e3c39465a67eae3f3be44bb918540d39b68da3"}, - {file = "google_re2-1.1-2-cp38-cp38-win_amd64.whl", hash = "sha256:59efeb77c0dcdbe37794c61f29c5b1f34bc06e8ec309a111ccdd29d380644d70"}, - {file = "google_re2-1.1-2-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:221e38c27e1dd9ccb8e911e9c7aed6439f68ce81e7bb74001076830b0d6e931d"}, - {file = "google_re2-1.1-2-cp39-cp39-macosx_11_0_x86_64.whl", hash = "sha256:d9145879e6c2e1b814445300b31f88a675e1f06c57564670d95a1442e8370c27"}, - {file = "google_re2-1.1-2-cp39-cp39-macosx_12_0_arm64.whl", hash = "sha256:c8a12f0740e2a52826bdbf95569a4b0abdf413b4012fa71e94ad25dd4715c6e5"}, - {file = "google_re2-1.1-2-cp39-cp39-macosx_12_0_x86_64.whl", hash = "sha256:9c9998f71466f4db7bda752aa7c348b2881ff688e361108fe500caad1d8b9cb2"}, - {file = "google_re2-1.1-2-cp39-cp39-macosx_13_0_arm64.whl", hash = "sha256:0c39f69b702005963a3d3bf78743e1733ad73efd7e6e8465d76e3009e4694ceb"}, - {file = "google_re2-1.1-2-cp39-cp39-macosx_13_0_x86_64.whl", hash = "sha256:6d0ce762dee8d6617d0b1788a9653e805e83a23046c441d0ea65f1e27bf84114"}, - {file = "google_re2-1.1-2-cp39-cp39-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:ecf3619d98c9b4a7844ab52552ad32597cdbc9a5bdbc7e3435391c653600d1e2"}, - {file = "google_re2-1.1-2-cp39-cp39-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:9a1426a8cbd1fa004974574708d496005bd379310c4b1c7012be4bc75efde7a8"}, - {file = "google_re2-1.1-2-cp39-cp39-manylinux_2_24_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:a1a30626ba48b4070f3eab272d860ef1952e710b088792c4d68dddb155be6bfc"}, - {file = "google_re2-1.1-2-cp39-cp39-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:1b9c1ffcfbc3095b6ff601ec2d2bf662988f6ea6763bc1c9d52bec55881f8fde"}, - {file = "google_re2-1.1-2-cp39-cp39-win32.whl", hash = "sha256:32ecf995a252c0548404c1065ba4b36f1e524f1f4a86b6367a1a6c3da3801e30"}, - {file = "google_re2-1.1-2-cp39-cp39-win_amd64.whl", hash = "sha256:e7865410f3b112a3609739283ec3f4f6f25aae827ff59c6bfdf806fd394d753e"}, - {file = "google_re2-1.1-3-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:3b21f83f0a201009c56f06fcc7294a33555ede97130e8a91b3f4cae01aed1d73"}, - {file = "google_re2-1.1-3-cp310-cp310-macosx_11_0_x86_64.whl", hash = "sha256:b38194b91354a38db1f86f25d09cdc6ac85d63aee4c67b43da3048ce637adf45"}, - {file = "google_re2-1.1-3-cp310-cp310-macosx_12_0_arm64.whl", hash = "sha256:e7da3da8d6b5a18d6c3b61b11cc5b66b8564eaedce99d2312b15b6487730fc76"}, - {file = "google_re2-1.1-3-cp310-cp310-macosx_12_0_x86_64.whl", hash = "sha256:aeca656fb10d8638f245331aabab59c9e7e051ca974b366dd79e6a9efb12e401"}, - {file = "google_re2-1.1-3-cp310-cp310-macosx_13_0_arm64.whl", hash = "sha256:2069d6dc94f5fa14a159bf99cad2f11e9c0f8ec3b7f44a4dde9e59afe5d1c786"}, - {file = "google_re2-1.1-3-cp310-cp310-macosx_13_0_x86_64.whl", hash = "sha256:2319a39305a4931cb5251451f2582713418a19bef2af7adf9e2a7a0edd939b99"}, - {file = "google_re2-1.1-3-cp310-cp310-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:eb98fc131699756c6d86246f670a5e1c1cc1ba85413c425ad344cb30479b246c"}, - {file = "google_re2-1.1-3-cp310-cp310-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:a6e038986d8ffe4e269f8532f03009f229d1f6018d4ac0dabc8aff876338f6e0"}, - {file = "google_re2-1.1-3-cp310-cp310-manylinux_2_24_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:8618343ee658310e0f53bf586fab7409de43ce82bf8d9f7eb119536adc9783fd"}, - {file = "google_re2-1.1-3-cp310-cp310-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:d8140ca861cfe00602319cefe2c7b8737b379eb07fb328b51dc44584f47a2718"}, - {file = "google_re2-1.1-3-cp310-cp310-win32.whl", hash = "sha256:41f439c5c54e8a3a0a1fa2dbd1e809d3f643f862df7b16dd790f36a1238a272e"}, - {file = "google_re2-1.1-3-cp310-cp310-win_amd64.whl", hash = "sha256:fe20e97a33176d96d3e4b5b401de35182b9505823abea51425ec011f53ef5e56"}, - {file = "google_re2-1.1-3-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:7c39ff52b1765db039f690ee5b7b23919d8535aae94db7996079fbde0098c4d7"}, - {file = "google_re2-1.1-3-cp311-cp311-macosx_11_0_x86_64.whl", hash = "sha256:5420be674fd164041639ba4c825450f3d4bd635572acdde16b3dcd697f8aa3ef"}, - {file = "google_re2-1.1-3-cp311-cp311-macosx_12_0_arm64.whl", hash = "sha256:ff53881cf1ce040f102a42d39db93c3f835f522337ae9c79839a842f26d97733"}, - {file = "google_re2-1.1-3-cp311-cp311-macosx_12_0_x86_64.whl", hash = "sha256:8d04600b0b53523118df2e413a71417c408f20dee640bf07dfab601c96a18a77"}, - {file = "google_re2-1.1-3-cp311-cp311-macosx_13_0_arm64.whl", hash = "sha256:c4835d4849faa34a7fa1074098d81c420ed6c0707a3772482b02ce14f2a7c007"}, - {file = "google_re2-1.1-3-cp311-cp311-macosx_13_0_x86_64.whl", hash = "sha256:3309a9b81251d35fee15974d0ae0581a9a375266deeafdc3a3ac0d172a742357"}, - {file = "google_re2-1.1-3-cp311-cp311-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:e2b51cafee7e0bc72d0a4a454547bd8f257cde412ac9f1a2dc46a203b5e42cf4"}, - {file = "google_re2-1.1-3-cp311-cp311-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:83f5f1cb52f832c2297d271ee8c56cf5e9053448162e5d2223d513f729bad908"}, - {file = "google_re2-1.1-3-cp311-cp311-manylinux_2_24_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:55865a1ace92be3f7953b2e2b38b901d8074a367aa491daee43260a53a7fc6f0"}, - {file = "google_re2-1.1-3-cp311-cp311-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:cec2167dd142e583e98c783bd0d28b8cf5a9cdbe1f7407ba4163fe3ccb613cb9"}, - {file = "google_re2-1.1-3-cp311-cp311-win32.whl", hash = "sha256:a0bc1fe96849e4eb8b726d0bba493f5b989372243b32fe20729cace02e5a214d"}, - {file = "google_re2-1.1-3-cp311-cp311-win_amd64.whl", hash = "sha256:e6310a156db96fc5957cb007dd2feb18476898654530683897469447df73a7cd"}, - {file = "google_re2-1.1-3-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:8e63cd10ea006088b320e8c5d308da1f6c87aa95138a71c60dd7ca1c8e91927e"}, - {file = "google_re2-1.1-3-cp312-cp312-macosx_11_0_x86_64.whl", hash = "sha256:12b566830a334178733a85e416b1e0507dbc0ceb322827616fe51ef56c5154f1"}, - {file = "google_re2-1.1-3-cp312-cp312-macosx_12_0_arm64.whl", hash = "sha256:442e18c9d46b225c1496919c16eafe8f8d9bb4091b00b4d3440da03c55bbf4ed"}, - {file = "google_re2-1.1-3-cp312-cp312-macosx_12_0_x86_64.whl", hash = "sha256:c54c00263a9c39b2dacd93e9636319af51e3cf885c080b9680a9631708326460"}, - {file = "google_re2-1.1-3-cp312-cp312-macosx_13_0_arm64.whl", hash = "sha256:15a3caeeb327bc22e0c9f95eb76890fec8874cacccd2b01ff5c080ab4819bbec"}, - {file = "google_re2-1.1-3-cp312-cp312-macosx_13_0_x86_64.whl", hash = "sha256:59ec0d2cced77f715d41f6eafd901f6b15c11e28ba25fe0effdc1de554d78e75"}, - {file = "google_re2-1.1-3-cp312-cp312-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:185bf0e3441aed3840590f8e42f916e2920d235eb14df2cbc2049526803d3e71"}, - {file = "google_re2-1.1-3-cp312-cp312-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:586d3f2014eea5be14d8de53374d9b79fa99689160e00efa64b5fe93af326087"}, - {file = "google_re2-1.1-3-cp312-cp312-manylinux_2_24_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:cc2575082de4ffd234d9607f3ae67ca22b15a1a88793240e2045f3b3a36a5795"}, - {file = "google_re2-1.1-3-cp312-cp312-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:59c5ad438eddb3630def394456091284d7bbc5b89351987f94f3792d296d1f96"}, - {file = "google_re2-1.1-3-cp312-cp312-win32.whl", hash = "sha256:5b9878c53f2bf16f75bf71d4ddd57f6611351408d5821040e91c53ebdf82c373"}, - {file = "google_re2-1.1-3-cp312-cp312-win_amd64.whl", hash = "sha256:4fdecfeb213110d0a85bad335a8e7cdb59fea7de81a4fe659233f487171980f9"}, - {file = "google_re2-1.1-3-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:2dd87bacab32b709c28d0145fe75a956b6a39e28f0726d867375dba5721c76c1"}, - {file = "google_re2-1.1-3-cp38-cp38-macosx_11_0_x86_64.whl", hash = "sha256:55d24c61fe35dddc1bb484593a57c9f60f9e66d7f31f091ef9608ed0b6dde79f"}, - {file = "google_re2-1.1-3-cp38-cp38-macosx_12_0_arm64.whl", hash = "sha256:a0cf1180d908622df648c26b0cd09281f92129805ccc56a39227fdbfeab95cb4"}, - {file = "google_re2-1.1-3-cp38-cp38-macosx_12_0_x86_64.whl", hash = "sha256:09586f07f3f88d432265c75976da1c619ab7192cd7ebdf53f4ae0776c19e4b56"}, - {file = "google_re2-1.1-3-cp38-cp38-macosx_13_0_arm64.whl", hash = "sha256:539f1b053402203576e919a06749198da4ae415931ee28948a1898131ae932ce"}, - {file = "google_re2-1.1-3-cp38-cp38-macosx_13_0_x86_64.whl", hash = "sha256:abf0bcb5365b0e27a5a23f3da403dffdbbac2c0e3a3f1535a8b10cc121b5d5fb"}, - {file = "google_re2-1.1-3-cp38-cp38-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:19c83e5bbed7958213eeac3aa71c506525ce54faf03e07d0b96cd0a764890511"}, - {file = "google_re2-1.1-3-cp38-cp38-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:3348e77330ff672dc44ec01894fa5d93c409a532b6d688feac55e714e9059920"}, - {file = "google_re2-1.1-3-cp38-cp38-manylinux_2_24_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:06b63edb57c5ce5a13eabfd71155e346b9477dc8906dec7c580d4f70c16a7e0d"}, - {file = "google_re2-1.1-3-cp38-cp38-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:12fe57ba2914092b83338d61d8def9ebd5a2bd0fd8679eceb5d4c2748105d5c0"}, - {file = "google_re2-1.1-3-cp38-cp38-win32.whl", hash = "sha256:80796e08d24e606e675019fe8de4eb5c94bb765be13c384f2695247d54a6df75"}, - {file = "google_re2-1.1-3-cp38-cp38-win_amd64.whl", hash = "sha256:3c2257dedfe7cc5deb6791e563af9e071a9d414dad89e37ac7ad22f91be171a9"}, - {file = "google_re2-1.1-3-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:43a0cd77c87c894f28969ac622f94b2e6d1571261dfdd785026848a25cfdc9b9"}, - {file = "google_re2-1.1-3-cp39-cp39-macosx_11_0_x86_64.whl", hash = "sha256:1038990b77fd66f279bd66a0832b67435ea925e15bb59eafc7b60fdec812b616"}, - {file = "google_re2-1.1-3-cp39-cp39-macosx_12_0_arm64.whl", hash = "sha256:fb5dda6875d18dd45f0f24ebced6d1f7388867c8fb04a235d1deab7ea479ce38"}, - {file = "google_re2-1.1-3-cp39-cp39-macosx_12_0_x86_64.whl", hash = "sha256:bb1d164965c6d57a351b421d2f77c051403766a8b75aaa602324ee2451fff77f"}, - {file = "google_re2-1.1-3-cp39-cp39-macosx_13_0_arm64.whl", hash = "sha256:a072ebfa495051d07ffecbf6ce21eb84793568d5c3c678c00ed8ff6b8066ab31"}, - {file = "google_re2-1.1-3-cp39-cp39-macosx_13_0_x86_64.whl", hash = "sha256:4eb66c8398c8a510adc97978d944b3b29c91181237218841ea1a91dc39ec0e54"}, - {file = "google_re2-1.1-3-cp39-cp39-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:f7c8b57b1f559553248d1757b7fa5b2e0cc845666738d155dff1987c2618264e"}, - {file = "google_re2-1.1-3-cp39-cp39-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:9162f6aa4f25453c682eb176f21b8e2f40205be9f667e98a54b3e1ff10d6ee75"}, - {file = "google_re2-1.1-3-cp39-cp39-manylinux_2_24_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:a2d65ddf67fd7bf94705626871d463057d3d9a3538d41022f95b9d8f01df36e1"}, - {file = "google_re2-1.1-3-cp39-cp39-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:d140c7b9395b4d1e654127aa1c99bcc603ed01000b7bc7e28c52562f1894ec12"}, - {file = "google_re2-1.1-3-cp39-cp39-win32.whl", hash = "sha256:80c5fc200f64b2d903eeb07b8d6cefc620a872a0240c7caaa9aca05b20f5568f"}, - {file = "google_re2-1.1-3-cp39-cp39-win_amd64.whl", hash = "sha256:9eb6dbcee9b5dc4069bbc0634f2eb039ca524a14bed5868fdf6560aaafcbca06"}, - {file = "google_re2-1.1-4-cp310-cp310-macosx_12_0_arm64.whl", hash = "sha256:0db114d7e1aa96dbcea452a40136d7d747d60cbb61394965774688ef59cccd4e"}, - {file = "google_re2-1.1-4-cp310-cp310-macosx_12_0_x86_64.whl", hash = "sha256:82133958e003a1344e5b7a791b9a9dd7560b5c8f96936dbe16f294604524a633"}, - {file = "google_re2-1.1-4-cp310-cp310-macosx_13_0_arm64.whl", hash = "sha256:9e74fd441d1f3d917d3303e319f61b82cdbd96b9a5ba919377a6eef1504a1e2b"}, - {file = "google_re2-1.1-4-cp310-cp310-macosx_13_0_x86_64.whl", hash = "sha256:734a2e7a4541c57253b5ebee24f3f3366ba3658bcad01da25fb623c78723471a"}, - {file = "google_re2-1.1-4-cp310-cp310-macosx_14_0_arm64.whl", hash = "sha256:d88d5eecbc908abe16132456fae13690d0508f3ac5777f320ef95cb6cab9a961"}, - {file = "google_re2-1.1-4-cp310-cp310-macosx_14_0_x86_64.whl", hash = "sha256:b91db80b171ecec435a07977a227757dd487356701a32f556fa6fca5d0a40522"}, - {file = "google_re2-1.1-4-cp310-cp310-manylinux_2_24_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:6b23129887a64bb9948af14c84705273ed1a40054e99433b4acccab4dcf6a226"}, - {file = "google_re2-1.1-4-cp310-cp310-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:5dc1a0cc7cd19261dcaf76763e2499305dbb7e51dc69555167cdb8af98782698"}, - {file = "google_re2-1.1-4-cp310-cp310-win32.whl", hash = "sha256:3b2ab1e2420b5dd9743a2d6bc61b64e5f708563702a75b6db86637837eaeaf2f"}, - {file = "google_re2-1.1-4-cp310-cp310-win_amd64.whl", hash = "sha256:92efca1a7ef83b6df012d432a1cbc71d10ff42200640c0f9a5ff5b343a48e633"}, - {file = "google_re2-1.1-4-cp311-cp311-macosx_12_0_arm64.whl", hash = "sha256:854818fd4ce79787aca5ba459d6e5abe4ca9be2c684a5b06a7f1757452ca3708"}, - {file = "google_re2-1.1-4-cp311-cp311-macosx_12_0_x86_64.whl", hash = "sha256:4ceef51174b6f653b6659a8fdaa9c38960c5228b44b25be2a3bcd8566827554f"}, - {file = "google_re2-1.1-4-cp311-cp311-macosx_13_0_arm64.whl", hash = "sha256:ee49087c3db7e6f5238105ab5299c09e9b77516fe8cfb0a37e5f1e813d76ecb8"}, - {file = "google_re2-1.1-4-cp311-cp311-macosx_13_0_x86_64.whl", hash = "sha256:dc2312854bdc01410acc5d935f1906a49cb1f28980341c20a68797ad89d8e178"}, - {file = "google_re2-1.1-4-cp311-cp311-macosx_14_0_arm64.whl", hash = "sha256:0dc0d2e42296fa84a3cb3e1bd667c6969389cd5cdf0786e6b1f911ae2d75375b"}, - {file = "google_re2-1.1-4-cp311-cp311-macosx_14_0_x86_64.whl", hash = "sha256:6bf04ced98453b035f84320f348f67578024f44d2997498def149054eb860ae8"}, - {file = "google_re2-1.1-4-cp311-cp311-manylinux_2_24_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:1d6b6ef11dc4ab322fa66c2f3561925f2b5372a879c3ed764d20e939e2fd3e5f"}, - {file = "google_re2-1.1-4-cp311-cp311-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:0dcde6646fa9a97fd3692b3f6ae7daf7f3277d7500b6c253badeefa11db8956a"}, - {file = "google_re2-1.1-4-cp311-cp311-win32.whl", hash = "sha256:5f4f0229deb057348893574d5b0a96d055abebac6debf29d95b0c0e26524c9f6"}, - {file = "google_re2-1.1-4-cp311-cp311-win_amd64.whl", hash = "sha256:4713ddbe48a18875270b36a462b0eada5e84d6826f8df7edd328d8706b6f9d07"}, - {file = "google_re2-1.1-4-cp312-cp312-macosx_12_0_arm64.whl", hash = "sha256:40a698300b8faddbb325662973f839489c89b960087060bd389c376828978a04"}, - {file = "google_re2-1.1-4-cp312-cp312-macosx_12_0_x86_64.whl", hash = "sha256:103d2d7ac92ba23911a151fd1fc7035cbf6dc92a7f6aea92270ebceb5cd5acd3"}, - {file = "google_re2-1.1-4-cp312-cp312-macosx_13_0_arm64.whl", hash = "sha256:51fb7182bccab05e8258a2b6a63dda1a6b4a9e8dfb9b03ec50e50c49c2827dd4"}, - {file = "google_re2-1.1-4-cp312-cp312-macosx_13_0_x86_64.whl", hash = "sha256:65383022abd63d7b620221eba7935132b53244b8b463d8fdce498c93cf58b7b7"}, - {file = "google_re2-1.1-4-cp312-cp312-macosx_14_0_arm64.whl", hash = "sha256:396281fc68a9337157b3ffcd9392c6b7fcb8aab43e5bdab496262a81d56a4ecc"}, - {file = "google_re2-1.1-4-cp312-cp312-macosx_14_0_x86_64.whl", hash = "sha256:8198adcfcff1c680e052044124621730fc48d08005f90a75487f5651f1ebfce2"}, - {file = "google_re2-1.1-4-cp312-cp312-manylinux_2_24_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:81f7bff07c448aec4db9ca453d2126ece8710dbd9278b8bb09642045d3402a96"}, - {file = "google_re2-1.1-4-cp312-cp312-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:b7dacf730fd7d6ec71b11d6404b0b26e230814bfc8e9bb0d3f13bec9b5531f8d"}, - {file = "google_re2-1.1-4-cp312-cp312-win32.whl", hash = "sha256:8c764f62f4b1d89d1ef264853b6dd9fee14a89e9b86a81bc2157fe3531425eb4"}, - {file = "google_re2-1.1-4-cp312-cp312-win_amd64.whl", hash = "sha256:0be2666df4bc5381a5d693585f9bbfefb0bfd3c07530d7e403f181f5de47254a"}, - {file = "google_re2-1.1-4-cp38-cp38-macosx_12_0_arm64.whl", hash = "sha256:5cb1b63a0bfd8dd65d39d2f3b2e5ae0a06ce4b2ce5818a1d1fc78a786a252673"}, - {file = "google_re2-1.1-4-cp38-cp38-macosx_12_0_x86_64.whl", hash = "sha256:e41751ce6b67a95230edd0772226dc94c2952a2909674cd69df9804ed0125307"}, - {file = "google_re2-1.1-4-cp38-cp38-macosx_13_0_arm64.whl", hash = "sha256:b998cfa2d50bf4c063e777c999a7e8645ec7e5d7baf43ad71b1e2e10bb0300c3"}, - {file = "google_re2-1.1-4-cp38-cp38-macosx_13_0_x86_64.whl", hash = "sha256:226ca3b0c2e970f3fc82001ac89e845ecc7a4bb7c68583e7a76cda70b61251a7"}, - {file = "google_re2-1.1-4-cp38-cp38-macosx_14_0_arm64.whl", hash = "sha256:9adec1f734ebad7c72e56c85f205a281d8fe9bf6583bc21020157d3f2812ce89"}, - {file = "google_re2-1.1-4-cp38-cp38-macosx_14_0_x86_64.whl", hash = "sha256:9c34f3c64ba566af967d29e11299560e6fdfacd8ca695120a7062b6ed993b179"}, - {file = "google_re2-1.1-4-cp38-cp38-manylinux_2_24_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:e1b85385fe293838e0d0b6e19e6c48ba8c6f739ea92ce2e23b718afe7b343363"}, - {file = "google_re2-1.1-4-cp38-cp38-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:4694daa8a8987cfb568847aa872f9990e930c91a68c892ead876411d4b9012c3"}, - {file = "google_re2-1.1-4-cp38-cp38-win32.whl", hash = "sha256:5e671e9be1668187e2995aac378de574fa40df70bb6f04657af4d30a79274ce0"}, - {file = "google_re2-1.1-4-cp38-cp38-win_amd64.whl", hash = "sha256:f66c164d6049a8299f6dfcfa52d1580576b4b9724d6fcdad2f36f8f5da9304b6"}, - {file = "google_re2-1.1-4-cp39-cp39-macosx_12_0_arm64.whl", hash = "sha256:25cb17ae0993a48c70596f3a3ef5d659638106401cc8193f51c0d7961b3b3eb7"}, - {file = "google_re2-1.1-4-cp39-cp39-macosx_12_0_x86_64.whl", hash = "sha256:5f101f86d14ca94ca4dcf63cceaa73d351f2be2481fcaa29d9e68eeab0dc2a88"}, - {file = "google_re2-1.1-4-cp39-cp39-macosx_13_0_arm64.whl", hash = "sha256:4e82591e85bf262a6d74cff152867e05fc97867c68ba81d6836ff8b0e7e62365"}, - {file = "google_re2-1.1-4-cp39-cp39-macosx_13_0_x86_64.whl", hash = "sha256:1f61c09b93ffd34b1e2557e5a9565039f935407a5786dbad46f64f1a484166e6"}, - {file = "google_re2-1.1-4-cp39-cp39-macosx_14_0_arm64.whl", hash = "sha256:12b390ad8c7e74bab068732f774e75e0680dade6469b249a721f3432f90edfc3"}, - {file = "google_re2-1.1-4-cp39-cp39-macosx_14_0_x86_64.whl", hash = "sha256:1284343eb31c2e82ed2d8159f33ba6842238a56782c881b07845a6d85613b055"}, - {file = "google_re2-1.1-4-cp39-cp39-manylinux_2_24_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:6c7b38e0daf2c06e4d3163f4c732ab3ad2521aecfed6605b69e4482c612da303"}, - {file = "google_re2-1.1-4-cp39-cp39-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:1f4d4f0823e8b2f6952a145295b1ff25245ce9bb136aff6fe86452e507d4c1dd"}, - {file = "google_re2-1.1-4-cp39-cp39-win32.whl", hash = "sha256:1afae56b2a07bb48cfcfefaa15ed85bae26a68f5dc7f9e128e6e6ea36914e847"}, - {file = "google_re2-1.1-4-cp39-cp39-win_amd64.whl", hash = "sha256:aa7d6d05911ab9c8adbf3c225a7a120ab50fd2784ac48f2f0d140c0b7afc2b55"}, ] [[package]] @@ -4381,13 +4223,10 @@ files = [ {file = "lxml-4.9.3-cp27-cp27m-macosx_11_0_x86_64.whl", hash = "sha256:b0a545b46b526d418eb91754565ba5b63b1c0b12f9bd2f808c852d9b4b2f9b5c"}, {file = "lxml-4.9.3-cp27-cp27m-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:075b731ddd9e7f68ad24c635374211376aa05a281673ede86cbe1d1b3455279d"}, {file = "lxml-4.9.3-cp27-cp27m-manylinux_2_5_x86_64.manylinux1_x86_64.whl", hash = "sha256:1e224d5755dba2f4a9498e150c43792392ac9b5380aa1b845f98a1618c94eeef"}, - {file = "lxml-4.9.3-cp27-cp27m-win32.whl", hash = "sha256:2c74524e179f2ad6d2a4f7caf70e2d96639c0954c943ad601a9e146c76408ed7"}, - {file = "lxml-4.9.3-cp27-cp27m-win_amd64.whl", hash = "sha256:4f1026bc732b6a7f96369f7bfe1a4f2290fb34dce00d8644bc3036fb351a4ca1"}, {file = "lxml-4.9.3-cp27-cp27mu-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:c0781a98ff5e6586926293e59480b64ddd46282953203c76ae15dbbbf302e8bb"}, {file = "lxml-4.9.3-cp27-cp27mu-manylinux_2_5_x86_64.manylinux1_x86_64.whl", hash = "sha256:cef2502e7e8a96fe5ad686d60b49e1ab03e438bd9123987994528febd569868e"}, {file = "lxml-4.9.3-cp310-cp310-macosx_11_0_x86_64.whl", hash = "sha256:b86164d2cff4d3aaa1f04a14685cbc072efd0b4f99ca5708b2ad1b9b5988a991"}, {file = "lxml-4.9.3-cp310-cp310-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_24_i686.whl", hash = "sha256:42871176e7896d5d45138f6d28751053c711ed4d48d8e30b498da155af39aebd"}, - {file = "lxml-4.9.3-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_24_x86_64.whl", hash = "sha256:ae8b9c6deb1e634ba4f1930eb67ef6e6bf6a44b6eb5ad605642b2d6d5ed9ce3c"}, {file = "lxml-4.9.3-cp310-cp310-manylinux_2_28_aarch64.whl", hash = "sha256:411007c0d88188d9f621b11d252cce90c4a2d1a49db6c068e3c16422f306eab8"}, {file = "lxml-4.9.3-cp310-cp310-manylinux_2_28_x86_64.whl", hash = "sha256:cd47b4a0d41d2afa3e58e5bf1f62069255aa2fd6ff5ee41604418ca925911d76"}, {file = "lxml-4.9.3-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:0e2cb47860da1f7e9a5256254b74ae331687b9672dfa780eed355c4c9c3dbd23"}, @@ -4396,7 +4235,6 @@ files = [ {file = "lxml-4.9.3-cp310-cp310-win_amd64.whl", hash = "sha256:97047f0d25cd4bcae81f9ec9dc290ca3e15927c192df17331b53bebe0e3ff96d"}, {file = "lxml-4.9.3-cp311-cp311-macosx_11_0_universal2.whl", hash = "sha256:1f447ea5429b54f9582d4b955f5f1985f278ce5cf169f72eea8afd9502973dd5"}, {file = "lxml-4.9.3-cp311-cp311-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_24_i686.whl", hash = "sha256:57d6ba0ca2b0c462f339640d22882acc711de224d769edf29962b09f77129cbf"}, - {file = "lxml-4.9.3-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_24_x86_64.whl", hash = "sha256:9767e79108424fb6c3edf8f81e6730666a50feb01a328f4a016464a5893f835a"}, {file = "lxml-4.9.3-cp311-cp311-manylinux_2_28_aarch64.whl", hash = "sha256:71c52db65e4b56b8ddc5bb89fb2e66c558ed9d1a74a45ceb7dcb20c191c3df2f"}, {file = "lxml-4.9.3-cp311-cp311-manylinux_2_28_x86_64.whl", hash = "sha256:d73d8ecf8ecf10a3bd007f2192725a34bd62898e8da27eb9d32a58084f93962b"}, {file = "lxml-4.9.3-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:0a3d3487f07c1d7f150894c238299934a2a074ef590b583103a45002035be120"}, @@ -4416,7 +4254,6 @@ files = [ {file = "lxml-4.9.3-cp36-cp36m-macosx_11_0_x86_64.whl", hash = "sha256:64f479d719dc9f4c813ad9bb6b28f8390360660b73b2e4beb4cb0ae7104f1c12"}, {file = "lxml-4.9.3-cp36-cp36m-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_24_i686.whl", hash = "sha256:dd708cf4ee4408cf46a48b108fb9427bfa00b9b85812a9262b5c668af2533ea5"}, {file = "lxml-4.9.3-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5c31c7462abdf8f2ac0577d9f05279727e698f97ecbb02f17939ea99ae8daa98"}, - {file = "lxml-4.9.3-cp36-cp36m-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_24_x86_64.whl", hash = "sha256:e3cd95e10c2610c360154afdc2f1480aea394f4a4f1ea0a5eacce49640c9b190"}, {file = "lxml-4.9.3-cp36-cp36m-manylinux_2_28_x86_64.whl", hash = "sha256:4930be26af26ac545c3dffb662521d4e6268352866956672231887d18f0eaab2"}, {file = "lxml-4.9.3-cp36-cp36m-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:4aec80cde9197340bc353d2768e2a75f5f60bacda2bab72ab1dc499589b3878c"}, {file = "lxml-4.9.3-cp36-cp36m-manylinux_2_5_x86_64.manylinux1_x86_64.whl", hash = "sha256:14e019fd83b831b2e61baed40cab76222139926b1fb5ed0e79225bc0cae14584"}, @@ -4426,7 +4263,6 @@ files = [ {file = "lxml-4.9.3-cp36-cp36m-win_amd64.whl", hash = "sha256:bef4e656f7d98aaa3486d2627e7d2df1157d7e88e7efd43a65aa5dd4714916cf"}, {file = "lxml-4.9.3-cp37-cp37m-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_24_i686.whl", hash = "sha256:46f409a2d60f634fe550f7133ed30ad5321ae2e6630f13657fb9479506b00601"}, {file = "lxml-4.9.3-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.manylinux_2_24_aarch64.whl", hash = "sha256:4c28a9144688aef80d6ea666c809b4b0e50010a2aca784c97f5e6bf143d9f129"}, - {file = "lxml-4.9.3-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_24_x86_64.whl", hash = "sha256:141f1d1a9b663c679dc524af3ea1773e618907e96075262726c7612c02b149a4"}, {file = "lxml-4.9.3-cp37-cp37m-manylinux_2_28_x86_64.whl", hash = "sha256:53ace1c1fd5a74ef662f844a0413446c0629d151055340e9893da958a374f70d"}, {file = "lxml-4.9.3-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:17a753023436a18e27dd7769e798ce302963c236bc4114ceee5b25c18c52c693"}, {file = "lxml-4.9.3-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.whl", hash = "sha256:7d298a1bd60c067ea75d9f684f5f3992c9d6766fadbc0bcedd39750bf344c2f4"}, @@ -4436,7 +4272,6 @@ files = [ {file = "lxml-4.9.3-cp37-cp37m-win_amd64.whl", hash = "sha256:120fa9349a24c7043854c53cae8cec227e1f79195a7493e09e0c12e29f918e52"}, {file = "lxml-4.9.3-cp38-cp38-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_24_i686.whl", hash = "sha256:4d2d1edbca80b510443f51afd8496be95529db04a509bc8faee49c7b0fb6d2cc"}, {file = "lxml-4.9.3-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.manylinux_2_24_aarch64.whl", hash = "sha256:8d7e43bd40f65f7d97ad8ef5c9b1778943d02f04febef12def25f7583d19baac"}, - {file = "lxml-4.9.3-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_24_x86_64.whl", hash = "sha256:71d66ee82e7417828af6ecd7db817913cb0cf9d4e61aa0ac1fde0583d84358db"}, {file = "lxml-4.9.3-cp38-cp38-manylinux_2_28_x86_64.whl", hash = "sha256:6fc3c450eaa0b56f815c7b62f2b7fba7266c4779adcf1cece9e6deb1de7305ce"}, {file = "lxml-4.9.3-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:65299ea57d82fb91c7f019300d24050c4ddeb7c5a190e076b5f48a2b43d19c42"}, {file = "lxml-4.9.3-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.whl", hash = "sha256:eadfbbbfb41b44034a4c757fd5d70baccd43296fb894dba0295606a7cf3124aa"}, @@ -4446,7 +4281,6 @@ files = [ {file = "lxml-4.9.3-cp38-cp38-win_amd64.whl", hash = "sha256:92af161ecbdb2883c4593d5ed4815ea71b31fafd7fd05789b23100d081ecac96"}, {file = "lxml-4.9.3-cp39-cp39-macosx_11_0_x86_64.whl", hash = "sha256:9bb6ad405121241e99a86efff22d3ef469024ce22875a7ae045896ad23ba2340"}, {file = "lxml-4.9.3-cp39-cp39-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_24_i686.whl", hash = "sha256:8ed74706b26ad100433da4b9d807eae371efaa266ffc3e9191ea436087a9d6a7"}, - {file = "lxml-4.9.3-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_24_x86_64.whl", hash = "sha256:fbf521479bcac1e25a663df882c46a641a9bff6b56dc8b0fafaebd2f66fb231b"}, {file = "lxml-4.9.3-cp39-cp39-manylinux_2_28_aarch64.whl", hash = "sha256:303bf1edce6ced16bf67a18a1cf8339d0db79577eec5d9a6d4a80f0fb10aa2da"}, {file = "lxml-4.9.3-cp39-cp39-manylinux_2_28_x86_64.whl", hash = "sha256:5515edd2a6d1a5a70bfcdee23b42ec33425e405c5b351478ab7dc9347228f96e"}, {file = "lxml-4.9.3-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:690dafd0b187ed38583a648076865d8c229661ed20e48f2335d68e2cf7dc829d"}, @@ -4457,16 +4291,13 @@ files = [ {file = "lxml-4.9.3-cp39-cp39-win_amd64.whl", hash = "sha256:4dd9a263e845a72eacb60d12401e37c616438ea2e5442885f65082c276dfb2b2"}, {file = "lxml-4.9.3-pp310-pypy310_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:6689a3d7fd13dc687e9102a27e98ef33730ac4fe37795d5036d18b4d527abd35"}, {file = "lxml-4.9.3-pp37-pypy37_pp73-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_24_i686.whl", hash = "sha256:f6bdac493b949141b733c5345b6ba8f87a226029cbabc7e9e121a413e49441e0"}, - {file = "lxml-4.9.3-pp37-pypy37_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_24_x86_64.whl", hash = "sha256:05186a0f1346ae12553d66df1cfce6f251589fea3ad3da4f3ef4e34b2d58c6a3"}, {file = "lxml-4.9.3-pp37-pypy37_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:c2006f5c8d28dee289f7020f721354362fa304acbaaf9745751ac4006650254b"}, {file = "lxml-4.9.3-pp38-pypy38_pp73-macosx_11_0_x86_64.whl", hash = "sha256:5c245b783db29c4e4fbbbfc9c5a78be496c9fea25517f90606aa1f6b2b3d5f7b"}, {file = "lxml-4.9.3-pp38-pypy38_pp73-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_24_i686.whl", hash = "sha256:4fb960a632a49f2f089d522f70496640fdf1218f1243889da3822e0a9f5f3ba7"}, - {file = "lxml-4.9.3-pp38-pypy38_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_24_x86_64.whl", hash = "sha256:50670615eaf97227d5dc60de2dc99fb134a7130d310d783314e7724bf163f75d"}, {file = "lxml-4.9.3-pp38-pypy38_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:9719fe17307a9e814580af1f5c6e05ca593b12fb7e44fe62450a5384dbf61b4b"}, {file = "lxml-4.9.3-pp38-pypy38_pp73-win_amd64.whl", hash = "sha256:3331bece23c9ee066e0fb3f96c61322b9e0f54d775fccefff4c38ca488de283a"}, {file = "lxml-4.9.3-pp39-pypy39_pp73-macosx_11_0_x86_64.whl", hash = "sha256:ed667f49b11360951e201453fc3967344d0d0263aa415e1619e85ae7fd17b4e0"}, {file = "lxml-4.9.3-pp39-pypy39_pp73-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_24_i686.whl", hash = "sha256:8b77946fd508cbf0fccd8e400a7f71d4ac0e1595812e66025bac475a8e811694"}, - {file = "lxml-4.9.3-pp39-pypy39_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.manylinux_2_24_x86_64.whl", hash = "sha256:e4da8ca0c0c0aea88fd46be8e44bd49716772358d648cce45fe387f7b92374a7"}, {file = "lxml-4.9.3-pp39-pypy39_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:fe4bda6bd4340caa6e5cf95e73f8fea5c4bfc55763dd42f1b50a94c1b4a2fbd4"}, {file = "lxml-4.9.3-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:f3df3db1d336b9356dd3112eae5f5c2b8b377f3bc826848567f10bfddfee77e9"}, {file = "lxml-4.9.3.tar.gz", hash = "sha256:48628bd53a426c9eb9bc066a923acaa0878d1e86129fd5359aee99285f4eed9c"}, @@ -6603,7 +6434,6 @@ files = [ {file = "pymongo-4.6.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:8ab6bcc8e424e07c1d4ba6df96f7fb963bcb48f590b9456de9ebd03b88084fe8"}, {file = "pymongo-4.6.0-cp312-cp312-win32.whl", hash = "sha256:47aa128be2e66abd9d1a9b0437c62499d812d291f17b55185cb4aa33a5f710a4"}, {file = "pymongo-4.6.0-cp312-cp312-win_amd64.whl", hash = "sha256:014e7049dd019a6663747ca7dae328943e14f7261f7c1381045dfc26a04fa330"}, - {file = "pymongo-4.6.0-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:e24025625bad66895b1bc3ae1647f48f0a92dd014108fb1be404c77f0b69ca67"}, {file = "pymongo-4.6.0-cp37-cp37m-manylinux1_i686.whl", hash = "sha256:288c21ab9531b037f7efa4e467b33176bc73a0c27223c141b822ab4a0e66ff2a"}, {file = "pymongo-4.6.0-cp37-cp37m-manylinux1_x86_64.whl", hash = "sha256:747c84f4e690fbe6999c90ac97246c95d31460d890510e4a3fa61b7d2b87aa34"}, {file = "pymongo-4.6.0-cp37-cp37m-manylinux2014_aarch64.whl", hash = "sha256:055f5c266e2767a88bb585d01137d9c7f778b0195d3dbf4a487ef0638be9b651"}, @@ -7498,28 +7328,28 @@ pyasn1 = ">=0.1.3" [[package]] name = "ruff" -version = "0.3.2" +version = "0.3.4" description = "An extremely fast Python linter and code formatter, written in Rust." optional = false python-versions = ">=3.7" files = [ - {file = "ruff-0.3.2-py3-none-macosx_10_12_x86_64.macosx_11_0_arm64.macosx_10_12_universal2.whl", hash = "sha256:77f2612752e25f730da7421ca5e3147b213dca4f9a0f7e0b534e9562c5441f01"}, - {file = "ruff-0.3.2-py3-none-macosx_10_12_x86_64.whl", hash = "sha256:9966b964b2dd1107797be9ca7195002b874424d1d5472097701ae8f43eadef5d"}, - {file = "ruff-0.3.2-py3-none-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b83d17ff166aa0659d1e1deaf9f2f14cbe387293a906de09bc4860717eb2e2da"}, - {file = "ruff-0.3.2-py3-none-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:bb875c6cc87b3703aeda85f01c9aebdce3d217aeaca3c2e52e38077383f7268a"}, - {file = "ruff-0.3.2-py3-none-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:be75e468a6a86426430373d81c041b7605137a28f7014a72d2fc749e47f572aa"}, - {file = "ruff-0.3.2-py3-none-manylinux_2_17_ppc64.manylinux2014_ppc64.whl", hash = "sha256:967978ac2d4506255e2f52afe70dda023fc602b283e97685c8447d036863a302"}, - {file = "ruff-0.3.2-py3-none-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:1231eacd4510f73222940727ac927bc5d07667a86b0cbe822024dd00343e77e9"}, - {file = "ruff-0.3.2-py3-none-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:2c6d613b19e9a8021be2ee1d0e27710208d1603b56f47203d0abbde906929a9b"}, - {file = "ruff-0.3.2-py3-none-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c8439338a6303585d27b66b4626cbde89bb3e50fa3cae86ce52c1db7449330a7"}, - {file = "ruff-0.3.2-py3-none-musllinux_1_2_aarch64.whl", hash = "sha256:de8b480d8379620cbb5ea466a9e53bb467d2fb07c7eca54a4aa8576483c35d36"}, - {file = "ruff-0.3.2-py3-none-musllinux_1_2_armv7l.whl", hash = "sha256:b74c3de9103bd35df2bb05d8b2899bf2dbe4efda6474ea9681280648ec4d237d"}, - {file = "ruff-0.3.2-py3-none-musllinux_1_2_i686.whl", hash = "sha256:f380be9fc15a99765c9cf316b40b9da1f6ad2ab9639e551703e581a5e6da6745"}, - {file = "ruff-0.3.2-py3-none-musllinux_1_2_x86_64.whl", hash = "sha256:0ac06a3759c3ab9ef86bbeca665d31ad3aa9a4b1c17684aadb7e61c10baa0df4"}, - {file = "ruff-0.3.2-py3-none-win32.whl", hash = "sha256:9bd640a8f7dd07a0b6901fcebccedadeb1a705a50350fb86b4003b805c81385a"}, - {file = "ruff-0.3.2-py3-none-win_amd64.whl", hash = "sha256:0c1bdd9920cab5707c26c8b3bf33a064a4ca7842d91a99ec0634fec68f9f4037"}, - {file = "ruff-0.3.2-py3-none-win_arm64.whl", hash = "sha256:5f65103b1d76e0d600cabd577b04179ff592064eaa451a70a81085930e907d0b"}, - {file = "ruff-0.3.2.tar.gz", hash = "sha256:fa78ec9418eb1ca3db392811df3376b46471ae93792a81af2d1cbb0e5dcb5142"}, + {file = "ruff-0.3.4-py3-none-macosx_10_12_x86_64.macosx_11_0_arm64.macosx_10_12_universal2.whl", hash = "sha256:60c870a7d46efcbc8385d27ec07fe534ac32f3b251e4fc44b3cbfd9e09609ef4"}, + {file = "ruff-0.3.4-py3-none-macosx_10_12_x86_64.whl", hash = "sha256:6fc14fa742e1d8f24910e1fff0bd5e26d395b0e0e04cc1b15c7c5e5fe5b4af91"}, + {file = "ruff-0.3.4-py3-none-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d3ee7880f653cc03749a3bfea720cf2a192e4f884925b0cf7eecce82f0ce5854"}, + {file = "ruff-0.3.4-py3-none-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:cf133dd744f2470b347f602452a88e70dadfbe0fcfb5fd46e093d55da65f82f7"}, + {file = "ruff-0.3.4-py3-none-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3f3860057590e810c7ffea75669bdc6927bfd91e29b4baa9258fd48b540a4365"}, + {file = "ruff-0.3.4-py3-none-manylinux_2_17_ppc64.manylinux2014_ppc64.whl", hash = "sha256:986f2377f7cf12efac1f515fc1a5b753c000ed1e0a6de96747cdf2da20a1b369"}, + {file = "ruff-0.3.4-py3-none-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:c4fd98e85869603e65f554fdc5cddf0712e352fe6e61d29d5a6fe087ec82b76c"}, + {file = "ruff-0.3.4-py3-none-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:64abeed785dad51801b423fa51840b1764b35d6c461ea8caef9cf9e5e5ab34d9"}, + {file = "ruff-0.3.4-py3-none-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:df52972138318bc7546d92348a1ee58449bc3f9eaf0db278906eb511889c4b50"}, + {file = "ruff-0.3.4-py3-none-musllinux_1_2_aarch64.whl", hash = "sha256:98e98300056445ba2cc27d0b325fd044dc17fcc38e4e4d2c7711585bd0a958ed"}, + {file = "ruff-0.3.4-py3-none-musllinux_1_2_armv7l.whl", hash = "sha256:519cf6a0ebed244dce1dc8aecd3dc99add7a2ee15bb68cf19588bb5bf58e0488"}, + {file = "ruff-0.3.4-py3-none-musllinux_1_2_i686.whl", hash = "sha256:bb0acfb921030d00070539c038cd24bb1df73a2981e9f55942514af8b17be94e"}, + {file = "ruff-0.3.4-py3-none-musllinux_1_2_x86_64.whl", hash = "sha256:cf187a7e7098233d0d0c71175375c5162f880126c4c716fa28a8ac418dcf3378"}, + {file = "ruff-0.3.4-py3-none-win32.whl", hash = "sha256:af27ac187c0a331e8ef91d84bf1c3c6a5dea97e912a7560ac0cef25c526a4102"}, + {file = "ruff-0.3.4-py3-none-win_amd64.whl", hash = "sha256:de0d5069b165e5a32b3c6ffbb81c350b1e3d3483347196ffdf86dc0ef9e37dd6"}, + {file = "ruff-0.3.4-py3-none-win_arm64.whl", hash = "sha256:6810563cc08ad0096b57c717bd78aeac888a1bfd38654d9113cb3dc4d3f74232"}, + {file = "ruff-0.3.4.tar.gz", hash = "sha256:f0f4484c6541a99862b693e13a151435a279b271cff20e37101116a21e2a1ad1"}, ] [[package]] @@ -8454,6 +8284,17 @@ files = [ {file = "types_python_dateutil-2.8.19.14-py3-none-any.whl", hash = "sha256:f977b8de27787639986b4e28963263fd0e5158942b3ecef91b9335c130cb1ce9"}, ] +[[package]] +name = "types-pytz" +version = "2024.1.0.20240203" +description = "Typing stubs for pytz" +optional = false +python-versions = ">=3.8" +files = [ + {file = "types-pytz-2024.1.0.20240203.tar.gz", hash = "sha256:c93751ee20dfc6e054a0148f8f5227b9a00b79c90a4d3c9f464711a73179c89e"}, + {file = "types_pytz-2024.1.0.20240203-py3-none-any.whl", hash = "sha256:9679eef0365db3af91ef7722c199dbb75ee5c1b67e3c4dd7bfbeb1b8a71c21a3"}, +] + [[package]] name = "types-pyyaml" version = "6.0.12.11" @@ -8994,6 +8835,7 @@ bigquery = ["gcsfs", "google-cloud-bigquery", "grpcio", "pyarrow"] cli = ["cron-descriptor", "pipdeptree"] databricks = ["databricks-sql-connector"] dbt = ["dbt-athena-community", "dbt-bigquery", "dbt-core", "dbt-databricks", "dbt-duckdb", "dbt-redshift", "dbt-snowflake"] +dremio = ["pyarrow"] duckdb = ["duckdb", "duckdb"] filesystem = ["botocore", "s3fs"] gcp = ["gcsfs", "google-cloud-bigquery", "grpcio"] @@ -9012,4 +8854,4 @@ weaviate = ["weaviate-client"] [metadata] lock-version = "2.0" python-versions = ">=3.8.1,<3.13" -content-hash = "e6e43e82afedfa274c91f3fd13dbbddd9cac64f386d2f5f1c4564ff6f5784cd2" +content-hash = "1a0cf7f14a3c3846981cb5fdddd2a17b8f9ba2024ad6f6f8f3413dc40497122e" diff --git a/pyproject.toml b/pyproject.toml index 861fbcd5c3..cffb47ef45 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -103,6 +103,7 @@ mssql = ["pyodbc"] synapse = ["pyodbc", "adlfs", "pyarrow"] qdrant = ["qdrant-client"] databricks = ["databricks-sql-connector"] +dremio = ["pyarrow"] [tool.poetry.scripts] dlt = "dlt.cli._dlt:_main" @@ -144,6 +145,7 @@ cryptography = "^41.0.7" google-api-python-client = ">=1.7.11" pytest-asyncio = "^0.23.5" types-sqlalchemy = "^1.4.53.38" +types-pytz = ">=2024.1.0.20240203" ruff = "^0.3.2" pyjwt = "^2.8.0" diff --git a/tests/cases.py b/tests/cases.py index 8885df0c1b..c6ee363519 100644 --- a/tests/cases.py +++ b/tests/cases.py @@ -226,6 +226,11 @@ def assert_all_data_types_row( ensure_pendulum_datetime(expected_rows["col4_precision"]), 3 # type: ignore[arg-type] ) + if "col11" in expected_rows: + expected_rows["col11"] = reduce_pendulum_datetime_precision( + ensure_pendulum_time(expected_rows["col11"]), timestamp_precision # type: ignore[arg-type] + ).isoformat() + if "col11_precision" in expected_rows: parsed_time = ensure_pendulum_time(db_mapping["col11_precision"]) db_mapping["col11_precision"] = reduce_pendulum_datetime_precision(parsed_time, 3) diff --git a/tests/cli/test_init_command.py b/tests/cli/test_init_command.py index bf5c21c80f..3d9fd0909f 100644 --- a/tests/cli/test_init_command.py +++ b/tests/cli/test_init_command.py @@ -11,6 +11,9 @@ import re from packaging.requirements import Requirement +# import that because O3 modules cannot be unloaded +import cryptography.hazmat.bindings._rust + import dlt diff --git a/tests/load/dremio/__init__.py b/tests/load/dremio/__init__.py new file mode 100644 index 0000000000..7f077d7b73 --- /dev/null +++ b/tests/load/dremio/__init__.py @@ -0,0 +1,3 @@ +from tests.utils import skip_if_not_active + +skip_if_not_active("dremio") diff --git a/tests/load/dremio/bootstrap/Dockerfile b/tests/load/dremio/bootstrap/Dockerfile new file mode 100644 index 0000000000..1f1e63eac7 --- /dev/null +++ b/tests/load/dremio/bootstrap/Dockerfile @@ -0,0 +1,3 @@ +FROM python:3.12-slim +RUN apt-get update -y && apt-get upgrade -y && apt-get install -y curl +COPY . . \ No newline at end of file diff --git a/tests/load/dremio/bootstrap/bootstrap_dremio.sh b/tests/load/dremio/bootstrap/bootstrap_dremio.sh new file mode 100755 index 0000000000..36f1c8ad5d --- /dev/null +++ b/tests/load/dremio/bootstrap/bootstrap_dremio.sh @@ -0,0 +1,57 @@ +#!/usr/bin/env bash + +set -euxo pipefail + +DREMIO_URL=http://dremio:9047 + +# Poll the Dremio "live" endpoint until ready +until (curl "${DREMIO_URL}/live") do echo '...waiting...' && sleep 1; done; + +# Bootstrap the first user +curl "${DREMIO_URL}/apiv2/bootstrap/firstuser" \ + -X 'PUT' \ + -H 'Content-Type: application/json' \ + -H 'Authorization: null' \ + -d '{"userName":"dremio","firstName":"Dremio","lastName":"Admin","email":"dremio.admin@foo.com","createdAt":1694089769453,"password":"dremio123","extra":null}' \ + --fail-with-body + +# Get token for admin user to use with api +output=$(curl -X POST "${DREMIO_URL}/apiv2/login" \ + -H 'Accept: */*' \ + -H 'Connection: keep-alive' \ + -H 'Content-Type: application/json' \ + -d '{"userName":"dremio","password":"dremio123"}' \ + --fail-with-body +) +dremio_token=$(echo "$output" | python -c "import sys, json; print(json.load(sys.stdin)['token'])") +echo "$dremio_token" + +# Need to increase the "single_field_size_bytes" limit otherwise some tests fail. +curl "${DREMIO_URL}/api/v3/sql" \ + -X 'POST' \ + -H 'Accept: */*' \ + -H "Authorization: _dremio${dremio_token}" \ + -H 'Connection: keep-alive' \ + -H 'Content-Type: application/json' \ + -d '{"sql": "alter system set limits.single_field_size_bytes = 200000;"}' \ + --fail-with-body + +# Create a NAS source. This will contain final ICEBERG tables. +curl "${DREMIO_URL}/apiv2/source/nas/?nocache=1708370225409" \ + -X 'PUT' \ + -H 'Accept: */*' \ + -H "Authorization: _dremio${dremio_token}" \ + -H 'Connection: keep-alive' \ + -H 'Content-Type: application/json' \ + -d @nas.json \ + --fail-with-body + +# Create an S3 source using minio. This will be used for staging data. +curl "${DREMIO_URL}/apiv2/source/minio/?nocache=1708370225409" \ + -X 'PUT' \ + -H 'Accept: */*' \ + -H "Authorization: _dremio${dremio_token}" \ + -H 'Connection: keep-alive' \ + -H 'Content-Type: application/json' \ + -d @minio.json \ + --fail-with-body diff --git a/tests/load/dremio/bootstrap/minio.json b/tests/load/dremio/bootstrap/minio.json new file mode 100644 index 0000000000..47603653fb --- /dev/null +++ b/tests/load/dremio/bootstrap/minio.json @@ -0,0 +1,55 @@ +{ + "id": "b102f4cb-d233-4145-b32a-cc78d4d1ad44", + "name": "minio", + "config": { + "credentialType": "ACCESS_KEY", + "accessKey": "minioadmin", + "accessSecret": "minioadmin", + "secure": false, + "externalBucketList": [ + "dlt-ci-test-bucket" + ], + "enableAsync": true, + "compatibilityMode": true, + "requesterPays": false, + "enableFileStatusCheck": true, + "isPartitionInferenceEnabled": false, + "rootPath": "/", + "defaultCtasFormat": "ICEBERG", + "propertyList": [ + { + "name": "fs.s3a.path.style.access", + "value": "true" + }, + { + "name": "fs.s3a.endpoint", + "value": "minio:9000" + } + ], + "whitelistedBuckets": [ + "dlt-ci-test-bucket" + ], + "isCachingEnabled": true, + "maxCacheSpacePct": 100 + }, + "allowCrossSourceSelection": false, + "disableMetadataValidityCheck": false, + "accelerationRefreshPeriod": 3600000, + "accelerationGracePeriod": 10800000, + "accelerationNeverExpire": false, + "accelerationNeverRefresh": false, + "metadataPolicy": { + "deleteUnavailableDatasets": true, + "autoPromoteDatasets": false, + "namesRefreshMillis": 3600000, + "datasetDefinitionRefreshAfterMillis": 3600000, + "datasetDefinitionExpireAfterMillis": 10800000, + "authTTLMillis": 86400000, + "updateMode": "PREFETCH_QUERIED" + }, + "type": "S3", + "accessControlList": { + "userControls": [], + "roleControls": [] + } +} \ No newline at end of file diff --git a/tests/load/dremio/bootstrap/nas.json b/tests/load/dremio/bootstrap/nas.json new file mode 100644 index 0000000000..fd894bdbee --- /dev/null +++ b/tests/load/dremio/bootstrap/nas.json @@ -0,0 +1,24 @@ +{ + "name": "nas", + "config": { + "path": "/tmp", + "defaultCtasFormat": "ICEBERG", + "propertyList": [] + }, + "accelerationRefreshPeriod": 3600000, + "accelerationGracePeriod": 10800000, + "metadataPolicy": { + "deleteUnavailableDatasets": true, + "autoPromoteDatasets": false, + "namesRefreshMillis": 3600000, + "datasetDefinitionRefreshAfterMillis": 3600000, + "datasetDefinitionExpireAfterMillis": 10800000, + "authTTLMillis": 86400000, + "updateMode": "PREFETCH_QUERIED" + }, + "type": "NAS", + "accessControlList": { + "userControls": [], + "roleControls": [] + } +} diff --git a/tests/load/dremio/docker-compose.yml b/tests/load/dremio/docker-compose.yml new file mode 100644 index 0000000000..37f611b62a --- /dev/null +++ b/tests/load/dremio/docker-compose.yml @@ -0,0 +1,50 @@ +version: "3.7" +services: + dremio: + image: dremio/dremio-oss + ports: + - "9047:9047" + - "31010:31010" + - "32010:32010" + - "45678:45678" + networks: + - dremio + minio: + image: minio/minio + command: server /data --console-address ":9001" + environment: + - MINIO_ROOT_USER=minioadmin + - MINIO_ROOT_PASSWORD=minioadmin + # volumes: + # - minio_data:/data + ports: + - "9010:9000" + - "9011:9001" + networks: + - dremio + createbuckets: + image: minio/mc + networks: + - dremio + depends_on: + - minio + entrypoint: > + /bin/sh -c " + sleep 5; + /usr/bin/mc alias set myminio http://minio:9000 minioadmin minioadmin; + /usr/bin/mc mb myminio/dlt-ci-test-bucket; + /usr/bin/mc anonymous set public myminio/dlt-ci-test-bucket; + exit 0; + " + bootstrapdremio: + depends_on: + - dremio + - createbuckets + build: + context: bootstrap + networks: + - dremio + command: ./bootstrap_dremio.sh +networks: + dremio: + name: dremio \ No newline at end of file diff --git a/tests/load/dremio/test_dremio_client.py b/tests/load/dremio/test_dremio_client.py new file mode 100644 index 0000000000..d0002dc343 --- /dev/null +++ b/tests/load/dremio/test_dremio_client.py @@ -0,0 +1,91 @@ +import pytest + +from dlt.common.schema import TColumnSchema, Schema +from dlt.destinations.impl.dremio.configuration import DremioClientConfiguration, DremioCredentials +from dlt.destinations.impl.dremio.dremio import DremioClient +from tests.load.utils import empty_schema + + +@pytest.fixture +def dremio_client(empty_schema: Schema) -> DremioClient: + creds = DremioCredentials() + creds.database = "test_database" + return DremioClient( + empty_schema, + DremioClientConfiguration(credentials=creds)._bind_dataset_name( + dataset_name="test_dataset" + ), + ) + + +def test_dremio_factory() -> None: + from dlt.destinations import dremio + + dest = dremio( + "grpc://username:password@host:1111/data_source", staging_data_source="s3_dlt_stage" + ) + config = dest.configuration(DremioClientConfiguration()._bind_dataset_name("test_dataset")) + assert config.staging_data_source == "s3_dlt_stage" + assert ( + config.credentials.to_url().render_as_string(hide_password=False) + == "grpc://username:password@host:1111/data_source" + ) + assert ( + config.credentials.to_native_representation() + == "grpc://username:password@host:1111/data_source" + ) + # simplified url ad needed by pydremio + assert config.credentials.to_native_credentials() == "grpc://host:1111" + assert config.credentials.db_kwargs() == {"password": "password", "username": "username"} + + +@pytest.mark.parametrize( + argnames=("new_columns", "generate_alter", "expected_sql"), + argvalues=[ + ( + [ + TColumnSchema(name="foo", data_type="text", partition=True), + TColumnSchema(name="bar", data_type="bigint", sort=True), + TColumnSchema(name="baz", data_type="double"), + ], + False, + [ + 'CREATE TABLE "test_database"."test_dataset"."event_test_table"' + ' (\n"foo" VARCHAR ,\n"bar" BIGINT ,\n"baz" DOUBLE )\nPARTITION BY' + ' ("foo")\nLOCALSORT BY ("bar")' + ], + ), + ( + [ + TColumnSchema(name="foo", data_type="text", partition=True), + TColumnSchema(name="bar", data_type="bigint", partition=True), + TColumnSchema(name="baz", data_type="double"), + ], + False, + [ + 'CREATE TABLE "test_database"."test_dataset"."event_test_table"' + ' (\n"foo" VARCHAR ,\n"bar" BIGINT ,\n"baz" DOUBLE )\nPARTITION BY' + ' ("foo","bar")' + ], + ), + ( + [ + TColumnSchema(name="foo", data_type="text"), + TColumnSchema(name="bar", data_type="bigint"), + TColumnSchema(name="baz", data_type="double"), + ], + False, + [ + 'CREATE TABLE "test_database"."test_dataset"."event_test_table"' + ' (\n"foo" VARCHAR ,\n"bar" BIGINT ,\n"baz" DOUBLE )' + ], + ), + ], +) +def test_get_table_update_sql(dremio_client, new_columns, generate_alter, expected_sql): + assert ( + dremio_client._get_table_update_sql( + table_name="event_test_table", new_columns=new_columns, generate_alter=generate_alter + ) + == expected_sql + ) diff --git a/tests/load/pipeline/test_dremio.py b/tests/load/pipeline/test_dremio.py new file mode 100644 index 0000000000..9f4fd75c93 --- /dev/null +++ b/tests/load/pipeline/test_dremio.py @@ -0,0 +1,34 @@ +import pytest +from typing import Iterator, Any + +import dlt +from tests.load.pipeline.utils import load_table_counts +from tests.load.utils import DestinationTestConfiguration, destinations_configs + + +@pytest.mark.parametrize( + "destination_config", + destinations_configs(default_sql_configs=True, subset=["dremio"]), + ids=lambda x: x.name, +) +def test_dremio(destination_config: DestinationTestConfiguration) -> None: + pipeline = destination_config.setup_pipeline( + "dremio-test", dataset_name="bar", full_refresh=True + ) + + @dlt.resource(name="items", write_disposition="replace") + def items() -> Iterator[Any]: + yield { + "id": 1, + "name": "item", + "sub_items": [{"id": 101, "name": "sub item 101"}, {"id": 101, "name": "sub item 102"}], + } + + print(pipeline.run([items])) + + 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 diff --git a/tests/load/pipeline/test_merge_disposition.py b/tests/load/pipeline/test_merge_disposition.py index fc6b82d1f4..86e955a48e 100644 --- a/tests/load/pipeline/test_merge_disposition.py +++ b/tests/load/pipeline/test_merge_disposition.py @@ -863,7 +863,7 @@ def data_resource(data): # compare observed records with expected records, now for child table qual_name = p.sql_client().make_qualified_table_name(table_name + "__val") - observed = [row[0] for row in select_data(p, f"SELECT value FROM {qual_name}")] + observed = [row[0] for row in select_data(p, f'SELECT "value" FROM {qual_name}')] assert sorted(observed) == [7, 8, 9] # type: ignore[type-var] table_name = "test_dedup_sort_hint_with_hard_delete" diff --git a/tests/load/pipeline/test_pipelines.py b/tests/load/pipeline/test_pipelines.py index ba36f4afab..36e89f517c 100644 --- a/tests/load/pipeline/test_pipelines.py +++ b/tests/load/pipeline/test_pipelines.py @@ -80,11 +80,11 @@ def data_fun() -> Iterator[Any]: # this will create default schema p.extract(data_fun) # _pipeline suffix removed when creating default schema name - assert p.default_schema_name in ["dlt_pytest", "dlt"] + assert p.default_schema_name in ["dlt_pytest", "dlt", "dlt_jb_pytest_runner"] # this will create additional schema p.extract(data_fun(), schema=dlt.Schema("names")) - assert p.default_schema_name in ["dlt_pytest", "dlt"] + assert p.default_schema_name in ["dlt_pytest", "dlt", "dlt_jb_pytest_runner"] assert "names" in p.schemas.keys() with pytest.raises(PipelineConfigMissing): @@ -816,7 +816,7 @@ def other_data(): column_schemas.pop("col11_null") column_schemas.pop("col11_precision") - if destination_config.destination == "redshift": + if destination_config.destination in ("redshift", "dremio"): data_types.pop("col7_precision") column_schemas.pop("col7_precision") @@ -865,7 +865,7 @@ def some_source(): schema=column_schemas, parse_complex_strings=destination_config.destination in ["snowflake", "bigquery", "redshift"], - timestamp_precision=3 if destination_config.destination == "athena" else 6, + timestamp_precision=3 if destination_config.destination in ("athena", "dremio") else 6, ) diff --git a/tests/load/pipeline/test_stage_loading.py b/tests/load/pipeline/test_stage_loading.py index 222171ca61..4d289a5384 100644 --- a/tests/load/pipeline/test_stage_loading.py +++ b/tests/load/pipeline/test_stage_loading.py @@ -94,13 +94,15 @@ def test_staging_load(destination_config: DestinationTestConfiguration) -> None: # check item of first row in db with pipeline.sql_client() as sql_client: + qual_name = sql_client.make_qualified_table_name if destination_config.destination in ["mssql", "synapse"]: - qual_name = sql_client.make_qualified_table_name rows = sql_client.execute_sql( f"SELECT TOP 1 url FROM {qual_name('issues')} WHERE id = 388089021" ) else: - rows = sql_client.execute_sql("SELECT url FROM issues WHERE id = 388089021 LIMIT 1") + rows = sql_client.execute_sql( + f"SELECT url FROM {qual_name('issues')} WHERE id = 388089021 LIMIT 1" + ) assert rows[0][0] == "https://api.github.com/repos/duckdb/duckdb/issues/71" if destination_config.supports_merge: @@ -125,10 +127,10 @@ def test_staging_load(destination_config: DestinationTestConfiguration) -> None: ) else: rows_1 = sql_client.execute_sql( - "SELECT number FROM issues WHERE id = 1232152492 LIMIT 1" + f"SELECT number FROM {qual_name('issues')} WHERE id = 1232152492 LIMIT 1" ) rows_2 = sql_client.execute_sql( - "SELECT number FROM issues WHERE id = 1142699354 LIMIT 1" + f"SELECT number FROM {qual_name('issues')} WHERE id = 1142699354 LIMIT 1" ) assert rows_1[0][0] == 105 assert rows_2[0][0] == 300 @@ -184,7 +186,10 @@ def test_all_data_types(destination_config: DestinationTestConfiguration) -> Non if destination_config.destination == "synapse" and destination_config.file_format == "parquet": # TIME columns are not supported for staged parquet loads into Synapse exclude_types.append("time") - if destination_config.destination == "redshift" and destination_config.file_format in ( + if destination_config.destination in ( + "redshift", + "dremio", + ) and destination_config.file_format in ( "parquet", "jsonl", ): diff --git a/tests/load/pipeline/test_write_disposition_changes.py b/tests/load/pipeline/test_write_disposition_changes.py index 164243cd55..50986727ed 100644 --- a/tests/load/pipeline/test_write_disposition_changes.py +++ b/tests/load/pipeline/test_write_disposition_changes.py @@ -124,6 +124,14 @@ def source(): "config", {} ) + # 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" + + if destination_allows_adding_root_key and not with_root_key: + return + # without a root key this will fail, it is expected if not with_root_key and destination_config.supports_merge: with pytest.raises(PipelineStepFailed): diff --git a/tests/load/test_job_client.py b/tests/load/test_job_client.py index a5dfb5d829..2204d1facf 100644 --- a/tests/load/test_job_client.py +++ b/tests/load/test_job_client.py @@ -393,6 +393,8 @@ def test_get_storage_table_with_all_types(client: SqlJobClientBase) -> None: continue if client.config.destination_type == "databricks" and c["data_type"] in ("complex", "time"): continue + if client.config.destination_type == "dremio" and c["data_type"] == "complex": + continue assert c["data_type"] == expected_c["data_type"] @@ -657,7 +659,9 @@ def test_retrieve_job(client: SqlJobClientBase, file_storage: FileStorage) -> No @pytest.mark.parametrize( - "destination_config", destinations_configs(default_sql_configs=True), ids=lambda x: x.name + "destination_config", + destinations_configs(default_sql_configs=True, exclude=["dremio"]), + ids=lambda x: x.name, ) def test_default_schema_name_init_storage(destination_config: DestinationTestConfiguration) -> None: with cm_yield_client_with_storage( diff --git a/tests/load/test_sql_client.py b/tests/load/test_sql_client.py index da162621b1..bd1ec5ba43 100644 --- a/tests/load/test_sql_client.py +++ b/tests/load/test_sql_client.py @@ -38,7 +38,7 @@ def client(request) -> Iterator[SqlJobClientBase]: @pytest.mark.parametrize( "client", - destinations_configs(default_sql_configs=True, exclude=["mssql", "synapse"]), + destinations_configs(default_sql_configs=True, exclude=["mssql", "synapse", "dremio"]), indirect=True, ids=lambda x: x.name, ) @@ -199,8 +199,12 @@ def test_execute_ddl(client: SqlJobClientBase) -> None: client.sql_client.execute_sql(f"INSERT INTO {f_q_table_name} VALUES (1.0)") rows = client.sql_client.execute_sql(f"SELECT * FROM {f_q_table_name}") assert rows[0][0] == Decimal("1.0") - # create view, note that bigquery will not let you execute a view that does not have fully qualified table names. - view_name = client.sql_client.make_qualified_table_name(f"view_tmp_{uniq_suffix}") + if client.config.destination_type == "dremio": + username = client.config.credentials["username"] + view_name = f'"@{username}"."view_tmp_{uniq_suffix}"' + else: + # create view, note that bigquery will not let you execute a view that does not have fully qualified table names. + view_name = client.sql_client.make_qualified_table_name(f"view_tmp_{uniq_suffix}") client.sql_client.execute_sql(f"CREATE VIEW {view_name} AS (SELECT * FROM {f_q_table_name});") rows = client.sql_client.execute_sql(f"SELECT * FROM {view_name}") assert rows[0][0] == Decimal("1.0") @@ -365,10 +369,11 @@ 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) - with pytest.raises(DatabaseUndefinedRelation) as term_ex: - with client.sql_client.execute_query("DROP SCHEMA UNKNOWN"): - pass - assert client.sql_client.is_dbapi_exception(term_ex.value.dbapi_exception) + if client.config.destination_type != "dremio": + with pytest.raises(DatabaseUndefinedRelation) as term_ex: + with client.sql_client.execute_query("DROP SCHEMA UNKNOWN"): + pass + assert client.sql_client.is_dbapi_exception(term_ex.value.dbapi_exception) @pytest.mark.parametrize( diff --git a/tests/load/utils.py b/tests/load/utils.py index 17dd6a24fd..1eb078c72b 100644 --- a/tests/load/utils.py +++ b/tests/load/utils.py @@ -65,14 +65,14 @@ ] # Filter out buckets not in all filesystem drivers -DEFAULT_BUCKETS = [GCS_BUCKET, AWS_BUCKET, FILE_BUCKET, MEMORY_BUCKET, AZ_BUCKET] -DEFAULT_BUCKETS = [ - bucket for bucket in DEFAULT_BUCKETS if bucket.split(":")[0] in ALL_FILESYSTEM_DRIVERS +WITH_GDRIVE_BUCKETS = [GCS_BUCKET, AWS_BUCKET, FILE_BUCKET, MEMORY_BUCKET, AZ_BUCKET, GDRIVE_BUCKET] +WITH_GDRIVE_BUCKETS = [ + bucket for bucket in WITH_GDRIVE_BUCKETS if bucket.split(":")[0] in ALL_FILESYSTEM_DRIVERS ] # temporary solution to include gdrive bucket in tests, # while gdrive is not working as a destination -WITH_GDRIVE_BUCKETS = [GDRIVE_BUCKET] + DEFAULT_BUCKETS +DEFAULT_BUCKETS = [bucket for bucket in WITH_GDRIVE_BUCKETS if bucket != GDRIVE_BUCKET] # Add r2 in extra buckets so it's not run for all tests R2_BUCKET_CONFIG = dict( @@ -175,7 +175,7 @@ def destinations_configs( destination_configs += [ DestinationTestConfiguration(destination=destination) for destination in SQL_DESTINATIONS - if destination not in ("athena", "mssql", "synapse", "databricks") + if destination not in ("athena", "mssql", "synapse", "databricks", "dremio") ] destination_configs += [ DestinationTestConfiguration(destination="duckdb", file_format="parquet") @@ -208,6 +208,15 @@ def destinations_configs( extra_info="az-authorization", ) ] + destination_configs += [ + DestinationTestConfiguration( + destination="dremio", + staging="filesystem", + file_format="parquet", + bucket_url=AWS_BUCKET, + supports_dbt=False, + ) + ] destination_configs += [ DestinationTestConfiguration(destination="mssql", supports_dbt=False), DestinationTestConfiguration(destination="synapse", supports_dbt=False), @@ -302,6 +311,13 @@ def destinations_configs( bucket_url=AZ_BUCKET, extra_info="az-authorization", ), + DestinationTestConfiguration( + destination="dremio", + staging="filesystem", + file_format="parquet", + bucket_url=AWS_BUCKET, + supports_dbt=False, + ), ] if all_staging_configs: diff --git a/tests/utils.py b/tests/utils.py index 73e99c3fcd..410c2363d3 100644 --- a/tests/utils.py +++ b/tests/utils.py @@ -48,6 +48,7 @@ "destination", "synapse", "databricks", + "dremio", } NON_SQL_DESTINATIONS = {"filesystem", "weaviate", "dummy", "motherduck", "qdrant", "destination"} SQL_DESTINATIONS = IMPLEMENTED_DESTINATIONS - NON_SQL_DESTINATIONS