diff --git a/.github/docker/docker-compose.yml b/.github/docker/docker-compose.yml index 5a70fc7..86e38eb 100644 --- a/.github/docker/docker-compose.yml +++ b/.github/docker/docker-compose.yml @@ -5,6 +5,7 @@ services: restart: always ports: - "2136:2136" + - "8765:8765" hostname: localhost environment: - YDB_USE_IN_MEMORY_PDISKS=true diff --git a/dbt/adapters/ydb/column.py b/dbt/adapters/ydb/column.py index 1ce40ed..7d97701 100644 --- a/dbt/adapters/ydb/column.py +++ b/dbt/adapters/ydb/column.py @@ -89,8 +89,8 @@ def numeric_type(cls, dtype: str, precision: Any, scale: Any) -> str: @classmethod def nested_type(cls, dtype: str, is_nullable: bool) -> str: template = "{}" - if is_nullable: - template = template.format("Nullable({})") + if not is_nullable: + template = template.format("{} NOT NULL") return template.format(dtype) def literal(self, value): diff --git a/dbt/adapters/ydb/impl.py b/dbt/adapters/ydb/impl.py index 65c6c72..d348fca 100644 --- a/dbt/adapters/ydb/impl.py +++ b/dbt/adapters/ydb/impl.py @@ -300,7 +300,7 @@ def valid_incremental_strategies(self): @available.parse_none def get_column_schema_from_query(self, sql: str, *_) -> List[YDBColumn]: - logger.info(f"Try to get column schema from query: \n{sql}") + logger.debug(f"Try to get column schema from query: \n{sql}") connection = self.connections.get_thread_connection() dbapi_connection = connection.handle @@ -308,6 +308,39 @@ def get_column_schema_from_query(self, sql: str, *_) -> List[YDBColumn]: cur.execute(sql) return [YDBColumn(col[0], col[1]) for col in cur.description] + def timestamp_add_sql(self, add_to: str, number: int = 1, interval: str = "hour") -> str: + match interval: + case "day": + return f"{add_to} + DateTime::IntervalFromDays({number})" + case "hour": + return f"{add_to} + DateTime::IntervalFromHours({number})" + case "minute": + return f"{add_to} + DateTime::IntervalFromMinutes({number})" + case "second": + return f"{add_to} + DateTime::IntervalFromSeconds({number})" + case "millisecond": + return f"{add_to} + DateTime::IntervalFromMilliseconds({number})" + case "microsecond": + return f"{add_to} + DateTime::IntervalFromMicroseconds({number})" + case _: + raise DbtRuntimeError( + f"Unsupported value for interval: {interval}, only day, hour," + "minute, second, millisecond, microsecond are supported" + ) + + def string_add_sql( + self, + add_to: str, + value: str, + location="append", + ) -> str: + if location == "append": + return f"{add_to} || Text('{value}')" + elif location == "prepend": + return f"Text('{value}') || {add_to}" + else: + raise DbtRuntimeError(f'Got an unexpected location value of "{location}"') + COLUMNS_EQUAL_SQL = ''' diff --git a/dbt/include/ydb/macros/adapters.sql b/dbt/include/ydb/macros/adapters.sql index 3ba68c8..03cf984 100644 --- a/dbt/include/ydb/macros/adapters.sql +++ b/dbt/include/ydb/macros/adapters.sql @@ -186,6 +186,5 @@ Example 3 of 3 of required macros that does not have a default implementation. */ {% macro ydb__current_timestamp() -%} -'''Returns current UTC time''' CurrentUtcTimestamp() {% endmacro %} diff --git a/dbt/include/ydb/macros/materializations/models/table.sql b/dbt/include/ydb/macros/materializations/models/table.sql index 3b36f37..823721c 100644 --- a/dbt/include/ydb/macros/materializations/models/table.sql +++ b/dbt/include/ydb/macros/materializations/models/table.sql @@ -35,7 +35,7 @@ , TTL = {{ ttl_expr }} {%- endif -%} ) - as ( + as {{ sql }} - ) + {%- endmacro %} \ No newline at end of file diff --git a/dbt/include/ydb/macros/materializations/snapshots/helpers.sql b/dbt/include/ydb/macros/materializations/snapshots/helpers.sql new file mode 100644 index 0000000..5fd861a --- /dev/null +++ b/dbt/include/ydb/macros/materializations/snapshots/helpers.sql @@ -0,0 +1,266 @@ + +{% macro ydb__build_snapshot_table(strategy, sql) %} + {% set columns = config.get('snapshot_table_column_names') or get_snapshot_table_column_names() %} + + select `sbq`.*, + {{ strategy.scd_id }} as {{ columns.dbt_scd_id }}, + {{ strategy.updated_at }} as {{ columns.dbt_updated_at }}, + {{ strategy.updated_at }} as {{ columns.dbt_valid_from }}, + {{ ydb__get_dbt_valid_to_current(strategy, columns) }} + {%- if strategy.hard_deletes == 'new_record' -%} + , 'False' as {{ columns.dbt_is_deleted }} + {% endif -%} + from ( + {{ sql }} + ) sbq + +{% endmacro %} + +{% macro ydb__get_dbt_valid_to_current(strategy, columns) %} + {% set dbt_valid_to_current = config.get('dbt_valid_to_current') or "null" %} + coalesce(case when {{ strategy.updated_at }} = {{ strategy.updated_at }} then null else {{ strategy.updated_at }} end, {{dbt_valid_to_current}}) + as {{ columns.dbt_valid_to }} +{% endmacro %} + +{% macro ydb__build_snapshot_staging_table(strategy, sql, target_relation) %} + {% set temp_relation = make_temp_relation(target_relation) %} + + {% do drop_relation(temp_relation) %} + + {% set select = snapshot_staging_table(strategy, sql, target_relation) %} + + {% call statement('build_snapshot_staging_relation') %} + {{ ydb__create_snapshot_table_as(False, temp_relation, select) }} + {% endcall %} + + {% do return(temp_relation) %} +{% endmacro %} + +{% macro ydb__create_snapshot_table_as(temporary, relation, sql) -%} + {%- set sql_header = config.get('sql_header', none) -%} + + {% set columns = config.get('snapshot_table_column_names') or get_snapshot_table_column_names() %} + + {%- set primary_key_expr = columns.dbt_scd_id -%} + + {%- set store_type = 'row' -%} + + {%- set auto_partitioning_by_size = model['config'].get('auto_partitioning_by_size') -%} + {%- set auto_partitioning_partition_size_mb = model['config'].get('auto_partitioning_partition_size_mb') -%} + {%- set ttl_expr = model['config'].get('ttl') -%} + + {{ sql_header if sql_header is not none }} + + create {% if temporary: -%}temporary{%- endif %} table + {{ relation.include(database=(not temporary), schema=(not temporary)) }} + (primary key ({{ primary_key_expr }})) + {% set contract_config = config.get('contract') %} + {% if contract_config.enforced and (not temporary) %} + {{ get_assert_columns_equivalent(sql) }} + {{ get_table_columns_and_constraints() }} + {%- set sql = get_select_subquery(sql) %} + {% endif %} + WITH ( + STORE = {{ store_type }} + {%- if auto_partitioning_by_size is not none -%} + , AUTO_PARTITIONING_BY_SIZE = {{ auto_partitioning_by_size }} + {%- endif -%} + {%- if auto_partitioning_partition_size_mb is not none -%} + , AUTO_PARTITIONING_PARTITION_SIZE_MB = {{ auto_partitioning_partition_size_mb }} + {%- endif -%} + {%- if ttl_expr is not none -%} + , TTL = {{ ttl_expr }} + {%- endif -%} + ) + as + {{ sql }} + +{%- endmacro %} + + +{% macro ydb__snapshot_staging_table(strategy, source_sql, target_relation) -%} + {% set columns = config.get('snapshot_table_column_names') or get_snapshot_table_column_names() %} + {% if strategy.hard_deletes == 'new_record' %} + {% set new_scd_id = snapshot_hash_arguments([columns.dbt_scd_id, snapshot_get_time()]) %} + {% endif %} + + select `insertions`.* from ( + select + 'insert' as dbt_change_type, + `source_data`.* + {%- if strategy.hard_deletes == 'new_record' -%} + ,'False' as {{ columns.dbt_is_deleted }} + {%- endif %} + + from ( + select `source`.*, {{ unique_key_fields(strategy.unique_key) }}, + {{ strategy.updated_at }} as {{ columns.dbt_updated_at }}, + {{ strategy.updated_at }} as {{ columns.dbt_valid_from }}, + {{ ydb__get_dbt_valid_to_current(strategy, columns) }}, + {{ strategy.scd_id }} as {{ columns.dbt_scd_id }} + + from ({{ source_sql }}) as source + + ) as source_data + left outer join ( + select `target`.*, {{ unique_key_fields(strategy.unique_key) }} + from {{ target_relation }} as target + where + {% if config.get('dbt_valid_to_current') %} + {% set source_unique_key = columns.dbt_valid_to | trim %} + {% set target_unique_key = config.get('dbt_valid_to_current') | trim %} + + {# The exact equals semantics between NULL values depends on the current behavior flag set. Also, update records if the source field is null #} + ( {{ equals(source_unique_key, target_unique_key) }} or {{ source_unique_key }} is null ) + {% else %} + {{ columns.dbt_valid_to }} is null + {% endif %} + ) as snapshotted_data + on {{ unique_key_join_on(strategy.unique_key, "snapshotted_data", "source_data") }} + where {{ unique_key_is_null(strategy.unique_key, "snapshotted_data") }} + or ({{ unique_key_is_not_null(strategy.unique_key, "snapshotted_data") }} and ( + {{ strategy.row_changed }} {%- if strategy.hard_deletes == 'new_record' -%} or snapshotted_data.{{ columns.dbt_is_deleted }} = 'True' {% endif %} + ) + + ) + ) as insertions + union all + select `updates`.* from ( + select + 'update' as dbt_change_type, + source_data.*, + snapshotted_data.{{ columns.dbt_scd_id }} as {{ columns.dbt_scd_id }}, + {%- if strategy.hard_deletes == 'new_record' -%} + , snapshotted_data.{{ columns.dbt_is_deleted }} + {%- endif %} + + from ( + select `source`.*, {{ unique_key_fields(strategy.unique_key) }}, + {{ strategy.updated_at }} as {{ columns.dbt_updated_at }}, + {{ strategy.updated_at }} as {{ columns.dbt_valid_from }}, + {{ strategy.updated_at }} as {{ columns.dbt_valid_to }} + + from ({{ source_sql }}) as source + ) as source_data + join ( + select `target`.*, {{ unique_key_fields(strategy.unique_key) }} + from {{ target_relation }} as target + where + {% if config.get('dbt_valid_to_current') %} + {% set source_unique_key = columns.dbt_valid_to | trim %} + {% set target_unique_key = config.get('dbt_valid_to_current') | trim %} + + {# The exact equals semantics between NULL values depends on the current behavior flag set. Also, update records if the source field is null #} + ( {{ equals(source_unique_key, target_unique_key) }} or {{ source_unique_key }} is null ) + {% else %} + {{ columns.dbt_valid_to }} is null + {% endif %} + ) as snapshotted_data + on {{ unique_key_join_on(strategy.unique_key, "snapshotted_data", "source_data") }} + where ( + {{ strategy.row_changed }} {%- if strategy.hard_deletes == 'new_record' -%} or snapshotted_data.{{ columns.dbt_is_deleted }} = 'True' {% endif %} + ) + ) as updates + {%- if strategy.hard_deletes == 'invalidate' or strategy.hard_deletes == 'new_record' %} + union all + select `deletes`.* from ( + select + 'delete' as dbt_change_type, + -- source_data.*, we are not able to use it because we lose not null constraint on unique key + snapshotted_data.{{ strategy.unique_key }} as {{ strategy.unique_key }}, + + {{ snapshot_get_time() }} as {{ columns.dbt_valid_from }}, + {{ snapshot_get_time() }} as {{ columns.dbt_updated_at }}, + {{ snapshot_get_time() }} as {{ columns.dbt_valid_to }}, + snapshotted_data.{{ columns.dbt_scd_id }} as {{ columns.dbt_scd_id }}, + {%- if strategy.hard_deletes == 'new_record' -%} + , snapshotted_data.{{ columns.dbt_is_deleted }} + {%- endif %} + from ( + select `target`.*, {{ unique_key_fields(strategy.unique_key) }} + from {{ target_relation }} as target + where + {% if config.get('dbt_valid_to_current') %} + {% set source_unique_key = columns.dbt_valid_to | trim %} + {% set target_unique_key = config.get('dbt_valid_to_current') | trim %} + + {# The exact equals semantics between NULL values depends on the current behavior flag set. Also, update records if the source field is null #} + ( {{ equals(source_unique_key, target_unique_key) }} or {{ source_unique_key }} is null ) + {% else %} + {{ columns.dbt_valid_to }} is null + {% endif %} + ) as snapshotted_data + left join ( + select `source`.*, {{ unique_key_fields(strategy.unique_key) }} + from ({{ source_sql }}) as source + ) as source_data + on {{ unique_key_join_on(strategy.unique_key, "snapshotted_data", "source_data") }} + where {{ unique_key_is_null(strategy.unique_key, "source_data") }} + + {%- if strategy.hard_deletes == 'new_record' %} + and not ( + --avoid updating the record's valid_to if the latest entry is marked as deleted + snapshotted_data.{{ columns.dbt_is_deleted }} = 'True' + and snapshotted_data.{{ columns.dbt_valid_to }} is null + ) + {%- endif %} + ) as deletes + {%- endif %} + {%- if strategy.hard_deletes == 'new_record' %} + union all + select `deletion_records`.* from ( + select + 'insert' as dbt_change_type, + {# + If a column has been added to the source it won't yet exist in the + snapshotted table so we insert a null value as a placeholder for the column. + #} + {%- for col in source_sql_cols -%} + {%- if col.name in snapshotted_cols -%} + snapshotted_data.{{ adapter.quote(col.column) }}, + {%- else -%} + NULL as {{ adapter.quote(col.column) }}, + {%- endif -%} + {% endfor -%} + {%- if strategy.unique_key | is_list -%} + {%- for key in strategy.unique_key -%} + snapshotted_data.{{ key }} as dbt_unique_key_{{ loop.index }}, + {% endfor -%} + {%- else -%} + snapshotted_data.dbt_unique_key as dbt_unique_key, + {% endif -%} + {{ snapshot_get_time() }} as {{ columns.dbt_valid_from }}, + {{ snapshot_get_time() }} as {{ columns.dbt_updated_at }}, + snapshotted_data.{{ columns.dbt_valid_to }} as {{ columns.dbt_valid_to }}, + {{ new_scd_id }} as {{ columns.dbt_scd_id }}, + 'True' as {{ columns.dbt_is_deleted }} + from ( + select `target`.*, {{ unique_key_fields(strategy.unique_key) }} + from {{ target_relation }} as target + where + {% if config.get('dbt_valid_to_current') %} + {% set source_unique_key = columns.dbt_valid_to | trim %} + {% set target_unique_key = config.get('dbt_valid_to_current') | trim %} + + {# The exact equals semantics between NULL values depends on the current behavior flag set. Also, update records if the source field is null #} + ( {{ equals(source_unique_key, target_unique_key) }} or {{ source_unique_key }} is null ) + {% else %} + {{ columns.dbt_valid_to }} is null + {% endif %} + ) as snapshotted_data + left join ( + select `source`.*, {{ unique_key_fields(strategy.unique_key) }} + from ({{ source_sql }}) as source + ) as source_data + on {{ unique_key_join_on(strategy.unique_key, "snapshotted_data", "source_data") }} + where {{ unique_key_is_null(strategy.unique_key, "source_data") }} + and not ( + --avoid inserting a new record if the latest one is marked as deleted + snapshotted_data.{{ columns.dbt_is_deleted }} = 'True' + and snapshotted_data.{{ columns.dbt_valid_to }} is null + ) + ) as deletion_records + {%- endif %} + + +{%- endmacro %} \ No newline at end of file diff --git a/dbt/include/ydb/macros/materializations/snapshots/snapshot.sql b/dbt/include/ydb/macros/materializations/snapshots/snapshot.sql new file mode 100644 index 0000000..5516e3c --- /dev/null +++ b/dbt/include/ydb/macros/materializations/snapshots/snapshot.sql @@ -0,0 +1,111 @@ +{% materialization snapshot, adapter='ydb' %} + + {%- set target_table = model.get('alias', model.get('name')) -%} + + {%- set strategy_name = config.get('strategy') -%} + {%- set unique_key = config.get('unique_key') %} + -- grab current tables grants config for comparision later on + {%- set grant_config = config.get('grants') -%} + + {% set target_relation_exists, target_relation = get_or_create_relation( + database=model.database, + schema=model.schema, + identifier=target_table, + type='table') -%} + + {%- if not target_relation.is_table -%} + {% do exceptions.relation_wrong_type(target_relation, 'table') %} + {%- endif -%} + + + {{ run_hooks(pre_hooks, inside_transaction=False) }} + + {{ run_hooks(pre_hooks, inside_transaction=True) }} + + {% set strategy_macro = strategy_dispatch(strategy_name) %} + {# The model['config'] parameter below is no longer used, but passing anyway for compatibility #} + {# It was a dictionary of config, instead of the config object from the context #} + {% set strategy = strategy_macro(model, "snapshotted_data", "source_data", model['config'], target_relation_exists) %} + + {% if not target_relation_exists %} + + {% set build_sql = build_snapshot_table(strategy, model['compiled_code']) %} + {% set build_or_select_sql = build_sql %} + {% set final_sql = ydb__create_snapshot_table_as(False, target_relation, build_sql) %} + + {% else %} + + {% set columns = config.get("snapshot_table_column_names") or get_snapshot_table_column_names() %} + + {{ adapter.assert_valid_snapshot_target_given_strategy(target_relation, columns, strategy) }} + + {% set build_or_select_sql = snapshot_staging_table(strategy, sql, target_relation) %} + {% set staging_table = ydb__build_snapshot_staging_table(strategy, sql, target_relation) %} + + -- this may no-op if the database does not require column expansion + {% do adapter.expand_target_column_types(from_relation=staging_table, + to_relation=target_relation) %} + + {% set remove_columns = ['dbt_change_type', 'DBT_CHANGE_TYPE', 'dbt_unique_key', 'DBT_UNIQUE_KEY'] %} + {% if unique_key | is_list %} + {% for key in strategy.unique_key %} + {{ remove_columns.append('dbt_unique_key_' + loop.index|string) }} + {{ remove_columns.append('DBT_UNIQUE_KEY_' + loop.index|string) }} + {% endfor %} + {% endif %} + + {% set missing_columns = adapter.get_missing_columns(staging_table, target_relation) + | rejectattr('name', 'in', remove_columns) + | list %} + + {% do create_columns(target_relation, missing_columns) %} + + {% set source_columns = adapter.get_columns_in_relation(staging_table) + | rejectattr('name', 'in', remove_columns) + | list %} + + {% set quoted_source_columns = [] %} + {% for column in source_columns %} + {% do quoted_source_columns.append(adapter.quote(column.name)) %} + {% endfor %} + + {% set final_sql = snapshot_merge_sql( + target = target_relation, + source = staging_table, + insert_cols = quoted_source_columns + ) + %} + + {% endif %} + + + {{ check_time_data_types(build_or_select_sql) }} + + {% call statement('main') %} + {{ final_sql }} + {% endcall %} + + {% set should_revoke = should_revoke(target_relation_exists, full_refresh_mode=False) %} + {% do apply_grants(target_relation, grant_config, should_revoke=should_revoke) %} + + {% do persist_docs(target_relation, model) %} + + {% if not target_relation_exists %} + {% do create_indexes(target_relation) %} + {% endif %} + + {{ run_hooks(post_hooks, inside_transaction=True) }} + + {{ adapter.commit() }} + + {% if staging_table is defined %} + {% do post_snapshot(staging_table) %} + {% do drop_relation(staging_table) %} + {% endif %} + + + {{ run_hooks(post_hooks, inside_transaction=False) }} + + {{ return({'relations': [target_relation]}) }} + +{% endmaterialization %} diff --git a/dbt/include/ydb/macros/materializations/snapshots/snapshot_merge.sql b/dbt/include/ydb/macros/materializations/snapshots/snapshot_merge.sql new file mode 100644 index 0000000..da5d41e --- /dev/null +++ b/dbt/include/ydb/macros/materializations/snapshots/snapshot_merge.sql @@ -0,0 +1,23 @@ +{% macro ydb__snapshot_merge_sql(target, source, insert_cols) -%} + {%- set insert_cols_csv = insert_cols | join(', ') -%} + {%- set columns = config.get("snapshot_table_column_names") or get_snapshot_table_column_names() -%} + + -- update for matched (closing old versions) + UPDATE {{ target.render() }} ON + SELECT + tmp.{{ columns.dbt_scd_id }} AS {{ columns.dbt_scd_id }}, tmp.{{ columns.dbt_valid_to }} AS {{ columns.dbt_valid_to }} + FROM {{ target.render() }} snapshot + JOIN {{ source }} tmp ON tmp.{{ columns.dbt_scd_id }}=snapshot.{{ columns.dbt_scd_id }} + WHERE + snapshot.{{ columns.dbt_valid_to }} IS NULL + AND + tmp.dbt_change_type IN ('update', 'delete') + ; + + -- upsert for new rows (not matched) + UPSERT INTO {{ target.render() }} ({{ insert_cols_csv }}) + SELECT {{ insert_cols_csv }} + FROM {{ source }} + WHERE dbt_change_type = 'insert' + ; +{% endmacro %} diff --git a/dbt/include/ydb/macros/materializations/snapshots/strategies.sql b/dbt/include/ydb/macros/materializations/snapshots/strategies.sql new file mode 100644 index 0000000..98896bf --- /dev/null +++ b/dbt/include/ydb/macros/materializations/snapshots/strategies.sql @@ -0,0 +1,52 @@ +{% macro ydb__snapshot_hash_arguments(args) -%} + Digest::Md5Hex({%- for arg in args -%} + coalesce(cast({{ arg }} as varchar ), '') + {% if not loop.last %} || '|' || {% endif %} + {%- endfor -%}) +{%- endmacro %} + + +-- We have to wrap not expr to not (expr) to avoid YDB syntax error +{% macro snapshot_check_strategy(node, snapshotted_rel, current_rel, model_config, target_exists) %} + {# The model_config parameter is no longer used, but is passed in anyway for compatibility. #} + {% set check_cols_config = config.get('check_cols') %} + {% set primary_key = config.get('unique_key') %} + {% set hard_deletes = adapter.get_hard_deletes_behavior(config) %} + {% set invalidate_hard_deletes = hard_deletes == 'invalidate' %} + {% set updated_at = config.get('updated_at') or snapshot_get_time() %} + + {% set column_added = false %} + + {% set column_added, check_cols = snapshot_check_all_get_existing_columns(node, target_exists, check_cols_config) %} + + {%- set row_changed_expr -%} + ( + {%- if column_added -%} + {{ get_true_sql() }} + {%- else -%} + {%- for col in check_cols -%} + {{ snapshotted_rel }}.{{ col }} != {{ current_rel }}.{{ col }} + or + ( + (({{ snapshotted_rel }}.{{ col }} is null) and not ({{ current_rel }}.{{ col }} is null)) + or + ((not ({{ snapshotted_rel }}.{{ col }} is null)) and ({{ current_rel }}.{{ col }} is null)) + ) + {%- if not loop.last %} or {% endif -%} + {%- endfor -%} + {%- endif -%} + ) + {%- endset %} + + {% set scd_args = api.Relation.scd_args(primary_key, updated_at) %} + {% set scd_id_expr = snapshot_hash_arguments(scd_args) %} + + {% do return({ + "unique_key": primary_key, + "updated_at": updated_at, + "row_changed": row_changed_expr, + "scd_id": scd_id_expr, + "invalidate_hard_deletes": invalidate_hard_deletes, + "hard_deletes": hard_deletes + }) %} +{% endmacro %} diff --git a/examples/jaffle_shop/models/schema.yml b/examples/jaffle_shop/models/schema.yml index 381349c..a615757 100644 --- a/examples/jaffle_shop/models/schema.yml +++ b/examples/jaffle_shop/models/schema.yml @@ -80,3 +80,11 @@ models: description: Amount of the order (AUD) paid for by gift card tests: - not_null + +sources: + - name: jaffle_shop + database: raw + schema: jaffle_shop + tables: + - name: orders + - name: customers \ No newline at end of file diff --git a/examples/jaffle_shop/snapshots/orders_snapshot.yml b/examples/jaffle_shop/snapshots/orders_snapshot.yml new file mode 100644 index 0000000..c88741f --- /dev/null +++ b/examples/jaffle_shop/snapshots/orders_snapshot.yml @@ -0,0 +1,7 @@ +snapshots: + - name: orders_snapshot + relation: ref('orders') + config: + strategy: check + check_cols: all + unique_key: order_id \ No newline at end of file diff --git a/tests/functional/adapter/basic/test_snapshot_check_cols.py b/tests/functional/adapter/basic/test_snapshot_check_cols.py index 7fada40..3a57d7f 100644 --- a/tests/functional/adapter/basic/test_snapshot_check_cols.py +++ b/tests/functional/adapter/basic/test_snapshot_check_cols.py @@ -1,5 +1,5 @@ -# from dbt.tests.adapter.basic.test_snapshot_check_cols import BaseSnapshotCheckCols +from dbt.tests.adapter.basic.test_snapshot_check_cols import BaseSnapshotCheckCols -# class TestSnapshotCheckCols(BaseSnapshotCheckCols): -# pass +class TestSnapshotCheckCols(BaseSnapshotCheckCols): + pass diff --git a/tests/functional/adapter/basic/test_snapshot_timestamp.py b/tests/functional/adapter/basic/test_snapshot_timestamp.py index d167159..d9ebf37 100644 --- a/tests/functional/adapter/basic/test_snapshot_timestamp.py +++ b/tests/functional/adapter/basic/test_snapshot_timestamp.py @@ -1,5 +1,5 @@ -# from dbt.tests.adapter.basic.test_snapshot_timestamp import BaseSnapshotTimestamp +from dbt.tests.adapter.basic.test_snapshot_timestamp import BaseSnapshotTimestamp -# class TestSnapshotTimestamp(BaseSnapshotTimestamp): -# pass +class TestSnapshotTimestamp(BaseSnapshotTimestamp): + pass