File: client.py

package info (click to toggle)
python-clickhouse-driver 0.2.5-2
  • links: PTS, VCS
  • area: main
  • in suites: forky, sid, trixie
  • size: 2,516 kB
  • sloc: python: 10,950; pascal: 42; makefile: 31; sh: 3
file content (792 lines) | stat: -rw-r--r-- 30,231 bytes parent folder | download | duplicates (2)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
import re
import ssl
from collections import deque
from contextlib import contextmanager
from time import time
import types
from urllib.parse import urlparse, parse_qs, unquote

from . import errors, defines
from .block import ColumnOrientedBlock, RowOrientedBlock
from .connection import Connection
from .log import log_block
from .protocol import ServerPacketTypes
from .result import (
    IterQueryResult, ProgressQueryResult, QueryResult, QueryInfo
)
from .util.escape import escape_params
from .util.helpers import column_chunks, chunks, asbool


class Client(object):
    """
    Client for communication with the ClickHouse server.
    Single connection is established per each connected instance of the client.

    :param settings: Dictionary of settings that passed to every query (except
                     for the client settings, see below). Defaults to ``None``
                     (no additional settings). See all available settings in
                     `ClickHouse docs
                     <https://clickhouse.com/docs/en/operations/settings/settings/>`_.
    :param \\**kwargs: All other args are passed to the
                       :py:class:`~clickhouse_driver.connection.Connection`
                       constructor.

    The following keys when passed in ``settings`` are used for configuring the
    client itself:

        * ``insert_block_size`` -- chunk size to split rows for ``INSERT``.
          Defaults to ``1048576``.
        * ``strings_as_bytes`` -- turns off string column encoding/decoding.
        * ``strings_encoding`` -- specifies string encoding. UTF-8 by default.
        * ``use_numpy`` -- Use NumPy for columns reading. New in version
                           *0.2.0*.
        * ``opentelemetry_traceparent`` -- OpenTelemetry traceparent header as
                           described by W3C Trace Context recommendation.
                           New in version *0.2.2*.
        * ``opentelemetry_tracestate`` -- OpenTelemetry tracestate header as
                           described by W3C Trace Context recommendation.
                           New in version *0.2.2*.
        * ``quota_key`` -- A string to differentiate quotas when the user have
                           keyed quotas configured on server.
                           New in version *0.2.3*.
        * ``input_format_null_as_default`` -- Initialize null fields with
                           default values if data type of this field is not
                           nullable. Does not work for NumPy. Default: False.
                           New in version *0.2.4*.
        * ``round_robin`` -- If ``alt_hosts`` are provided the query will be
                           executed on host picked with round-robin algorithm.
                           New in version *0.2.5*.
    """

    available_client_settings = (
        'insert_block_size',  # TODO: rename to max_insert_block_size
        'strings_as_bytes',
        'strings_encoding',
        'use_numpy',
        'opentelemetry_traceparent',
        'opentelemetry_tracestate',
        'quota_key',
        'input_format_null_as_default'
    )

    def __init__(self, *args, **kwargs):
        self.settings = (kwargs.pop('settings', None) or {}).copy()

        self.client_settings = {
            'insert_block_size': int(self.settings.pop(
                'insert_block_size', defines.DEFAULT_INSERT_BLOCK_SIZE,
            )),
            'strings_as_bytes': self.settings.pop(
                'strings_as_bytes', False
            ),
            'strings_encoding': self.settings.pop(
                'strings_encoding', defines.STRINGS_ENCODING
            ),
            'use_numpy': self.settings.pop(
                'use_numpy', False
            ),
            'opentelemetry_traceparent': self.settings.pop(
                'opentelemetry_traceparent', None
            ),
            'opentelemetry_tracestate': self.settings.pop(
                'opentelemetry_tracestate', ''
            ),
            'quota_key': self.settings.pop(
                'quota_key', ''
            ),
            'input_format_null_as_default': self.settings.pop(
                'input_format_null_as_default', False
            )
        }

        if self.client_settings['use_numpy']:
            try:
                from .numpy.result import (
                    NumpyIterQueryResult, NumpyProgressQueryResult,
                    NumpyQueryResult
                )
                self.query_result_cls = NumpyQueryResult
                self.iter_query_result_cls = NumpyIterQueryResult
                self.progress_query_result_cls = NumpyProgressQueryResult
            except ImportError:
                raise RuntimeError('Extras for NumPy must be installed')
        else:
            self.query_result_cls = QueryResult
            self.iter_query_result_cls = IterQueryResult
            self.progress_query_result_cls = ProgressQueryResult

        round_robin = kwargs.pop('round_robin', False)
        self.connections = deque([Connection(*args, **kwargs)])

        if round_robin and 'alt_hosts' in kwargs:
            alt_hosts = kwargs.pop('alt_hosts')
            for host in alt_hosts.split(','):
                url = urlparse('clickhouse://' + host)

                connection_kwargs = kwargs.copy()
                if len(args) > 2:
                    # port as positional argument
                    connection_args = (url.hostname, url.port) + args[2:]
                else:
                    # port as keyword argument
                    connection_args = (url.hostname, ) + args[1:]
                    connection_kwargs['port'] = url.port

                connection = Connection(*connection_args, **connection_kwargs)
                self.connections.append(connection)

        self.connection = self.get_connection()
        self.reset_last_query()
        super(Client, self).__init__()

    def __enter__(self):
        return self

    def __exit__(self, exc_type, exc_val, exc_tb):
        self.disconnect()

    def get_connection(self):
        if hasattr(self, 'connection'):
            self.connections.append(self.connection)

        connection = self.connections.popleft()

        connection.context.settings = self.settings
        connection.context.client_settings = self.client_settings
        return connection

    def disconnect(self):
        self.disconnect_connection()
        for connection in self.connections:
            connection.disconnect()

    def disconnect_connection(self):
        """
        Disconnects from the server.
        """
        self.connection.disconnect()
        self.reset_last_query()

    def reset_last_query(self):
        self.last_query = None

    def receive_result(self, with_column_types=False, progress=False,
                       columnar=False):

        gen = self.packet_generator()

        if progress:
            return self.progress_query_result_cls(
                gen, with_column_types=with_column_types, columnar=columnar
            )

        else:
            result = self.query_result_cls(
                gen, with_column_types=with_column_types, columnar=columnar
            )
            return result.get_result()

    def iter_receive_result(self, with_column_types=False):
        gen = self.packet_generator()

        result = self.iter_query_result_cls(
            gen, with_column_types=with_column_types
        )

        for rows in result:
            for row in rows:
                yield row

    def packet_generator(self):
        while True:
            try:
                packet = self.receive_packet()
                if not packet:
                    break

                if packet is True:
                    continue

                yield packet

            except (Exception, KeyboardInterrupt):
                self.disconnect()
                raise

    def receive_packet(self):
        packet = self.connection.receive_packet()

        if packet.type == ServerPacketTypes.EXCEPTION:
            raise packet.exception

        elif packet.type == ServerPacketTypes.PROGRESS:
            self.last_query.store_progress(packet.progress)
            return packet

        elif packet.type == ServerPacketTypes.END_OF_STREAM:
            return False

        elif packet.type == ServerPacketTypes.DATA:
            return packet

        elif packet.type == ServerPacketTypes.TOTALS:
            return packet

        elif packet.type == ServerPacketTypes.EXTREMES:
            return packet

        elif packet.type == ServerPacketTypes.PROFILE_INFO:
            self.last_query.store_profile(packet.profile_info)
            return True

        else:
            return True

    def make_query_settings(self, settings):
        settings = dict(settings or {})

        # Pick client-related settings.
        client_settings = self.client_settings.copy()
        for key in self.available_client_settings:
            if key in settings:
                client_settings[key] = settings.pop(key)

        self.connection.context.client_settings = client_settings

        # The rest of settings are ClickHouse-related.
        query_settings = self.settings.copy()
        query_settings.update(settings)
        self.connection.context.settings = query_settings

    def track_current_database(self, query):
        query = query.strip('; ')
        if query.lower().startswith('use '):
            self.connection.database = query[4:].strip()

    def establish_connection(self, settings):
        num_connections = len(self.connections)
        if hasattr(self, 'connection'):
            num_connections += 1

        for i in range(num_connections):
            try:
                self.connection = self.get_connection()
                self.make_query_settings(settings)
                self.connection.force_connect()
                self.last_query = QueryInfo()

            except (errors.SocketTimeoutError, errors.NetworkError):
                if i < num_connections - 1:
                    continue
                raise

            return

    @contextmanager
    def disconnect_on_error(self, query, settings):
        try:
            self.establish_connection(settings)

            yield

            self.track_current_database(query)

        except (Exception, KeyboardInterrupt):
            self.disconnect()
            raise

    def execute(self, query, params=None, with_column_types=False,
                external_tables=None, query_id=None, settings=None,
                types_check=False, columnar=False):
        """
        Executes query.

        Establishes new connection if it wasn't established yet.
        After query execution connection remains intact for next queries.
        If connection can't be reused it will be closed and new connection will
        be created.

        :param query: query that will be send to server.
        :param params: substitution parameters for SELECT queries and data for
                       INSERT queries. Data for INSERT can be `list`, `tuple`
                       or :data:`~types.GeneratorType`.
                       Defaults to ``None`` (no parameters  or data).
        :param with_column_types: if specified column names and types will be
                                  returned alongside with result.
                                  Defaults to ``False``.
        :param external_tables: external tables to send.
                                Defaults to ``None`` (no external tables).
        :param query_id: the query identifier. If no query id specified
                         ClickHouse server will generate it.
        :param settings: dictionary of query settings.
                         Defaults to ``None`` (no additional settings).
        :param types_check: enables type checking of data for INSERT queries.
                            Causes additional overhead. Defaults to ``False``.
        :param columnar: if specified the result of the SELECT query will be
                         returned in column-oriented form.
                         It also allows to INSERT data in columnar form.
                         Defaults to ``False`` (row-like form).

        :return: * number of inserted rows for INSERT queries with data.
                   Returning rows count from INSERT FROM SELECT is not
                   supported.
                 * if `with_column_types=False`: `list` of `tuples` with
                   rows/columns.
                 * if `with_column_types=True`: `tuple` of 2 elements:
                    * The first element is `list` of `tuples` with
                      rows/columns.
                    * The second element information is about columns: names
                      and types.
        """

        start_time = time()

        with self.disconnect_on_error(query, settings):
            # INSERT queries can use list/tuple/generator of list/tuples/dicts.
            # For SELECT parameters can be passed in only in dict right now.
            is_insert = isinstance(params, (list, tuple, types.GeneratorType))

            if is_insert:
                rv = self.process_insert_query(
                    query, params, external_tables=external_tables,
                    query_id=query_id, types_check=types_check,
                    columnar=columnar
                )
            else:
                rv = self.process_ordinary_query(
                    query, params=params, with_column_types=with_column_types,
                    external_tables=external_tables,
                    query_id=query_id, types_check=types_check,
                    columnar=columnar
                )
            self.last_query.store_elapsed(time() - start_time)
            return rv

    def execute_with_progress(
            self, query, params=None, with_column_types=False,
            external_tables=None, query_id=None, settings=None,
            types_check=False, columnar=False):
        """
        Executes SELECT query with progress information.
        See, :ref:`execute-with-progress`.

        :param query: query that will be send to server.
        :param params: substitution parameters for SELECT queries and data for
                       INSERT queries. Data for INSERT can be `list`, `tuple`
                       or :data:`~types.GeneratorType`.
                       Defaults to ``None`` (no parameters  or data).
        :param with_column_types: if specified column names and types will be
                                  returned alongside with result.
                                  Defaults to ``False``.
        :param external_tables: external tables to send.
                                Defaults to ``None`` (no external tables).
        :param query_id: the query identifier. If no query id specified
                         ClickHouse server will generate it.
        :param settings: dictionary of query settings.
                         Defaults to ``None`` (no additional settings).
        :param types_check: enables type checking of data for INSERT queries.
                            Causes additional overhead. Defaults to ``False``.
        :param columnar: if specified the result will be returned in
                         column-oriented form.
                         Defaults to ``False`` (row-like form).
        :return: :ref:`progress-query-result` proxy.
        """

        with self.disconnect_on_error(query, settings):
            return self.process_ordinary_query_with_progress(
                query, params=params, with_column_types=with_column_types,
                external_tables=external_tables, query_id=query_id,
                types_check=types_check, columnar=columnar
            )

    def execute_iter(
            self, query, params=None, with_column_types=False,
            external_tables=None, query_id=None, settings=None,
            types_check=False, chunk_size=1):
        """
        *New in version 0.0.14.*

        Executes SELECT query with results streaming. See, :ref:`execute-iter`.

        :param query: query that will be send to server.
        :param params: substitution parameters for SELECT queries and data for
                       INSERT queries. Data for INSERT can be `list`, `tuple`
                       or :data:`~types.GeneratorType`.
                       Defaults to ``None`` (no parameters  or data).
        :param with_column_types: if specified column names and types will be
                                  returned alongside with result.
                                  Defaults to ``False``.
        :param external_tables: external tables to send.
                                Defaults to ``None`` (no external tables).
        :param query_id: the query identifier. If no query id specified
                         ClickHouse server will generate it.
        :param settings: dictionary of query settings.
                         Defaults to ``None`` (no additional settings).
        :param types_check: enables type checking of data for INSERT queries.
                            Causes additional overhead. Defaults to ``False``.
        :param chunk_size: chunk query results.
        :return: :ref:`iter-query-result` proxy.
        """
        with self.disconnect_on_error(query, settings):
            rv = self.iter_process_ordinary_query(
                query, params=params, with_column_types=with_column_types,
                external_tables=external_tables,
                query_id=query_id, types_check=types_check
            )
            return chunks(rv, chunk_size) if chunk_size > 1 else rv

    def query_dataframe(
            self, query, params=None, external_tables=None, query_id=None,
            settings=None):
        """
        *New in version 0.2.0.*

        Queries DataFrame with specified SELECT query.

        :param query: query that will be send to server.
        :param params: substitution parameters.
                       Defaults to ``None`` (no parameters  or data).
        :param external_tables: external tables to send.
                                Defaults to ``None`` (no external tables).
        :param query_id: the query identifier. If no query id specified
                         ClickHouse server will generate it.
        :param settings: dictionary of query settings.
                         Defaults to ``None`` (no additional settings).
        :return: pandas DataFrame.
        """

        try:
            import pandas as pd
        except ImportError:
            raise RuntimeError('Extras for NumPy must be installed')

        data, columns = self.execute(
            query, columnar=True, with_column_types=True, params=params,
            external_tables=external_tables, query_id=query_id,
            settings=settings
        )

        columns = [re.sub(r'\W', '_', name) for name, type_ in columns]
        return pd.DataFrame(
            {col: d for d, col in zip(data, columns)}, columns=columns
        )

    def insert_dataframe(
            self, query, dataframe, external_tables=None, query_id=None,
            settings=None):
        """
        *New in version 0.2.0.*

        Inserts pandas DataFrame with specified query.

        :param query: query that will be send to server.
        :param dataframe: pandas DataFrame.
        :param external_tables: external tables to send.
                                Defaults to ``None`` (no external tables).
        :param query_id: the query identifier. If no query id specified
                         ClickHouse server will generate it.
        :param settings: dictionary of query settings.
                         Defaults to ``None`` (no additional settings).
        :return: number of inserted rows.
        """

        try:
            import pandas as pd  # noqa: F401
        except ImportError:
            raise RuntimeError('Extras for NumPy must be installed')

        start_time = time()

        with self.disconnect_on_error(query, settings):
            self.connection.send_query(query, query_id=query_id)
            self.connection.send_external_tables(external_tables)

            sample_block = self.receive_sample_block()
            rv = None
            if sample_block:
                columns = [x[0] for x in sample_block.columns_with_types]
                # raise if any columns are missing from the dataframe
                diff = set(columns) - set(dataframe.columns)
                if len(diff):
                    msg = "DataFrame missing required columns: {}"
                    raise ValueError(msg.format(list(diff)))

                data = [dataframe[column].values for column in columns]
                rv = self.send_data(sample_block, data, columnar=True)
                self.receive_end_of_query()

            self.last_query.store_elapsed(time() - start_time)
            return rv

    def process_ordinary_query_with_progress(
            self, query, params=None, with_column_types=False,
            external_tables=None, query_id=None,
            types_check=False, columnar=False):

        if params is not None:
            query = self.substitute_params(
                query, params, self.connection.context
            )

        self.connection.send_query(query, query_id=query_id)
        self.connection.send_external_tables(external_tables,
                                             types_check=types_check)
        return self.receive_result(with_column_types=with_column_types,
                                   progress=True, columnar=columnar)

    def process_ordinary_query(
            self, query, params=None, with_column_types=False,
            external_tables=None, query_id=None,
            types_check=False, columnar=False):

        if params is not None:
            query = self.substitute_params(
                query, params, self.connection.context
            )

        self.connection.send_query(query, query_id=query_id)
        self.connection.send_external_tables(external_tables,
                                             types_check=types_check)
        return self.receive_result(with_column_types=with_column_types,
                                   columnar=columnar)

    def iter_process_ordinary_query(
            self, query, params=None, with_column_types=False,
            external_tables=None, query_id=None,
            types_check=False):

        if params is not None:
            query = self.substitute_params(
                query, params, self.connection.context
            )

        self.connection.send_query(query, query_id=query_id)
        self.connection.send_external_tables(external_tables,
                                             types_check=types_check)
        return self.iter_receive_result(with_column_types=with_column_types)

    def process_insert_query(self, query_without_data, data,
                             external_tables=None, query_id=None,
                             types_check=False, columnar=False):
        self.connection.send_query(query_without_data, query_id=query_id)
        self.connection.send_external_tables(external_tables,
                                             types_check=types_check)

        sample_block = self.receive_sample_block()
        if sample_block:
            rv = self.send_data(sample_block, data,
                                types_check=types_check, columnar=columnar)
            self.receive_end_of_query()
            return rv

    def receive_sample_block(self):
        while True:
            packet = self.connection.receive_packet()

            if packet.type == ServerPacketTypes.DATA:
                return packet.block

            elif packet.type == ServerPacketTypes.EXCEPTION:
                raise packet.exception

            elif packet.type == ServerPacketTypes.LOG:
                log_block(packet.block)

            elif packet.type == ServerPacketTypes.TABLE_COLUMNS:
                pass

            else:
                message = self.connection.unexpected_packet_message(
                    'Data, Exception, Log or TableColumns', packet.type
                )
                raise errors.UnexpectedPacketFromServerError(message)

    def send_data(self, sample_block, data, types_check=False, columnar=False):
        inserted_rows = 0

        client_settings = self.connection.context.client_settings
        block_cls = ColumnOrientedBlock if columnar else RowOrientedBlock

        if client_settings['use_numpy']:
            try:
                from .numpy.helpers import column_chunks as numpy_column_chunks

                if columnar:
                    slicer = numpy_column_chunks
                else:
                    raise ValueError(
                        'NumPy inserts is only allowed with columnar=True'
                    )

            except ImportError:
                raise RuntimeError('Extras for NumPy must be installed')

        else:
            slicer = column_chunks if columnar else chunks

        for chunk in slicer(data, client_settings['insert_block_size']):
            block = block_cls(sample_block.columns_with_types, chunk,
                              types_check=types_check)
            self.connection.send_data(block)
            inserted_rows += block.num_rows

        # Empty block means end of data.
        self.connection.send_data(block_cls())
        return inserted_rows

    def receive_end_of_query(self):
        while True:
            packet = self.connection.receive_packet()

            if packet.type == ServerPacketTypes.END_OF_STREAM:
                break

            elif packet.type == ServerPacketTypes.PROGRESS:
                continue

            elif packet.type == ServerPacketTypes.EXCEPTION:
                raise packet.exception

            elif packet.type == ServerPacketTypes.LOG:
                log_block(packet.block)

            elif packet.type == ServerPacketTypes.TABLE_COLUMNS:
                pass

            elif packet.type == ServerPacketTypes.PROFILE_EVENTS:
                pass

            else:
                message = self.connection.unexpected_packet_message(
                    'Exception, EndOfStream or Log', packet.type
                )
                raise errors.UnexpectedPacketFromServerError(message)

    def cancel(self, with_column_types=False):
        # TODO: Add warning if already cancelled.
        self.connection.send_cancel()
        # Client must still read until END_OF_STREAM packet.
        return self.receive_result(with_column_types=with_column_types)

    def substitute_params(self, query, params, context):
        """
        Substitutes parameters into a provided query.

        For example::

            client = Client(...)

            substituted_query = client.substitute_params(
                query='SELECT 1234, %(foo)s',
                params={'foo': 'bar'},
                context=client.connection.context
            )

            # prints: SELECT 1234, 'bar'
            print(substituted_query)
        """
        if not isinstance(params, dict):
            raise ValueError('Parameters are expected in dict form')

        escaped = escape_params(params, context)
        return query % escaped

    @classmethod
    def from_url(cls, url):
        """
        Return a client configured from the given URL.

        For example::

            clickhouse://[user:password]@localhost:9000/default
            clickhouses://[user:password]@localhost:9440/default

        Three URL schemes are supported:
            clickhouse:// creates a normal TCP socket connection
            clickhouses:// creates a SSL wrapped TCP socket connection

        Any additional querystring arguments will be passed along to
        the Connection class's initializer.
        """
        url = urlparse(url)

        settings = {}
        kwargs = {}

        host = url.hostname

        if url.port is not None:
            kwargs['port'] = url.port

        path = url.path.replace('/', '', 1)
        if path:
            kwargs['database'] = path

        if url.username is not None:
            kwargs['user'] = unquote(url.username)

        if url.password is not None:
            kwargs['password'] = unquote(url.password)

        if url.scheme == 'clickhouses':
            kwargs['secure'] = True

        compression_algs = {'lz4', 'lz4hc', 'zstd'}
        timeouts = {
            'connect_timeout',
            'send_receive_timeout',
            'sync_request_timeout'
        }

        for name, value in parse_qs(url.query).items():
            if not value or not len(value):
                continue

            value = value[0]

            if name == 'compression':
                value = value.lower()
                if value in compression_algs:
                    kwargs[name] = value
                else:
                    kwargs[name] = asbool(value)

            elif name == 'secure':
                kwargs[name] = asbool(value)

            elif name == 'use_numpy':
                settings[name] = asbool(value)

            elif name == 'round_robin':
                kwargs[name] = asbool(value)

            elif name == 'client_name':
                kwargs[name] = value

            elif name in timeouts:
                kwargs[name] = float(value)

            elif name == 'compress_block_size':
                kwargs[name] = int(value)

            elif name == 'settings_is_important':
                kwargs[name] = asbool(value)

            # ssl
            elif name == 'verify':
                kwargs[name] = asbool(value)
            elif name == 'ssl_version':
                kwargs[name] = getattr(ssl, value)
            elif name in ['ca_certs', 'ciphers', 'keyfile', 'certfile',
                          'server_hostname']:
                kwargs[name] = value
            elif name == 'alt_hosts':
                kwargs['alt_hosts'] = value
            else:
                settings[name] = value

        if settings:
            kwargs['settings'] = settings

        return cls(host, **kwargs)