Skip to content

Commit

Permalink
Checkpoint for constraints tests
Browse files Browse the repository at this point in the history
  • Loading branch information
genzgd committed Nov 21, 2023
1 parent 9d776d9 commit b025901
Show file tree
Hide file tree
Showing 7 changed files with 178 additions and 30 deletions.
3 changes: 2 additions & 1 deletion CHANGELOG.md
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
### Release [1.5.0], TBD
#### Improvement
#### Improvements
- Compatible with dbt 1.5.x
- Contract support (using exact column data types)

#### Bug Fix
- Fix s3 macro when bucket includes `https://` prefix. Closes https://github.com/ClickHouse/dbt-clickhouse/issues/192.
Expand Down
7 changes: 7 additions & 0 deletions README.md
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ pip install dbt-clickhouse
- [x] Ephemeral materialization
- [x] Distributed table materialization (experimental)
- [x] Distributed incremental materialization (experimental)
- [x] Contracts

# Usage Notes

Expand Down Expand Up @@ -185,6 +186,12 @@ keys used to populate the parameters of the S3 table function:

See the [S3 test file](https://github.com/ClickHouse/dbt-clickhouse/blob/main/tests/integration/adapter/test_s3.py) for examples of how to use this macro.

# Contracts and Constraints

Only exact column type contracts are supported. For example, a contract with a UInt32 column type will fail if the model returns a UInt64 or other integer type.
ClickHouse also support _only_ `CHECK` constraints on the entire table/model. Primary key, foreign key, unique, and column level CHECK constraints are not supported.
(See ClickHouse documentation on primary/order by keys.)

# Distributed materializations

Notes:
Expand Down
11 changes: 0 additions & 11 deletions dbt/include/clickhouse/macros/adapters.sql
Original file line number Diff line number Diff line change
@@ -1,14 +1,3 @@
{% macro clickhouse__create_view_as(relation, sql) -%}
{%- set sql_header = config.get('sql_header', none) -%}

{{ sql_header if sql_header is not none }}

create view {{ relation.include(database=False) }} {{ on_cluster_clause(relation)}}
as (
{{ sql }}
)
{%- endmacro %}

{% macro clickhouse__list_schemas(database) %}
{% call statement('list_schemas', fetch_result=True, auto_begin=False) %}
select name from system.databases
Expand Down
25 changes: 14 additions & 11 deletions dbt/include/clickhouse/macros/materializations/table.sql
Original file line number Diff line number Diff line change
Expand Up @@ -130,25 +130,23 @@
{%- endmacro -%}

{% macro clickhouse__create_table_as(temporary, relation, sql) -%}
{% set create_table = create_table_or_empty(temporary, relation, sql) %}
{% set has_contract = config.get('contract').enforced %}
{% set create_table = create_table_or_empty(temporary, relation, sql, has_contract) %}
{% if adapter.is_before_version('22.7.1.2484') -%}
{{ create_table }}
{%- else %}
{% call statement('create_table_empty') %}
{{ create_table }}
{% endcall %}
{{ clickhouse__insert_into(relation.include(database=False), sql) }}
{{ clickhouse__insert_into(relation.include(database=False), sql, has_contract) }}
{%- endif %}
{%- endmacro %}

{% macro create_table_or_empty(temporary, relation, sql) -%}
{% macro create_table_or_empty(temporary, relation, sql, has_contract) -%}
{%- set sql_header = config.get('sql_header', none) -%}
{%- set contract_config = config.get('contract') -%}

{{ sql_header if sql_header is not none }}

{{ log('ENFORCED', info=True) }}

{% if temporary -%}
create temporary table {{ relation.name }}
engine Memory
Expand All @@ -159,8 +157,8 @@
{%- else %}
create table {{ relation.include(database=False) }}
{{ on_cluster_clause(relation)}}
{%- if contract_config.enforced %}
{{ clickhouse__get_assert_columns_equivalent(sql) }}
{%- if has_contract%}
{{ get_assert_columns_equivalent(sql) }}
{{ get_table_columns_and_constraints() }}
{%- endif %}
{{ engine_clause() }}
Expand All @@ -169,7 +167,7 @@
{{ partition_cols(label="partition by") }}
{{ adapter.get_model_settings(model) }}

{%- if not contract_config.enforced %}
{%- if not has_contract %}
{%- if not adapter.is_before_version('22.7.1.2484') %}
empty
{%- endif %}
Expand All @@ -179,10 +177,15 @@

{%- endmacro %}

{% macro clickhouse__insert_into(target_relation, sql) %}
{% macro clickhouse__insert_into(target_relation, sql, has_contract) %}
{%- set dest_columns = adapter.get_columns_in_relation(target_relation) -%}
{%- set dest_cols_csv = dest_columns | map(attribute='quoted') | join(', ') -%}

insert into {{ target_relation }} ({{ dest_cols_csv }})
{{ sql }}
{%- if has_contract -%}
-- Use a subquery to get columns in the right order
SELECT {{ dest_cols_csv }} FROM ( {{ sql }} )
{%- else -%}
{{ sql }}
{%- endif -%}
{%- endmacro %}
16 changes: 16 additions & 0 deletions dbt/include/clickhouse/macros/materializations/view.sql
Original file line number Diff line number Diff line change
Expand Up @@ -65,3 +65,19 @@
{{ return({'relations': [target_relation]}) }}

{%- endmaterialization -%}


{% macro clickhouse__create_view_as(relation, sql) -%}
{%- set sql_header = config.get('sql_header', none) -%}
{{ sql_header if sql_header is not none }}

create view {{ relation.include(database=False) }} {{ on_cluster_clause(relation)}}
{% set contract_config = config.get('contract') %}
{% if contract_config.enforced %}
{{ get_assert_columns_equivalent(sql) }}
{%- endif %}
as (
{{ sql }}
)
{%- endmacro %}

Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@
enforced: true
columns:
- name: id
data_type: Int32
data_type: UInt32
description: hello
tests:
- unique
Expand Down Expand Up @@ -66,8 +66,8 @@
select
'blue' as color,
1 as id,
'2019-01-01' as date_day
1::UInt32 as id,
toDate('2019-01-01') as date_day
"""


Expand All @@ -84,3 +84,54 @@
1 as error,
'2019-01-01' as date_day
"""


my_model_data_type_sql = """
{{{{
config(
materialized = "table"
)
}}}}
select
{sql_value} as wrong_data_type_column_name
"""


model_data_type_schema_yml = """
version: 2
models:
- name: my_model_data_type
config:
contract:
enforced: true
columns:
- name: wrong_data_type_column_name
data_type: {data_type}
"""

my_model_view_wrong_name_sql = """
{{
config(
materialized = "view"
)
}}
select
'blue' as color,
1 as error,
toDate('2019-01-01') as date_day
"""

my_model_view_wrong_order_sql = """
{{
config(
materialized = "view"
)
}}
select
'blue' as color,
1::UInt32 as id,
toDate('2019-01-01') as date_day
"""
89 changes: 85 additions & 4 deletions tests/integration/adapter/constraints/test_constraints.py
Original file line number Diff line number Diff line change
@@ -1,8 +1,11 @@
import pytest
from dbt.tests.util import get_manifest, run_dbt, run_dbt_and_capture

from tests.integration.adapter.constraints.constraint_fixtures import (
from dbt.tests.util import get_manifest, run_dbt, run_dbt_and_capture, write_file
from fixtures_contraints import (
model_data_type_schema_yml,
model_schema_yml,
my_model_data_type_sql,
my_model_view_wrong_name_sql,
my_model_view_wrong_order_sql,
my_model_wrong_name_sql,
my_model_wrong_order_sql,
)
Expand All @@ -13,7 +16,20 @@ class ClickHouseConstraintsColumnsEqual:
dbt should catch these mismatches during its "preflight" checks.
"""

def __test__constraints_wrong_column_order(self, project):
@pytest.fixture
def data_types(self):
# sql_column_value, schema_data_type, error_data_type
return [
["1::Int32", "Int32", "Int32"],
["'1'", "String", "String"],
["true", "Bool", "Bool"],
["'2013-11-03'::DateTime", "DateTime", "DateTime"],
["['a','b','c']", "Array(String)", "Array(String)"],
["[1::Int32,2::Int32,3::Int32]", "Array(Int32)", "Array(Int32)"],
["'1'::Float64", "Float64", "Float64"],
]

def test__constraints_wrong_column_order(self, project):
# This no longer causes an error, since we enforce yaml column order
run_dbt(["run", "-s", "my_model_wrong_order"], expect_pass=True)
manifest = get_manifest(project.project_root)
Expand All @@ -36,6 +52,61 @@ def test__constraints_wrong_column_names(self, project):
expected = ["id", "error", "missing in definition", "missing in contract"]
assert all([(exp in log_output or exp.upper() in log_output) for exp in expected])

def test__constraints_wrong_column_data_types(self, project, data_types):
for (sql_column_value, schema_data_type, error_data_type) in data_types:
# Write parametrized data_type to sql file
write_file(
my_model_data_type_sql.format(sql_value=sql_column_value),
"models",
"my_model_data_type.sql",
)
write_file(
model_data_type_schema_yml.format(data_type='Int128'),
"models",
"constraints_schema.yml",
)

results, log_output = run_dbt_and_capture(
["run", "-s", "my_model_data_type"], expect_pass=False
)
manifest = get_manifest(project.project_root)
model_id = "model.test.my_model_data_type"
my_model_config = manifest.nodes[model_id].config
contract_actual_config = my_model_config.contract

assert contract_actual_config.enforced is True
expected = [
"wrong_data_type_column_name",
error_data_type,
"Int128",
"data type mismatch",
]
assert all([(exp in log_output or exp.upper() in log_output) for exp in expected])

def test__constraints_correct_column_data_types(self, project, data_types):
for (sql_column_value, schema_data_type, _) in data_types:
# Write parametrized data_type to sql file
write_file(
my_model_data_type_sql.format(sql_value=sql_column_value),
"models",
"my_model_data_type.sql",
)
# Write correct data_type to corresponding schema file
write_file(
model_data_type_schema_yml.format(data_type=schema_data_type),
"models",
"constraints_schema.yml",
)

run_dbt(["run", "-s", "my_model_data_type"])

manifest = get_manifest(project.project_root)
model_id = "model.test.my_model_data_type"
my_model_config = manifest.nodes[model_id].config
contract_actual_config = my_model_config.contract

assert contract_actual_config.enforced is True


class TestTableConstraintsColumnsEqual(ClickHouseConstraintsColumnsEqual):
@pytest.fixture(scope="class")
Expand All @@ -47,6 +118,16 @@ def models(self):
}


class TestViewConstraintsColumnsEqual(ClickHouseConstraintsColumnsEqual):
@pytest.fixture(scope="class")
def models(self):
return {
"my_model_wrong_order.sql": my_model_view_wrong_order_sql,
"my_model_wrong_name.sql": my_model_view_wrong_name_sql,
"constraints_schema.yml": model_schema_yml,
}


# class TestViewConstraintsColumnsEqual(BaseViewConstraintsColumnsEqual):
# pass
#
Expand Down

0 comments on commit b025901

Please sign in to comment.