Skip to content

Commit

Permalink
Merge pull request #64 from ClickHouse/custom_settings
Browse files Browse the repository at this point in the history
Allow adding SETTINGS section through the model
  • Loading branch information
guykoh authored Jun 18, 2022
2 parents c2a6fe3 + 338ce88 commit 702d02f
Show file tree
Hide file tree
Showing 8 changed files with 30 additions and 8 deletions.
2 changes: 1 addition & 1 deletion dbt/adapters/clickhouse/__version__.py
Original file line number Diff line number Diff line change
@@ -1 +1 @@
version = '1.1.0.1'
version = '1.1.0.2'
7 changes: 6 additions & 1 deletion dbt/adapters/clickhouse/connections.py
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@
import uuid
from contextlib import contextmanager
from dataclasses import dataclass
from typing import Any, Optional, Tuple
from typing import Any, Dict, Optional, Tuple

import agate
import clickhouse_connect
Expand Down Expand Up @@ -40,6 +40,7 @@ class ClickhouseCredentials(Credentials):
use_default_schema: bool = (
False # This is used in tests to make sure we connect always to the default database.
)
custom_settings: Optional[Dict[str, Any]] = None

@property
def type(self):
Expand Down Expand Up @@ -103,6 +104,9 @@ def open(cls, connection):
credentials = cls.get_credentials(connection.credentials)

try:
custom_settings = (
dict() if credentials.custom_settings is None else credentials.custom_settings
)
handle = clickhouse_connect.get_client(
host=credentials.host,
port=credentials.port,
Expand All @@ -116,6 +120,7 @@ def open(cls, connection):
verify=credentials.verify,
query_limit=0,
session_id='dbt::' + str(uuid.uuid4()),
**custom_settings,
)
connection.handle = handle
connection.state = 'open'
Expand Down
8 changes: 8 additions & 0 deletions dbt/adapters/clickhouse/impl.py
Original file line number Diff line number Diff line change
Expand Up @@ -253,6 +253,14 @@ def run_sql_for_tests(self, sql, fetch, conn):
finally:
conn.state = 'close'

@available
def get_model_settings(self, model):
settings = model['config'].get('settings', dict())
res = []
for key in settings:
res.append(f' {key}={settings[key]}')
return '' if len(res) == 0 else 'SETTINGS ' + ', '.join(res) + '\n'


def _expect_row_value(key: str, row: agate.Row):
if key not in row.keys():
Expand Down
1 change: 1 addition & 0 deletions dbt/include/clickhouse/macros/adapters.sql
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,7 @@
as (
{{ sql }}
)
{{ adapter.get_model_settings(model) }}
{%- endmacro %}

{% macro clickhouse__create_view_as(relation, sql) -%}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,9 @@
{% else %}
{% set old_relation = existing_relation.incorporate(path={"identifier": old_identifier}) %}
-- Create a table with only updated rows.
{% do run_query(create_table_as(False, tmp_relation, sql)) %}
{% call statement('creat_temp_table_statement') %}
{{ create_table_as(False, tmp_relation, sql) }}
{% endcall %}
{% if on_schema_change != 'ignore' %}
-- Update schema types if necessary.
{% do adapter.expand_target_column_types(
Expand Down Expand Up @@ -116,6 +118,7 @@

{% macro clickhouse__incremental_create(old_relation, target_relation) %}
create table {{ target_relation }} as {{ old_relation }}

{%- endmacro %}

{% macro clickhouse__incremental_cur_insert(old_relation, tmp_relation, target_relation, unique_key=none) %}
Expand All @@ -129,6 +132,7 @@
select ({{ unique_key }})
from {{ tmp_relation }}
)
{{ adapter.get_model_settings(model) }}
{%- endmacro %}

{% macro clickhouse__incremental_insert_from_table(tmp_relation, target_relation) %}
Expand All @@ -138,6 +142,7 @@
insert into {{ target_relation }} ({{ dest_cols_csv }})
select {{ dest_cols_csv }}
from {{ tmp_relation }}
{{ adapter.get_model_settings(model) }}
{%- endmacro %}

{% macro clickhouse__incremental_insert(target_relation, sql) %}
Expand All @@ -146,6 +151,7 @@

insert into {{ target_relation }} ({{ dest_cols_csv }})
{{ sql }}
{{ adapter.get_model_settings(model) }}
{%- endmacro %}

{% macro incremental_validate_on_schema_change(on_schema_change, default='ignore') %}
Expand Down
6 changes: 4 additions & 2 deletions dbt/include/clickhouse/macros/materializations/seed.sql
Original file line number Diff line number Diff line change
Expand Up @@ -3,8 +3,10 @@
{% set data_sql = adapter.get_csv_data(agate_table) %}

{% set sql -%}
insert into {{ this.render() }} ({{ cols_sql }}) format CSV
{{ data_sql }}
insert into {{ this.render() }} ({{ cols_sql }})
{{ adapter.get_model_settings(model) }}
format CSV
{{ data_sql }}
{%- endset %}

{% do adapter.add_query(sql, bindings=agate_table, abridge_sql_log=True) %}
Expand Down
2 changes: 1 addition & 1 deletion dev_requirements.txt
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
dbt-core==1.1.0
clickhouse-connect>=0.1.2
clickhouse-connect>=0.1.4
pytest==7.0.0
pytest-dotenv==0.5.2
dbt-tests-adapter==1.1
Expand Down
4 changes: 2 additions & 2 deletions setup.py
Original file line number Diff line number Diff line change
Expand Up @@ -53,8 +53,8 @@ def _dbt_clickhouse_version():
]
},
install_requires=[
f'dbt-core=={dbt_version}',
'clickhouse-connect>=0.1.2',
f'dbt-core~={dbt_version}',
'clickhouse-connect>=0.1.4',
],
python_requires=">=3.7",
platforms='any',
Expand Down

0 comments on commit 702d02f

Please sign in to comment.