-
Notifications
You must be signed in to change notification settings - Fork 11
/
dbt_assets.py
215 lines (171 loc) · 7.48 KB
/
dbt_assets.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
import json
import textwrap
from typing import Any, Mapping
from dagster import (
AutoMaterializePolicy,
AutoMaterializeRule,
AssetKey,
BackfillPolicy,
DailyPartitionsDefinition,
job,
op,
OpExecutionContext,
WeeklyPartitionsDefinition,
)
from dagster_cloud.dagster_insights import dbt_with_snowflake_insights
from dagster_dbt import (
DbtCliResource,
DagsterDbtTranslator,
default_metadata_from_dbt_resource_props,
DagsterDbtTranslatorSettings,
)
from dagster_dbt.asset_decorator import dbt_assets
from hooli_data_eng.resources import dbt_project
from dagster_dbt.freshness_builder import build_freshness_checks_from_dbt_assets
from dagster import build_sensor_for_freshness_checks
# many dbt assets use an incremental approach to avoid
# re-processing all data on each run
# this approach can be modelled in dagster using partitions
# this project includes assets with hourly and daily partitions
daily_partitions = DailyPartitionsDefinition(start_date="2023-05-25")
weekly_partitions = WeeklyPartitionsDefinition(start_date="2023-05-25")
DBT_MANIFEST = dbt_project.manifest_path
allow_outdated_parents_policy = AutoMaterializePolicy.eager().without_rules(
AutoMaterializeRule.skip_on_parent_outdated()
)
allow_outdated_and_missing_parents_policy = AutoMaterializePolicy.eager().without_rules(
AutoMaterializeRule.skip_on_parent_outdated(),
AutoMaterializeRule.skip_on_parent_missing(), # non-partitioned assets should run even if some upstream partitions are missing
)
class CustomDagsterDbtTranslator(DagsterDbtTranslator):
def get_description(self, dbt_resource_props: Mapping[str, Any]) -> str:
description = f"dbt model for: {dbt_resource_props['name']} \n \n"
return description + textwrap.indent(
dbt_resource_props.get("raw_code", ""), "\t"
)
def get_asset_key(self, dbt_resource_props: Mapping[str, Any]) -> AssetKey:
node_path = dbt_resource_props["path"]
prefix = node_path.split("/")[0]
if node_path == "models/sources.yml":
prefix = "RAW_DATA"
if node_path == "MARKETING/company_perf.sql":
prefix = "ANALYTICS"
return AssetKey([prefix, dbt_resource_props["name"]])
def get_group_name(self, dbt_resource_props: Mapping[str, Any]):
node_path = dbt_resource_props["path"]
prefix = node_path.split("/")[0]
if node_path == "models/sources.yml":
prefix = "RAW_DATA"
if node_path == "MARKETING/company_perf.sql":
prefix = "ANALYTICS"
return prefix
def get_metadata(self, dbt_resource_props: Mapping[str, Any]) -> Mapping[str, Any]:
metadata = {"partition_expr": "order_date"}
if dbt_resource_props["name"] == "orders_cleaned":
metadata = {"partition_expr": "dt"}
if dbt_resource_props["name"] == "users_cleaned":
metadata = {"partition_expr": "created_at"}
default_metadata = default_metadata_from_dbt_resource_props(dbt_resource_props)
return {**default_metadata, **metadata}
def get_auto_materialize_policy(self, dbt_resource_props: Mapping[str, Any]):
return allow_outdated_parents_policy
def get_owners(self, dbt_resource_props: Mapping[str, Any]):
return [
dbt_resource_props["group"]["owner"]["email"],
f"team:{dbt_resource_props['group']['name']}",
]
class CustomDagsterDbtTranslatorForViews(CustomDagsterDbtTranslator):
def get_auto_materialize_policy(self, dbt_resource_props: Mapping[str, Any]):
return allow_outdated_and_missing_parents_policy
def _process_partitioned_dbt_assets(context: OpExecutionContext, dbt: DbtCliResource):
# map partition key range to dbt vars
first_partition, last_partition = context.asset_partitions_time_window_for_output(
list(context.selected_output_names)[0]
)
dbt_vars = {"min_date": str(first_partition), "max_date": str(last_partition)}
dbt_args = ["build", "--vars", json.dumps(dbt_vars)]
# Invoke dbt CLI
dbt_cli_task = dbt.cli(dbt_args, context=context)
# Emits an AssetObservation for each asset materialization, which is used to
# identify the Snowflake credit consumption
yield from dbt_with_snowflake_insights(
context=context,
dbt_cli_invocation=dbt_cli_task,
dagster_events=dbt_cli_task.stream().fetch_row_counts().fetch_column_metadata(),
)
# fetch run_results.json to log compiled SQL
run_results_json = dbt_cli_task.get_artifact("run_results.json")
for result in run_results_json["results"]:
model_name = result.get("unique_id")
context.log.info(f"Compiled SQL for {model_name}:\n{result['compiled_code']}")
@dbt_assets(
manifest=DBT_MANIFEST,
select="orders_cleaned users_cleaned orders_augmented",
partitions_def=daily_partitions,
dagster_dbt_translator=CustomDagsterDbtTranslator(
settings=DagsterDbtTranslatorSettings(enable_asset_checks=True)
),
backfill_policy=BackfillPolicy.single_run(),
)
def daily_dbt_assets(context: OpExecutionContext, dbt2: DbtCliResource):
yield from _process_partitioned_dbt_assets(context=context, dbt=dbt2)
@dbt_assets(
manifest=DBT_MANIFEST,
select="weekly_order_summary order_stats",
partitions_def=weekly_partitions,
dagster_dbt_translator=CustomDagsterDbtTranslator(
DagsterDbtTranslatorSettings(enable_asset_checks=True)
),
backfill_policy=BackfillPolicy.single_run(),
)
def weekly_dbt_assets(context: OpExecutionContext, dbt2: DbtCliResource):
yield from _process_partitioned_dbt_assets(context=context, dbt=dbt2)
weekly_freshness_check = build_freshness_checks_from_dbt_assets(dbt_assets=[weekly_dbt_assets])
weekly_freshness_check_sensor=build_sensor_for_freshness_checks(
freshness_checks=weekly_freshness_check,
name="weekly_freshness_check_sensor"
)
@dbt_assets(
manifest=DBT_MANIFEST,
select="company_perf sku_stats company_stats locations_cleaned",
dagster_dbt_translator=CustomDagsterDbtTranslatorForViews(
DagsterDbtTranslatorSettings(enable_asset_checks=True)
),
)
def views_dbt_assets(context: OpExecutionContext, dbt2: DbtCliResource):
# Invoke dbt CLI
dbt_cli_task = dbt2.cli(["build"], context=context)
# Emits an AssetObservation for each asset materialization, which is used to
# identify the Snowflake credit consumption
yield from dbt_with_snowflake_insights(
context=context,
dbt_cli_invocation=dbt_cli_task,
dagster_events=dbt_cli_task.stream().fetch_row_counts().fetch_column_metadata(),
)
# fetch run_results.json to log compiled SQL
run_results_json = dbt_cli_task.get_artifact("run_results.json")
for result in run_results_json["results"]:
model_name = result.get("unique_id")
context.log.info(f"Compiled SQL for {model_name}:\n{result['compiled_code']}")
# This op will be used to run slim CI
@op(out={})
def dbt_slim_ci(dbt2: DbtCliResource):
dbt_command = [
"build",
"--select",
"state:modified+",
"--defer",
"--state",
dbt2.state_path,
]
yield from dbt2.cli(
args=dbt_command,
manifest=DBT_MANIFEST,
dagster_dbt_translator=CustomDagsterDbtTranslator(
DagsterDbtTranslatorSettings(enable_asset_checks=True)
),
).stream().fetch_row_counts().fetch_column_metadata()
# This job will be triggered by Pull Request and should only run new or changed dbt models
@job
def dbt_slim_ci_job():
dbt_slim_ci()