Skip to content

Commit

Permalink
feat: add TTL support (#254)
Browse files Browse the repository at this point in the history
  • Loading branch information
Ian2012 authored Apr 8, 2024
1 parent 0e4ef70 commit 1c9a15d
Show file tree
Hide file tree
Showing 4 changed files with 72 additions and 0 deletions.
2 changes: 2 additions & 0 deletions README.md
Original file line number Diff line number Diff line change
Expand Up @@ -102,6 +102,7 @@ your_profile_name:
| incremental_predicates | Additional conditions to be applied to the incremental materialization (only applied to `delete+insert` strategy | |
| settings | A map/dictionary of "TABLE" settings to be used to DDL statements like 'CREATE TABLE' with this model | |
| query_settings | A map/dictionary of ClickHouse user level settings to be used with `INSERT` or `DELETE` statements in conjunction with this model | |
| ttl | A TTL expression to be used with the table. The TTL expression is a string that can be used to specify the TTL for the table. | |

## ClickHouse Cluster

Expand Down Expand Up @@ -179,6 +180,7 @@ The following macros are included to facilitate creating ClickHouse specific tab
- `order_cols` -- Uses the `order_by` model configuration to assign a ClickHouse order by/sorting key. If not specified ClickHouse will use an empty tuple() and the table will be unsorted
- `primary_key_clause` -- Uses the `primary_key` model configuration property to assign a ClickHouse primary key. By default, primary key is set and ClickHouse will use the order by clause as the primary key.
- `on_cluster_clause` -- Uses the `cluster` profile property to add an `ON CLUSTER` clause to certain dbt-operations: distributed materializations, views creation, database creation.
- `ttl_config` -- Uses the `ttl` model configuration property to assign a ClickHouse table TTL expression. No TTL is assigned by default.

### s3Source Helper Macro

Expand Down
1 change: 1 addition & 0 deletions dbt/adapters/clickhouse/impl.py
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@ class ClickHouseConfig(AdapterConfig):
order_by: Optional[Union[List[str], str]] = 'tuple()'
partition_by: Optional[Union[List[str], str]] = None
sharding_key: Optional[Union[List[str], str]] = 'rand()'
ttl: Optional[Union[List[str], str]] = None


class ClickHouseAdapter(SQLAdapter):
Expand Down
7 changes: 7 additions & 0 deletions dbt/include/clickhouse/macros/materializations/table.sql
Original file line number Diff line number Diff line change
Expand Up @@ -121,6 +121,12 @@
{%- endif %}
{%- endmacro -%}

{% macro ttl_config(label) %}
{%- if config.get("ttl")%}
{{ label }} {{ config.get("ttl") }}
{%- endif %}
{%- endmacro -%}

{% macro on_cluster_clause(relation, force_sync) %}
{% set active_cluster = adapter.get_clickhouse_cluster_name() %}
{%- if active_cluster is not none and relation.should_on_cluster %}
Expand Down Expand Up @@ -170,6 +176,7 @@
{{ order_cols(label="order by") }}
{{ primary_key_clause(label="primary key") }}
{{ partition_cols(label="partition by") }}
{{ ttl_config(label="ttl")}}
{{ adapter.get_model_settings(model) }}

{%- if not has_contract %}
Expand Down
62 changes: 62 additions & 0 deletions tests/integration/adapter/clickhouse/test_clickhouse_table_ttl.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
import time
from datetime import datetime

import pytest
from dbt.tests.adapter.basic.files import model_base, schema_base_yml
from dbt.tests.adapter.basic.test_base import BaseSimpleMaterializations
from dbt.tests.util import relation_from_name, run_dbt


class TestTableTTL(BaseSimpleMaterializations):
@pytest.fixture(scope="class")
def models(self):
config_materialized_table = """
{{ config(
order_by='(some_date, id, name)',
engine='MergeTree()',
materialized='table',
settings={'allow_nullable_key': 1},
ttl='some_date + INTERVAL 5 SECONDS',
query_settings={'allow_nondeterministic_mutations': 1})
}}
"""
base_table_sql = config_materialized_table + model_base
return {
"table_model.sql": base_table_sql,
"schema.yml": schema_base_yml,
}

def test_base(self, project):
# seed command
results = run_dbt(["seed"])
# seed result length
assert len(results) == 1

# run command
results = run_dbt()
# run result length
assert len(results) == 1

# base table rowcount
relation = relation_from_name(project.adapter, "table_model")
result = project.run_sql(f"select count(*) as num_rows from {relation}", fetch="one")
# the dates from the seed are too old, so those are expired
assert result[0] == 0

# insert new data
now = datetime.now().strftime("%Y-%m-%d %H:%M:%S")
project.run_sql(f"insert into {relation} (*) values (11, 'Elian', '{now}')")

result = project.run_sql(f"select count(*) as num_rows from {relation}", fetch="one")
# the dates from the seed are too old, so those are expired
assert result[0] == 1

# wait for TTL to expire
time.sleep(6)

# optimize table
project.run_sql(f"OPTIMIZE TABLE {relation} FINAL")

# make sure is empty
result = project.run_sql(f"select count(*) as num_rows from {relation}", fetch="one")
assert result[0] == 0

0 comments on commit 1c9a15d

Please sign in to comment.