Skip to content

Commit 08bbbf9

Browse files
authored
Release 1 6 1 (#217)
* Identifier quoting checkpoint * Identifier quoting checkpoint * Fix distributed table local quoting * Fix issues with deduplication settings
1 parent 246a4d8 commit 08bbbf9

File tree

15 files changed

+82
-59
lines changed

15 files changed

+82
-59
lines changed

CHANGELOG.md

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1,3 +1,15 @@
1+
### Release [1.6.1], 2023-12-04
2+
#### Bug Fixes
3+
- Identifier quoting was disabled for tables/databases etc. This would cause failures for schemas or tables using reserved words
4+
or containing special characters. This has been fixed and some macros have been updated to correctly handle such identifiers.
5+
Note that there still may be untested edge cases where nonstandard identifiers cause issues, so they are still not recommended.
6+
Closes https://github.com/ClickHouse/dbt-clickhouse/issues/144. Thanks to [Alexandru Pisarenco](https://github.com/apisarenco) for the
7+
report and initial PR!
8+
- The new `allow_automatic_deduplication` setting was not being correctly propagated to the adapter, so setting it to `True`
9+
did not have the intended affect. In addition, this setting is now ignored for older ClickHouse versions that
10+
do not support `CREATE TABLE AS SELECT ... EMPTY`, since the automatic deduplication window is required to allow correct
11+
inserts in Replicated tables on those older versions. Fixes https://github.com/ClickHouse/dbt-clickhouse/issues/216.
12+
113
### Release [1.6.0], 2023-11-30
214
#### Improvements
315
- Compatible with dbt 1.6.x. Note that dbt new `clone` feature is not supported, as ClickHouse has no native "light weight"
Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1 +1 @@
1-
version = '1.6.0'
1+
version = '1.6.1'

dbt/adapters/clickhouse/credentials.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -33,7 +33,7 @@ class ClickHouseCredentials(Credentials):
3333
custom_settings: Optional[Dict[str, Any]] = None
3434
use_lw_deletes: bool = False
3535
local_suffix: str = 'local'
36-
allow_automatic_deduplication = False
36+
allow_automatic_deduplication: bool = False
3737

3838
@property
3939
def type(self):

dbt/adapters/clickhouse/dbclient.py

Lines changed: 9 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -6,6 +6,8 @@
66

77
from dbt.adapters.clickhouse.credentials import ClickHouseCredentials
88
from dbt.adapters.clickhouse.logger import logger
9+
from dbt.adapters.clickhouse.query import quote_identifier
10+
from dbt.adapters.clickhouse.util import compare_versions
911

1012
LW_DELETE_SETTING = 'allow_experimental_lightweight_delete'
1113
ND_MUTATION_SETTING = 'allow_nondeterministic_mutations'
@@ -82,7 +84,10 @@ def __init__(self, credentials: ClickHouseCredentials):
8284
self.close()
8385
raise ex
8486
self._model_settings = {}
85-
if not credentials.allow_automatic_deduplication:
87+
if (
88+
not credentials.allow_automatic_deduplication
89+
and compare_versions(self._server_version(), '22.7.1.2484') >= 0
90+
):
8691
self._model_settings[DEDUP_WINDOW_SETTING] = '0'
8792

8893
@abstractmethod
@@ -159,7 +164,7 @@ def _check_lightweight_deletes(self, requested: bool):
159164
def _ensure_database(self, database_engine, cluster_name) -> None:
160165
if not self.database:
161166
return
162-
check_db = f'EXISTS DATABASE {self.database}'
167+
check_db = f'EXISTS DATABASE {quote_identifier(self.database)}'
163168
try:
164169
db_exists = self.command(check_db)
165170
if not db_exists:
@@ -170,7 +175,7 @@ def _ensure_database(self, database_engine, cluster_name) -> None:
170175
else ''
171176
)
172177
self.command(
173-
f'CREATE DATABASE IF NOT EXISTS {self.database}{cluster_clause}{engine_clause}'
178+
f'CREATE DATABASE IF NOT EXISTS {quote_identifier(self.database)}{cluster_clause}{engine_clause}'
174179
)
175180
db_exists = self.command(check_db)
176181
if not db_exists:
@@ -194,7 +199,7 @@ def _check_atomic_exchange(self) -> bool:
194199
table_id = str(uuid.uuid1()).replace('-', '')
195200
swap_tables = [f'__dbt_exchange_test_{x}_{table_id}' for x in range(0, 2)]
196201
for table in swap_tables:
197-
self.command(create_cmd.format(table))
202+
self.command(create_cmd.format(quote_identifier(table)))
198203
try:
199204
self.command('EXCHANGE TABLES {} AND {}'.format(*swap_tables))
200205
return True

