/
step.py
505 lines (424 loc) · 15.8 KB
/
step.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
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
from abc import abstractmethod
from enum import Enum
from typing import (
TYPE_CHECKING,
FrozenSet,
List,
Mapping,
NamedTuple,
Optional,
Sequence,
Set,
Type,
Union,
cast,
)
from typing_extensions import TypeGuard
import dagster._check as check
from dagster._core.definitions.utils import validate_tags
from dagster._serdes.serdes import DefaultEnumSerializer, whitelist_for_serdes
from dagster._utils.merger import merge_dicts
from .handle import ResolvedFromDynamicStepHandle, StepHandle, UnresolvedStepHandle
from .inputs import StepInput, UnresolvedCollectStepInput, UnresolvedMappedStepInput
from .outputs import StepOutput
if TYPE_CHECKING:
from dagster._core.definitions.dependency import NodeHandle
class StepKindSerializer(DefaultEnumSerializer):
@classmethod
def value_from_storage_str(cls, storage_str: str, klass: Type) -> Enum:
# old name for unresolved mapped
if storage_str == "UNRESOLVED":
value = "UNRESOLVED_MAPPED"
else:
value = storage_str
return super().value_from_storage_str(value, klass)
@whitelist_for_serdes(serializer=StepKindSerializer)
class StepKind(Enum):
COMPUTE = "COMPUTE"
UNRESOLVED_MAPPED = "UNRESOLVED_MAPPED"
UNRESOLVED_COLLECT = "UNRESOLVED_COLLECT"
def is_executable_step(
step: Union["ExecutionStep", "UnresolvedMappedExecutionStep"]
) -> TypeGuard["ExecutionStep"]:
# This function is set up defensively to ensure new step types handled properly
if isinstance(step, ExecutionStep):
return True
elif isinstance(step, UnresolvedMappedExecutionStep):
return False
else:
check.failed(f"Unexpected execution step type {step}")
class IExecutionStep:
@property
@abstractmethod
def handle(self) -> Union[StepHandle, UnresolvedStepHandle, ResolvedFromDynamicStepHandle]:
pass
@property
@abstractmethod
def key(self) -> str:
pass
@property
@abstractmethod
def solid_handle(self) -> "NodeHandle":
pass
@property
@abstractmethod
def kind(self) -> StepKind:
pass
@property
@abstractmethod
def tags(self) -> Optional[Mapping[str, str]]:
pass
@property
@abstractmethod
def step_inputs(
self,
) -> Sequence[Union[StepInput, UnresolvedCollectStepInput, UnresolvedMappedStepInput]]:
pass
@property
@abstractmethod
def step_outputs(self) -> Sequence[StepOutput]:
pass
@abstractmethod
def step_input_named(
self, name: str
) -> Union[StepInput, UnresolvedCollectStepInput, UnresolvedMappedStepInput]:
pass
@abstractmethod
def step_output_named(self, name: str) -> StepOutput:
pass
@property
@abstractmethod
def step_output_dict(self) -> Mapping[str, StepOutput]:
pass
@property
@abstractmethod
def step_input_dict(self) -> Mapping[str, StepInput]:
pass
class ExecutionStep(
NamedTuple(
"_ExecutionStep",
[
("handle", Union[StepHandle, ResolvedFromDynamicStepHandle]),
("pipeline_name", str),
("step_input_dict", Mapping[str, StepInput]),
("step_output_dict", Mapping[str, StepOutput]),
("tags", Mapping[str, str]),
("logging_tags", Mapping[str, str]),
("key", str),
],
),
IExecutionStep,
):
"""
A fully resolved step in the execution graph.
"""
def __new__(
cls,
handle: Union[StepHandle, ResolvedFromDynamicStepHandle],
pipeline_name: str,
step_inputs: Sequence[StepInput],
step_outputs: Sequence[StepOutput],
tags: Optional[Mapping[str, str]],
logging_tags: Optional[Mapping[str, str]] = None,
key: Optional[str] = None,
):
return super(ExecutionStep, cls).__new__(
cls,
handle=check.inst_param(handle, "handle", (StepHandle, ResolvedFromDynamicStepHandle)),
pipeline_name=check.str_param(pipeline_name, "pipeline_name"),
step_input_dict={
si.name: si
for si in check.sequence_param(step_inputs, "step_inputs", of_type=StepInput)
},
step_output_dict={
so.name: so
for so in check.sequence_param(step_outputs, "step_outputs", of_type=StepOutput)
},
tags=validate_tags(check.opt_mapping_param(tags, "tags", key_type=str)),
logging_tags=merge_dicts(
{
"step_key": handle.to_key(),
"pipeline_name": pipeline_name,
"solid_name": handle.solid_handle.name,
},
check.opt_mapping_param(logging_tags, "logging_tags"),
),
# mypy can't tell that if default is set, this is guaranteed to be a str
key=cast(str, check.opt_str_param(key, "key", default=handle.to_key())),
)
@property
def solid_handle(self) -> "NodeHandle":
return self.handle.solid_handle
@property
def solid_name(self) -> str:
return self.solid_handle.name
@property
def kind(self) -> StepKind:
return StepKind.COMPUTE
@property
def step_outputs(self) -> Sequence[StepOutput]:
return list(self.step_output_dict.values())
@property
def step_inputs(self) -> Sequence[StepInput]:
return list(self.step_input_dict.values())
def has_step_output(self, name: str) -> bool:
check.str_param(name, "name")
return name in self.step_output_dict
def step_output_named(self, name: str) -> StepOutput:
check.str_param(name, "name")
return self.step_output_dict[name]
def has_step_input(self, name: str) -> bool:
check.str_param(name, "name")
return name in self.step_input_dict
def step_input_named(self, name: str) -> StepInput:
check.str_param(name, "name")
return self.step_input_dict[name]
def get_execution_dependency_keys(self) -> Set[str]:
deps = set()
for inp in self.step_inputs:
deps.update(inp.dependency_keys)
return deps
def get_mapping_key(self):
if isinstance(self.handle, ResolvedFromDynamicStepHandle):
return self.handle.mapping_key
return None
class UnresolvedMappedExecutionStep( # type: ignore
NamedTuple(
"_UnresolvedMappedExecutionStep",
[
("handle", UnresolvedStepHandle),
("pipeline_name", str),
("step_input_dict", Mapping[str, Union[StepInput, UnresolvedMappedStepInput]]),
("step_output_dict", Mapping[str, StepOutput]),
("tags", Mapping[str, str]),
],
),
IExecutionStep,
):
"""
A placeholder step that will become N ExecutionSteps once the upstream dynamic output resolves in to N mapping keys.
"""
def __new__(
cls,
handle: UnresolvedStepHandle,
pipeline_name: str,
step_inputs: Sequence[Union[StepInput, UnresolvedMappedStepInput]],
step_outputs: Sequence[StepOutput],
tags: Optional[Mapping[str, str]],
):
return super(UnresolvedMappedExecutionStep, cls).__new__(
cls,
handle=check.inst_param(handle, "handle", UnresolvedStepHandle),
pipeline_name=check.str_param(pipeline_name, "pipeline_name"),
step_input_dict={
si.name: si
for si in check.sequence_param(
step_inputs, "step_inputs", of_type=(StepInput, UnresolvedMappedStepInput)
)
},
step_output_dict={
so.name: so
for so in check.sequence_param(step_outputs, "step_outputs", of_type=StepOutput)
},
tags=check.opt_mapping_param(tags, "tags", key_type=str),
)
@property
def solid_handle(self) -> "NodeHandle":
return self.handle.solid_handle
@property
def key(self) -> str:
return self.handle.to_key()
@property
def kind(self) -> StepKind:
return StepKind.UNRESOLVED_MAPPED
@property
def step_outputs(self) -> Sequence[StepOutput]:
return list(self.step_output_dict.values())
@property
def step_inputs(self) -> Sequence[Union[StepInput, UnresolvedMappedStepInput]]:
return list(self.step_input_dict.values())
def step_input_named(self, name: str) -> Union[StepInput, UnresolvedMappedStepInput]:
check.str_param(name, "name")
return self.step_input_dict[name]
def step_output_named(self, name: str) -> StepOutput:
check.str_param(name, "name")
return self.step_output_dict[name]
def get_all_dependency_keys(self) -> Set[str]:
deps = set()
for inp in self.step_inputs:
if isinstance(inp, StepInput):
deps.update(
[handle.step_key for handle in inp.get_step_output_handle_dependencies()]
)
elif isinstance(inp, UnresolvedMappedStepInput):
deps.update(
[
handle.step_key
for handle in inp.get_step_output_handle_deps_with_placeholders()
]
)
else:
check.failed(f"Unexpected step input type {inp}")
return deps
@property
def resolved_by_step_key(self) -> str:
# this function will be removed in moving to supporting being downstream of multiple dynamic outputs
keys = self.resolved_by_step_keys
check.invariant(len(keys) == 1, "Unresolved step expects one and only one dynamic step key")
return list(keys)[0]
@property
def resolved_by_output_name(self) -> str:
# this function will be removed in moving to supporting being downstream of multiple dynamic outputs
keys = set()
for inp in self.step_inputs:
if isinstance(inp, UnresolvedMappedStepInput):
keys.add(inp.resolved_by_output_name)
check.invariant(
len(keys) == 1, "Unresolved step expects one and only one dynamic output name"
)
return list(keys)[0]
@property
def resolved_by_step_keys(self) -> FrozenSet[str]:
keys = set()
for inp in self.step_inputs:
if isinstance(inp, UnresolvedMappedStepInput):
keys.add(inp.resolved_by_step_key)
return frozenset(keys)
def resolve(
self, mappings: Mapping[str, Mapping[str, Optional[Sequence[str]]]]
) -> Sequence[ExecutionStep]:
check.invariant(
all(key in mappings for key in self.resolved_by_step_keys),
"resolving with mappings that do not contain all required step keys",
)
execution_steps: List[ExecutionStep] = []
mapping_keys = mappings[self.resolved_by_step_key][self.resolved_by_output_name]
# dynamic output skipped
if mapping_keys is None:
return execution_steps
for mapped_key in mapping_keys:
resolved_inputs = [_resolved_input(inp, mapped_key) for inp in self.step_inputs]
execution_steps.append(
ExecutionStep(
handle=ResolvedFromDynamicStepHandle(self.handle.solid_handle, mapped_key),
pipeline_name=self.pipeline_name,
step_inputs=resolved_inputs,
step_outputs=self.step_outputs,
tags=self.tags,
)
)
return execution_steps
def _resolved_input(
step_input: Union[StepInput, UnresolvedMappedStepInput],
map_key: str,
):
if isinstance(step_input, StepInput):
return step_input
return step_input.resolve(map_key)
class UnresolvedCollectExecutionStep( # type: ignore
NamedTuple(
"_UnresolvedCollectExecutionStep",
[
("handle", StepHandle),
("pipeline_name", str),
("step_input_dict", Mapping[str, Union[StepInput, UnresolvedCollectStepInput]]),
("step_output_dict", Mapping[str, StepOutput]),
("tags", Mapping[str, str]),
],
),
IExecutionStep,
):
"""
A placeholder step that will become 1 ExecutionStep that collects over a dynamic output or downstream from one once it resolves.
"""
def __new__(
cls,
handle: StepHandle,
pipeline_name: str,
step_inputs: Sequence[Union[StepInput, UnresolvedCollectStepInput]],
step_outputs: Sequence[StepOutput],
tags: Optional[Mapping[str, str]],
):
return super(UnresolvedCollectExecutionStep, cls).__new__(
cls,
handle=check.inst_param(handle, "handle", StepHandle),
pipeline_name=check.str_param(pipeline_name, "pipeline_name"),
step_input_dict={
si.name: si
for si in check.sequence_param(
step_inputs, "step_inputs", of_type=(StepInput, UnresolvedCollectStepInput)
)
},
step_output_dict={
so.name: so
for so in check.sequence_param(step_outputs, "step_outputs", of_type=StepOutput)
},
tags=check.opt_mapping_param(tags, "tags", key_type=str),
)
@property
def solid_handle(self) -> "NodeHandle":
return self.handle.solid_handle
@property
def key(self) -> str:
return self.handle.to_key()
@property
def kind(self) -> StepKind:
return StepKind.UNRESOLVED_COLLECT
@property
def step_inputs(self) -> Sequence[Union[StepInput, UnresolvedCollectStepInput]]:
return list(self.step_input_dict.values())
@property
def step_outputs(self) -> Sequence[StepOutput]:
return list(self.step_output_dict.values())
def step_input_named(self, name: str) -> Union[StepInput, UnresolvedCollectStepInput]:
check.str_param(name, "name")
return self.step_input_dict[name]
def step_output_named(self, name: str) -> StepOutput:
check.str_param(name, "name")
return self.step_output_dict[name]
def get_all_dependency_keys(self) -> Set[str]:
deps = set()
for inp in self.step_inputs:
if isinstance(inp, StepInput):
deps.update(
[handle.step_key for handle in inp.get_step_output_handle_dependencies()]
)
elif isinstance(inp, UnresolvedCollectStepInput):
deps.update(
[
handle.step_key
for handle in inp.get_step_output_handle_deps_with_placeholders()
]
)
else:
check.failed(f"Unexpected step input type {inp}")
return deps
@property
def resolved_by_step_keys(self) -> FrozenSet[str]:
keys = set()
for inp in self.step_inputs:
if isinstance(inp, UnresolvedCollectStepInput):
keys.add(inp.resolved_by_step_key)
return frozenset(keys)
def resolve(
self, mappings: Mapping[str, Mapping[str, Optional[Sequence[str]]]]
) -> ExecutionStep:
check.invariant(
all(key in mappings for key in self.resolved_by_step_keys),
"resolving with mappings that do not contain all required step keys",
)
resolved_inputs = []
for inp in self.step_inputs:
if isinstance(inp, StepInput):
resolved_inputs.append(inp)
else:
resolved_inputs.append(
inp.resolve(mappings[inp.resolved_by_step_key][inp.resolved_by_output_name])
)
return ExecutionStep(
handle=self.handle,
pipeline_name=self.pipeline_name,
step_inputs=resolved_inputs,
step_outputs=self.step_outputs,
tags=self.tags,
)