dbt/adapters/clickhouse/impl.py

Lines changed: 1 addition & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@
2323
from dbt.adapters.clickhouse.logger import logger
2424
from dbt.adapters.clickhouse.query import quote_identifier
2525
from dbt.adapters.clickhouse.relation import ClickHouseRelation
26+
from dbt.adapters.clickhouse.util import compare_versions
2627

2728
GET_CATALOG_MACRO_NAME = 'get_catalog'
2829
LIST_SCHEMAS_MACRO_NAME = 'list_schemas'
@@ -438,18 +439,6 @@ def test(row: agate.Row) -> bool:
438439
return test
439440

440441

441-
def compare_versions(v1: str, v2: str) -> int:
442-
v1_parts = v1.split('.')
443-
v2_parts = v2.split('.')
444-
for part1, part2 in zip(v1_parts, v2_parts):
445-
try:
446-
if int(part1) != int(part2):
447-
return 1 if int(part1) > int(part2) else -1
448-
except ValueError:
449-
raise DbtRuntimeError("Version must consist of only numbers separated by '.'")
450-
return 0
451-
452-
453442
COLUMNS_EQUAL_SQL = '''
454443
SELECT
455444
row_count_diff.difference as row_count_difference,

dbt/adapters/clickhouse/relation.py

Lines changed: 14 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -3,16 +3,18 @@
33

44
from dbt.adapters.base.relation import BaseRelation, Policy, Self
55
from dbt.contracts.graph.nodes import ManifestNode, SourceDefinition
6-
from dbt.contracts.relation import HasQuoting
6+
from dbt.contracts.relation import HasQuoting, Path, RelationType
77
from dbt.exceptions import DbtRuntimeError
88
from dbt.utils import deep_merge, merge
99

10+
from dbt.adapters.clickhouse.query import quote_identifier
11+
1012

1113
@dataclass
1214
class ClickHouseQuotePolicy(Policy):
13-
database: bool = False
14-
schema: bool = False
15-
identifier: bool = False
15+
database: bool = True
16+
schema: bool = True
17+
identifier: bool = True
1618

1719

1820
@dataclass
@@ -26,7 +28,7 @@ class ClickHouseIncludePolicy(Policy):
2628
class ClickHouseRelation(BaseRelation):
2729
quote_policy: Policy = field(default_factory=lambda: ClickHouseQuotePolicy())
2830
include_policy: Policy = field(default_factory=lambda: ClickHouseIncludePolicy())
29-
quote_character: str = ''
31+
quote_character: str = '`'
3032
can_exchange: bool = False
3133
can_on_cluster: bool = False
3234

@@ -35,13 +37,13 @@ def __post_init__(self):
3537
raise DbtRuntimeError(f'Cannot set database {self.database} in clickhouse!')
3638
self.path.database = ''
3739

38-
def render(self):
39-
if self.include_policy.database and self.include_policy.schema:
40-
raise DbtRuntimeError(
41-
'Got a clickhouse relation with schema and database set to '
42-
'include, but only one can be set'
43-
)
44-
return super().render()
40+
def render(self) -> str:
41+
return ".".join(quote_identifier(part) for _, part in self._render_iterator() if part)
42+
43+
def derivative(self, suffix: str, relation_type: Optional[str] = None) -> BaseRelation:
44+
path = Path(schema=self.path.schema, database='', identifier=self.path.identifier + suffix)
45+
derivative_type = RelationType[relation_type] if relation_type else self.type
46+
return ClickHouseRelation(type=derivative_type, path=path)
4547

4648
def matches(
4749
self,

dbt/adapters/clickhouse/util.py

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,13 @@
1+
from dbt.exceptions import DbtRuntimeError
2+
3+
4+
def compare_versions(v1: str, v2: str) -> int:
5+
v1_parts = v1.split('.')
6+
v2_parts = v2.split('.')
7+
for part1, part2 in zip(v1_parts, v2_parts):
8+
try:
9+
if int(part1) != int(part2):
10+
return 1 if int(part1) > int(part2) else -1
11+
except ValueError:
12+
raise DbtRuntimeError("Version must consist of only numbers separated by '.'")
13+
return 0

dbt/include/clickhouse/macros/materializations/incremental/incremental.sql

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -218,7 +218,8 @@
218218

219219
{% call statement('delete_existing_data') %}
220220
{% if is_distributed %}
221-
delete from {{ existing_relation }}{{ adapter.get_clickhouse_local_suffix() }} {{ on_cluster_clause(existing_relation) }} where ({{ unique_key }}) in (select {{ unique_key }}
221+
{%- set existing_local = existing_relation.derivative(adapter.get_clickhouse_local_suffix()) %}
222+
delete from {{ existing_local }} {{ on_cluster_clause(existing_relation) }} where ({{ unique_key }}) in (select {{ unique_key }}
222223
from {{ inserting_relation }})
223224
{% else %}
224225
delete from {{ existing_relation }} where ({{ unique_key }}) in (select {{ unique_key }}

dbt/include/clickhouse/macros/materializations/materialized_view.sql

Lines changed: 11 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -6,8 +6,7 @@
66
{%- materialization materialized_view, adapter='clickhouse' -%}
77

88
{%- set target_relation = this.incorporate(type='table') -%}
9-
{%- set mv_name = target_relation.name + '_mv' -%}
10-
{%- set target_mv = api.Relation.create(identifier=mv_name, schema=schema, database=database, type='materialized_view') -%}
9+
{%- set mv_relation = target_relation.derivative('_mv', 'MaterializedView') -%}
1110
{%- set cluster_clause = on_cluster_clause(target_relation) -%}
1211

1312
{# look for an existing relation for the target table and create backup relations if necessary #}
@@ -44,14 +43,14 @@
4443
{% elif existing_relation.can_exchange %}
4544
{{ log('Replacing existing materialized view' + target_relation.name) }}
4645
{% call statement('drop existing materialized view') %}
47-
drop view if exists {{ mv_name }} {{ cluster_clause }}
46+
drop view if exists {{ mv_relation }} {{ cluster_clause }}
4847
{% endcall %}
4948
{% call statement('main') -%}
5049
{{ get_create_table_as_sql(False, backup_relation, sql) }}
5150
{%- endcall %}
5251
{% do exchange_tables_atomic(backup_relation, existing_relation) %}
5352
{% call statement('create new materialized view') %}
54-
{{ clickhouse__create_mv_sql(mv_name, existing_relation.name, cluster_clause, sql) }}
53+
{{ clickhouse__create_mv_sql(mv_relation, existing_relation.name, cluster_clause, sql) }}
5554
{% endcall %}
5655
{% else %}
5756
{{ log('Replacing existing materialized view' + target_relation.name) }}
@@ -72,7 +71,7 @@
7271

7372
{{ run_hooks(post_hooks, inside_transaction=False) }}
7473

75-
{{ return({'relations': [target_relation, target_mv]}) }}
74+
{{ return({'relations': [target_relation, mv_relation]}) }}
7675

7776
{%- endmaterialization -%}
7877

@@ -88,13 +87,13 @@
8887
{{ get_create_table_as_sql(False, relation, sql) }}
8988
{% endcall %}
9089
{%- set cluster_clause = on_cluster_clause(relation) -%}
91-
{%- set mv_name = relation.name + '_mv' -%}
92-
{{ clickhouse__create_mv_sql(mv_name, relation.name, cluster_clause, sql) }}
90+
{%- set mv_relation = relation.derivative('_mv', 'MaterializedView') -%}
91+
{{ clickhouse__create_mv_sql(mv_relation, relation, cluster_clause, sql) }}
9392
{%- endmacro %}
9493

9594

96-
{% macro clickhouse__create_mv_sql(relation_name, target_table, cluster_clause, sql) -%}
97-
create materialized view if not exists {{ relation_name }} {{ cluster_clause }}
95+
{% macro clickhouse__create_mv_sql(mv_relation, target_table, cluster_clause, sql) -%}
96+
create materialized view if not exists {{ mv_relation }} {{ cluster_clause }}
9897
to {{ target_table }}
9998
as {{ sql }}
10099
{%- endmacro %}
@@ -103,9 +102,9 @@
103102
{% macro clickhouse__replace_mv(target_relation, existing_relation, intermediate_relation, backup_relation, sql) %}
104103
{# drop existing materialized view while we recreate the target table #}
105104
{%- set cluster_clause = on_cluster_clause(target_relation) -%}
106-
{%- set mv_name = target_relation.name + '_mv' -%}
105+
{%- set mv_relation = target_relation.derivative('_mv', 'MaterializedView') -%}
107106
{% call statement('drop existing mv') -%}
108-
drop view if exists {{ mv_name }} {{ cluster_clause }}
107+
drop view if exists {{ mv_relation }} {{ cluster_clause }}
109108
{%- endcall %}
110109

111110
{# recreate the target table #}
@@ -116,5 +115,5 @@
116115
{{ adapter.rename_relation(intermediate_relation, target_relation) }}
117116

118117
{# now that the target table is recreated, we can finally create our new view #}
119-
{{ clickhouse__create_mv_sql(mv_name, target_relation.name, cluster_clause, sql) }}
118+
{{ clickhouse__create_mv_sql(mv_relation, target_relation, cluster_clause, sql) }}
120119
{% endmacro %}

dbt/include/clickhouse/macros/materializations/snapshot.sql

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,7 @@
2525
{%- set insert_cols_csv = insert_cols | join(', ') -%}
2626
{%- set valid_to_col = adapter.quote('dbt_valid_to') -%}
2727

28-
{%- set upsert = target ~ '__snapshot_upsert' -%}
28+
{%- set upsert = target.derivative('__snapshot_upsert') -%}
2929
{% call statement('create_upsert_relation') %}
3030
create table if not exists {{ upsert }} as {{ target }}
3131
{% endcall %}

dbt/include/clickhouse/macros/materializations/table.sql

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -141,7 +141,7 @@
141141
{% call statement('create_table_empty') %}
142142
{{ create_table }}
143143
{% endcall %}
144-
{{ clickhouse__insert_into(relation.include(database=False), sql, has_contract) }}
144+
{{ clickhouse__insert_into(relation, sql, has_contract) }}
145145
{%- endif %}
146146
{%- endmacro %}
147147

@@ -151,7 +151,7 @@
151151
{{ sql_header if sql_header is not none }}
152152

153153
{% if temporary -%}
154-
create temporary table {{ relation.name }}
154+
create temporary table {{ relation }}
155155
engine Memory
156156
{{ order_cols(label="order by") }}
157157
{{ partition_cols(label="partition by") }}
@@ -160,7 +160,7 @@
160160
{{ sql }}
161161
)
162162
{%- else %}
163-
create table {{ relation.include(database=False) }}
163+
create table {{ relation }}
164164
{{ on_cluster_clause(relation)}}
165165
{%- if has_contract%}
166166
{{ get_assert_columns_equivalent(sql) }}

tests/integration/adapter/aliases/test_aliases.py

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -83,17 +83,17 @@ def test_alias_model_name(self, project):
8383
assert len(results) == 4
8484

8585
cluster = project.test_config['cluster']
86-
relation = relation_from_name(project.adapter, "foo")
86+
local_relation = relation_from_name(project.adapter, "foo_local")
8787

8888
result = project.run_sql(
89-
f"select max(tablename) AS tablename From clusterAllReplicas('{cluster}', {relation}_local) ",
89+
f"select max(tablename) AS tablename From clusterAllReplicas('{cluster}', {local_relation}) ",
9090
fetch="one",
9191
)
9292
assert result[0] == "foo"
9393

94-
relation = relation_from_name(project.adapter, "ref_foo_alias")
94+
local_relation = relation_from_name(project.adapter, "ref_foo_alias_local")
9595
result = project.run_sql(
96-
f"select max(tablename) AS tablename From clusterAllReplicas('{cluster}', {relation}_local) ",
96+
f"select max(tablename) AS tablename From clusterAllReplicas('{cluster}', {local_relation}) ",
9797
fetch="one",
9898
)
9999
assert result[0] == "ref_foo_alias"

tests/integration/adapter/clickhouse/test_clickhouse_table_materializations.py

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -77,23 +77,23 @@ def seeds(self):
7777
}
7878

7979
def assert_total_count_correct(self, project):
80-
'''Check if data is properly distributed'''
80+
# Check if data is properly distributed
8181
cluster = project.test_config['cluster']
82-
table_relation = relation_from_name(project.adapter, "distributed")
82+
table_relation = relation_from_name(project.adapter, "distributed_local")
8383
cluster_info = project.run_sql(
8484
f"select shard_num,max(host_name) as host_name, count(distinct replica_num) as replica_counts "
8585
f"from system.clusters where cluster='{cluster}' group by shard_num",
8686
fetch="all",
8787
)
8888
sum_count = project.run_sql(
89-
f"select count() From clusterAllReplicas('{cluster}',{table_relation}_local)",
89+
f"select count() From clusterAllReplicas('{cluster}',{table_relation})",
9090
fetch="one",
9191
)
9292
total_count = 0
9393
# total count should be equal to sum(count of each shard * replica_counts)
9494
for shard_num, host_name, replica_counts in cluster_info:
9595
count = project.run_sql(
96-
f"select count() From remote('{host_name}',{table_relation}_local)",
96+
f"select count() From remote('{host_name}',{table_relation})",
9797
fetch="one",
9898
)
9999
total_count += count[0] * replica_counts
@@ -103,7 +103,7 @@ def assert_total_count_correct(self, project):
103103
os.environ.get('DBT_CH_TEST_CLUSTER', '').strip() == '', reason='Not on a cluster'
104104
)
105105
def test_base(self, project):
106-
# cluster setting must exists
106+
# cluster setting must exist
107107
cluster = project.test_config['cluster']
108108
assert cluster
109109

tests/integration/adapter/materialized_view/test_materialized_view.py

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -8,6 +8,8 @@
88
import pytest
99
from dbt.tests.util import check_relation_types, run_dbt
1010

11+
from dbt.adapters.clickhouse.query import quote_identifier
12+
1113
PEOPLE_SEED_CSV = """
1214
id,name,age,department
1315
1231,Dade,33,engineering
@@ -116,7 +118,7 @@ def test_create(self, project):
116118
# insert some data and make sure it reaches the target table
117119
project.run_sql(
118120
f"""
119-
insert into {project.test_schema}.people ("id", "name", "age", "department")
121+
insert into {quote_identifier(project.test_schema)}.people ("id", "name", "age", "department")
120122
values (1232,'Dade',16,'engineering'), (9999,'eugene',40,'malware');
121123
"""
122124
)
@@ -153,7 +155,7 @@ def test_update(self, project):
153155

154156
project.run_sql(
155157
f"""
156-
insert into {project.test_schema}.people ("id", "name", "age", "department")
158+
insert into {quote_identifier(project.test_schema)}.people ("id", "name", "age", "department")
157159
values (1232,'Dade',11,'engineering'), (9999,'eugene',40,'malware');
158160
"""
159161
)

tests/unit/test_adapter.py renamed to tests/unit/test_util.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,4 @@
1-
from dbt.adapters.clickhouse.impl import compare_versions
1+
from dbt.adapters.clickhouse.util import compare_versions
22

33

44
def test_is_before_version():

0 commit comments

Comments
 (0)