From 8544942048ee74b7d92e47f10222f8840fd8c725 Mon Sep 17 00:00:00 2001 From: Nick Schrock Date: Mon, 13 Aug 2018 05:56:03 -0700 Subject: [PATCH 001/103] Execution Rewrite: The Compute Node This is a substantial rewrite of the execution pipeline and will open up a whole new range of explorations and more. The core architectural change here is that there is an entire new system of representation for the *execution* graph of computations (compute nodes) rather than just the PipelineDefintion. The compute node graph is constructed from the PipelineDefinition and input config. A number of things happen during this construction: 1. All sources, materializations, expectations, and transforms are modelled within the same abstraction: the compute node. All argument checking and validiations are done during compute node graph construction time, before any execution proceeds. This is similar to validation as a separate step in the graphql universe. 2. Compute nodes can have multiple inputs, multiple outputs, and a configuration dictionary (arg dict). This allows for maximum flexibility and for it to be generic enough to handled the different concepts listed in bullet 1. For example a source is just a compute node with *no* inputs, a single output, and an arg dict. A materialization is a compute node with an input (coming from the output of of a transform), no outputs, and an arg dict. A transform can have multiple inputs and (now) multiple outputs, but it current doesn't support an arbitrary arg dict. Expectations are also a compute node. 3. This will allow for dynamically configured dependencies as opposed to the more static construction. We are already using the dynamic compute node graph to change based on policy. If, for example, you do turn off expectations those compute nodes are never even constructed. If they are they dynamically placed before the appopriate input or after the appropriate output and set up to execute in parallel. 4. Sources and materializationsn as separate concepts are going to largely dissappears. I am still toying with how to expose them to the higher level config and pipeline definition API, but if they exist they will be largely symbolic. 5. More sophisticated expectations execution will happen via this mechanism. For example I fully anticipate that expectation compute nodes will have two outputs, one for the value being tested and passed through, but the other for the expectation result. This will allow the framework (or the user in some capacity) to add compute nodes that act on the expectation. 6. Composability: This is also an interesting angle for putting composability into this execution engine. It should be totally possible to build "composite" compute nodes that consist of a DAG of compute nodes. When taken as a whole they will have a set of inputs, a set of outputs, and a set of configuration. 7. Visualizations and debugging: This will be a powerful tool of visualization and debugging. One will be able to set config, apply it to a pipeline, and previz *exactly* what is going to execute in a very granular way. Execution should be able to be reported through this mechanism as well. Next up I'm going to leverage this architectural change to change the dependency API for pipeline definitions. --- python_modules/Makefile | 14 + .../dagster_ge_tests/test_pandas_ge.py | 46 +- python_modules/dagster/dagster/__init__.py | 26 +- .../dagster/dagster/check/__init__.py | 6 +- .../dagster/dagster/cli/pipeline.py | 13 +- python_modules/dagster/dagster/config.py | 4 +- .../dagster/dagster/core/compute_nodes.py | 891 ++++++++++++++++++ .../core/core_tests/test_compute_nodes.py | 83 ++ .../core/core_tests/test_core_execution.py | 208 ---- .../core/core_tests/test_error_handling.py | 59 -- .../core/core_tests/test_execute_solid.py | 16 +- .../core_tests/test_input_expectations.py | 100 -- .../core/core_tests/test_naming_collisions.py | 30 +- .../core_tests/test_output_expectations.py | 73 -- .../core/core_tests/test_pipeline_errors.py | 9 +- .../core_tests/test_pipeline_execution.py | 123 +-- .../core/core_tests/test_pipeline_meta.py | 5 - .../dagster/dagster/core/definitions.py | 142 ++- python_modules/dagster/dagster/core/errors.py | 4 +- .../dagster/dagster/core/execution.py | 869 +++-------------- .../dagster/dagster/core/execution_context.py | 137 +++ python_modules/dagster/dagster/core/graph.py | 119 ++- .../dagster/dagster/core/validation.py | 0 python_modules/dagster/dagster/graphviz.py | 7 +- .../pandas_kernel_tests/test_pandas_solids.py | 74 +- 25 files changed, 1632 insertions(+), 1426 deletions(-) create mode 100644 python_modules/Makefile create mode 100644 python_modules/dagster/dagster/core/compute_nodes.py create mode 100644 python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py delete mode 100644 python_modules/dagster/dagster/core/core_tests/test_core_execution.py delete mode 100644 python_modules/dagster/dagster/core/core_tests/test_error_handling.py delete mode 100644 python_modules/dagster/dagster/core/core_tests/test_input_expectations.py delete mode 100644 python_modules/dagster/dagster/core/core_tests/test_output_expectations.py create mode 100644 python_modules/dagster/dagster/core/execution_context.py create mode 100644 python_modules/dagster/dagster/core/validation.py diff --git a/python_modules/Makefile b/python_modules/Makefile new file mode 100644 index 0000000000000..e336be44bdc7d --- /dev/null +++ b/python_modules/Makefile @@ -0,0 +1,14 @@ +watch: + watchmedo shell-command --recursive --patterns="*.py" -i "*.pyc" --command \ + "pytest -s -vv "\ + "dagster/dagster/check/check_tests "\ + "dagster/dagster/core/core_tests "\ + "dagster/dagster/dagster_examples/dagster_examples_tests "\ + "dagster/dagster/pandas_kernel/pandas_kernel_tests "\ + "dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests "\ + "dagster/dagster/utils/utils_tests "\ + "dagster/dagster_tests "\ + "dagster-ge/dagster_ge_tests " \ + --drop dagster/dagster dagster/dagster_tests/ dagster-ge/dagster_ge dagster-ge/dagster_ge_tests + + diff --git a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py index 3eb2a8468dfc1..fed7beeb129f1 100644 --- a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py +++ b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py @@ -1,3 +1,5 @@ +import pytest + import pandas as pd import dagster @@ -68,7 +70,11 @@ def test_single_node_passing_expectation(): pipeline = dagster.PipelineDefinition(solids=[sum_solid]) result = execute_pipeline_in_memory( - dagster.ExecutionContext(), pipeline, input_values={'num_df': in_df} + dagster.ExecutionContext(), + pipeline, + input_values={sum_solid.name: { + 'num_df': in_df + }}, ) assert result.success assert result.result_list[0].success @@ -84,7 +90,11 @@ def test_single_node_passing_json_config_expectations(): pipeline = dagster.PipelineDefinition(solids=[sum_solid_expectations_config]) result = execute_pipeline_in_memory( - dagster.ExecutionContext(), pipeline, input_values={'num_df': in_df} + dagster.ExecutionContext(), + pipeline, + input_values={sum_solid_expectations_config.name: { + 'num_df': in_df + }}, ) assert result.success assert result.result_list[0].success @@ -99,17 +109,25 @@ def test_single_node_failing_expectation(): in_df = pd.DataFrame.from_dict({'num1': [1, 3], 'num2': [2, 4]}) pipeline = dagster.PipelineDefinition(solids=[sum_solid_fails_input_expectation]) result = execute_pipeline_in_memory( - dagster.ExecutionContext(), pipeline, input_values={'num_df': in_df}, throw_on_error=False + dagster.ExecutionContext(), + pipeline, + input_values={sum_solid_fails_input_expectation.name: { + 'num_df': in_df + }}, + throw_on_error=False ) assert not result.success - assert len(result.result_list) == 1 - first_solid_result = result.result_list[0] - assert not first_solid_result.success - assert first_solid_result.reason == DagsterExecutionFailureReason.EXPECTATION_FAILURE - assert isinstance(first_solid_result.input_expectation_results, dagster.InputExpectationResults) - input_expt_results = first_solid_result.input_expectation_results - assert len(input_expt_results.result_dict) == 1 - input_expt_result = list(input_expt_results.result_dict.values())[0] - assert isinstance(input_expt_result, dagster.InputExpectationResult) - assert len(list(input_expt_result.passes)) == 0 - assert len(list(input_expt_result.fails)) == 1 + + return + # TODO redo expectation result API + # assert len(result.result_list) == 1 + # first_solid_result = result.result_list[0] + # assert not first_solid_result.success + # assert first_solid_result.reason == DagsterExecutionFailureReason.EXPECTATION_FAILURE + # assert isinstance(first_solid_result.input_expectation_results, dagster.InputExpectationResults) + # input_expt_results = first_solid_result.input_expectation_results + # assert len(input_expt_results.result_dict) == 1 + # input_expt_result = list(input_expt_results.result_dict.values())[0] + # assert isinstance(input_expt_result, dagster.InputExpectationResult) + # assert len(list(input_expt_result.passes)) == 0 + # assert len(list(input_expt_result.fails)) == 1 diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index fde9e2fdf3fc9..9fabc3594dcbb 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -1,32 +1,26 @@ from __future__ import (absolute_import, division, print_function, unicode_literals) from builtins import * # pylint: disable=W0622,W0401 -from dagster.core.execution import ( - execute_pipeline, - ExecutionContext, - ExpectationResult, - InputExpectationInfo, - OutputExpectationInfo, - InputExpectationResult, - InputExpectationResults, -) +from dagster.core.execution import execute_pipeline +from dagster.core.execution_context import ExecutionContext from dagster.core.definitions import ( - PipelineContextDefinition, - PipelineDefinition, + ArgumentDefinition, + ExpectationDefinition, + ExpectationResult, InputDefinition, - OutputDefinition, MaterializationDefinition, - ExpectationDefinition, - SourceDefinition, + OutputDefinition, + PipelineContextDefinition, + PipelineDefinition, SolidDefinition, - ArgumentDefinition, + SourceDefinition, ) from dagster.core.decorators import ( + materialization, solid, source, - materialization, with_context, ) diff --git a/python_modules/dagster/dagster/check/__init__.py b/python_modules/dagster/dagster/check/__init__.py index a70e512955cc5..c85f445277d08 100644 --- a/python_modules/dagster/dagster/check/__init__.py +++ b/python_modules/dagster/dagster/check/__init__.py @@ -21,7 +21,7 @@ class NotImplementedCheckError(CheckError): def _param_type_mismatch_exception(obj, ttype, param_name): return ParameterCheckError( 'Param "{name}" is not a {type}. Got {obj} with is type {obj_type}.'.format( - name=param_name, obj=repr(obj), type=ttype.__name__, obj_type=type(obj).__name__ + name=param_name, obj=repr(obj), type=ttype.__name__, obj_type=type(obj) ) ) @@ -30,13 +30,13 @@ def _type_mismatch_error(obj, ttype, desc): if desc: return CheckError( 'Object {obj} is not a {type}. Got {obj} with type {obj_type}. Desc: {desc}'.format( - obj=repr(obj), type=ttype.__name__, obj_type=type(obj).__name__, desc=desc + obj=repr(obj), type=ttype.__name__, obj_type=type(obj), desc=desc ) ) else: return CheckError( 'Object {obj} is not a {type}. Got {obj} with type {obj_type}.'.format( - obj=repr(obj), type=ttype.__name__, obj_type=type(obj).__name__ + obj=repr(obj), type=ttype.__name__, obj_type=type(obj) ) ) diff --git a/python_modules/dagster/dagster/cli/pipeline.py b/python_modules/dagster/dagster/cli/pipeline.py index 7c899f8c17e1f..a03245f2e03aa 100644 --- a/python_modules/dagster/dagster/cli/pipeline.py +++ b/python_modules/dagster/dagster/cli/pipeline.py @@ -9,6 +9,7 @@ import dagster from dagster import check from dagster.core.execution import (DagsterExecutionFailureReason, execute_pipeline_iterator) +from dagster.core.graph import SolidGraph from dagster.graphviz import build_graphviz_graph from dagster.utils.indenting_printer import IndentingPrinter @@ -36,7 +37,8 @@ def list_command(config): click.echo('Description:') click.echo(format_description(pipeline.description, indent=' ' * 4)) click.echo('Solids: (Execution Order)') - for solid in pipeline.solid_graph.topological_solids: + solid_graph = SolidGraph(pipeline.solids, pipeline.dependency_structure) + for solid in solid_graph.topological_solids: click.echo(' ' + solid.name) click.echo('*************') @@ -156,14 +158,7 @@ def print_inputs(printer, solid): printer.line('Inputs:') for input_def in solid.inputs: with printer.with_indent(): - if input_def.depends_on: - printer.line( - 'Input: {name} (depends on {dep_name})'.format( - name=input_def.name, dep_name=input_def.depends_on.name - ) - ) - else: - printer.line('Input: {name}'.format(name=input_def.name)) + printer.line('Input: {name}'.format(name=input_def.name)) if input_def.sources: print_sources(printer, input_def.sources) diff --git a/python_modules/dagster/dagster/config.py b/python_modules/dagster/dagster/config.py index cb6a2f59f5be1..645b344b922ce 100644 --- a/python_modules/dagster/dagster/config.py +++ b/python_modules/dagster/dagster/config.py @@ -4,13 +4,15 @@ # lifted from https://bit.ly/2HcQAuv -class Materialization(namedtuple('MaterializationData', 'solid name args')): +class Materialization(namedtuple('MaterializationData', 'solid name args output_name')): def __new__(cls, solid, name, args): + DEFAULT_OUTPUT = 'result' return super(Materialization, cls).__new__( cls, solid=check.str_param(solid, 'solid'), name=check.str_param(name, 'name'), args=check.dict_param(args, 'args', key_type=str), + output_name=DEFAULT_OUTPUT, ) diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py new file mode 100644 index 0000000000000..42afb90bf5d0c --- /dev/null +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -0,0 +1,891 @@ +from collections import (namedtuple, defaultdict) +from contextlib import contextmanager +from enum import Enum +import uuid +import sys + +import toposort + +from dagster import (check, config) + +from dagster.utils.indenting_printer import IndentingPrinter +from dagster.utils.logging import (get_formatted_stack_trace, define_colored_console_logger, DEBUG) +from dagster.utils.timing import time_execution_scope + +from .argument_handling import validate_args + +from .definitions import ( + SolidDefinition, + InputDefinition, + OutputDefinition, + PipelineDefinition, + ExpectationDefinition, + MaterializationDefinition, +) + +from .execution_context import ExecutionContext + +from .errors import ( + DagsterInvariantViolationError, + DagsterUserCodeExecutionError, + DagsterExpectationFailedError, +) + +from .graph import create_subgraph + +from .types import ( + Any, + DagsterType, +) + +LOG_LEVEL = DEBUG +logger = define_colored_console_logger('dagster-compute-nodes', LOG_LEVEL) + +@contextmanager +def _user_code_error_boundary(context, msg, **kwargs): + ''' + Wraps the execution of user-space code in an error boundary. This places a uniform + policy around an user code invoked by the framework. This ensures that all user + errors are wrapped in the SolidUserCodeExecutionError, and that the original stack + trace of the user error is preserved, so that it can be reported without confusing + framework code in the stack trace, if a tool author wishes to do so. This has + been especially help in a notebooking context. + ''' + check.inst_param(context, 'context', ExecutionContext) + check.str_param(msg, 'msg') + + try: + yield + except Exception as e: + stack_trace = get_formatted_stack_trace(e) + context.error(str(e), stack_trace=stack_trace) + raise DagsterUserCodeExecutionError( + msg.format(**kwargs), e, user_exception=e, original_exc_info=sys.exc_info() + ) + + +class ComputeNodeTag(Enum): + TRANSFORM = 'TRANSFORM' + INPUT_EXPECTATION = 'INPUT_EXPECTATION' + OUTPUT_EXPECTATION = 'OUTPUT_EXPECTATION' + JOIN = 'JOIN' + SOURCE = 'SOURCE' + MATERIALIZATION = 'MATERIALIZATION' + INPUTSTUB = 'INPUTSTUB' + + +EXPECTATION_VALUE_OUTPUT = 'expectation_value' +JOIN_OUTPUT = 'join_output' +MATERIALIZATION_INPUT = 'mat_input' +EXPECTATION_INPUT = 'expectation_input' + +def _execute_core_transform(context, solid_transform_fn, values_dict): + ''' + Execute the user-specified transform for the solid. Wrap in an error boundary and do + all relevant logging and metrics tracking + ''' + check.inst_param(context, 'context', ExecutionContext) + check.callable_param(solid_transform_fn, 'solid_transform_fn') + check.dict_param(values_dict, 'values_dict', key_type=str) + + error_str = 'Error occured during core transform' + with _user_code_error_boundary(context, error_str): + with time_execution_scope() as timer_result: + transformed_value = solid_transform_fn(context, values_dict) + + context.metric('core_transform_time_ms', timer_result.millis) + + return transformed_value + + +class ComputeNodeOutputHandle(namedtuple('_ComputeNodeOutputHandle', 'compute_node output_name')): + def __new__(cls, compute_node, output_name): + return super(ComputeNodeOutputHandle, cls).__new__( + cls, + compute_node=check.inst_param(compute_node, 'compute_node', ComputeNode), + output_name=check.str_param(output_name, 'output_name'), + ) + + # Make this hashable so it be a key in a dictionary + + def __str__(self): + return f'ComputeNodeOutputHandle(guid="{self.compute_node.guid}", output_name="{self.output_name}")' + + def __hash__(self): + return hash(self.compute_node.guid + self.output_name) + + def __eq__(self, other): + return ( + self.compute_node.guid == other.compute_node.guid + and self.output_name == other.output_name + ) + + +class ComputeNodeInput: + def __init__(self, name, dagster_type, prev_output_handle): + self.name = check.str_param(name, 'name') + self.dagster_type = check.inst_param(dagster_type, 'dagster_type', DagsterType) + self.prev_output_handle = check.inst_param( + prev_output_handle, + 'prev_output_handle', + ComputeNodeOutputHandle, + ) + + +class ComputeNodeOutput: + def __init__(self, name, dagster_type): + self.name = check.str_param(name, 'name') + self.dagster_type = check.inst_param(dagster_type, 'dagster_type', DagsterType) + + +class ComputeNode: + def __init__(self, friendly_name, node_inputs, node_outputs, arg_dict, compute_fn, tag, solid): + self.guid = str(uuid.uuid4()) + self.friendly_name = check.str_param(friendly_name, 'friendly_name') + self.node_inputs = check.list_param(node_inputs, 'node_inputs', of_type=ComputeNodeInput) + + node_input_dict = {} + for node_input in node_inputs: + node_input_dict[node_input.name] = node_input + self.node_input_dict = node_input_dict + + self.node_outputs = check.list_param( + node_outputs, 'node_outputs', of_type=ComputeNodeOutput + ) + + self.arg_dict = check.dict_param(arg_dict, 'arg_dict', key_type=str) + self.compute_fn = check.callable_param(compute_fn, 'compute_fn') + self.tag = check.inst_param(tag, 'tag', ComputeNodeTag) + self.solid = check.inst_param(solid, 'solid', SolidDefinition) + + def _create_compute_node_result(self, compute_result): + check.inst_param(compute_result, 'compute_result', ComputeResult) + + value = compute_result.value + + check.invariant(len(self.node_outputs) == 1) + node_output = self.node_outputs[0] + + + if not node_output.dagster_type.is_python_valid_value(value): + raise DagsterInvariantViolationError( + f'''Solid {self.solid.name} output {value} + which does not match the type for Dagster Type + {node_output.dagster_type.name}''' + ) + + return ComputeNodeResult.success_result( + compute_node=self, + tag=self.tag, + success_data=ComputeNodeSuccessData( + output_name=compute_result.output_name, + value=compute_result.value, + ), + ) + + def execute(self, context, inputs): + check.inst_param(context, 'context', ExecutionContext) + check.dict_param(inputs, 'inputs', key_type=str) + + check.invariant(len(self.solid.outputs) == 1) + + logger.debug(f'Entering execution for {self.friendly_name}') + + # do runtime type checks of inputs versus node inputs + for input_name, input_value in inputs.items(): + compute_node_input = self.node_input_dict[input_name] + if not compute_node_input.dagster_type.is_python_valid_value(input_value): + raise DagsterInvariantViolationError( + f'''Solid {self.solid.name} input {input_name} + received value {input_value} which does not match the type for Dagster type + {compute_node_input.dagster_type.name}. Compute node {self.friendly_name}''' + ) + + error_str = 'TODO error string' + + try: + with _user_code_error_boundary(context, error_str): + compute_results = list(self.compute_fn(context, inputs)) + + if not self.node_outputs: + return + + for compute_result in compute_results: + yield self._create_compute_node_result(compute_result) + + except DagsterUserCodeExecutionError as dagster_user_exception: + yield ComputeNodeResult.failure_result( + compute_node=self, + tag=self.tag, + failure_data=ComputeNodeFailureData( + dagster_user_exception=dagster_user_exception, + ), + ) + return + + + + def output_named(self, name): + check.str_param(name, 'name') + + for node_output in self.node_outputs: + if node_output.name == name: + return node_output + + check.failed(f'output {name} not found') + + +class ComputeNodeSuccessData(namedtuple('_ComputeNodeSuccessData', 'output_name value')): + def __new__(cls, output_name, value): + return super(ComputeNodeSuccessData, cls).__new__( + cls, + output_name=check.str_param(output_name, 'output_name'), + value=value, + ) + +class ComputeNodeFailureData(namedtuple('_ComputeNodeFailureData', 'dagster_user_exception')): + def __new__(cls, dagster_user_exception): + return super(ComputeNodeFailureData, cls).__new__( + cls, + dagster_user_exception=check.inst_param( + dagster_user_exception, + 'dagster_user_exception', + DagsterUserCodeExecutionError, + ), + ) + +class ComputeNodeResult( + namedtuple( + '_ComputeNodeResult', + 'success compute_node tag success_data failure_data ', + ) +): + + @staticmethod + def success_result(compute_node, tag, success_data): + return ComputeNodeResult( + success=True, + compute_node=check.inst_param(compute_node, 'compute_node', ComputeNode), + tag=check.inst_param(tag, 'tag', ComputeNodeTag), + success_data=check.inst_param(success_data, 'success_data', ComputeNodeSuccessData), + failure_data=None, + ) + + @staticmethod + def failure_result(compute_node, tag, failure_data): + return ComputeNodeResult( + success=False, + compute_node=check.inst_param(compute_node, 'compute_node', ComputeNode), + tag=check.inst_param(tag, 'tag', ComputeNodeTag), + success_data=None, + failure_data=check.inst_param(failure_data, 'failure_data', ComputeNodeFailureData), + ) + + +def execute_compute_nodes(context, compute_nodes): + check.inst_param(context, 'context', ExecutionContext) + check.list_param(compute_nodes, 'compute_nodes', of_type=ComputeNode) + + intermediate_results = {} + for compute_node in compute_nodes: + input_values = {} + for node_input in compute_node.node_inputs: + prev_output_handle = node_input.prev_output_handle + if prev_output_handle not in intermediate_results: + check.failed( + f'Could not find handle {prev_output_handle} in results. ' + \ + f'current node: {compute_node.friendly_name}' + ) + input_value = intermediate_results[prev_output_handle].success_data.value + input_values[node_input.name] = input_value + + for result in compute_node.execute(context, input_values): + check.invariant(isinstance(result, ComputeNodeResult)) + yield result + output_handle = create_cn_output_handle(compute_node, result.success_data.output_name) + intermediate_results[output_handle] = result + + +def _yieldify(sync_compute_fn): + def _wrap(context, inputs): + yield sync_compute_fn(context, inputs) + return _wrap + +class SingleSyncOutputComputeNode(ComputeNode): + def __init__(self, *, sync_compute_fn, **kwargs): + super().__init__(compute_fn=_yieldify(sync_compute_fn), **kwargs) + + +class ComputeResult(namedtuple('_ComputeResult', 'output_name value')): + def __new__(cls, output_name, value): + return super(ComputeResult, cls).__new__( + cls, + check.str_param(output_name, 'output_name'), + value, + ) + +def create_compute_node_from_source_config(solid, input_name, source_config): + check.inst_param(solid, 'solid', SolidDefinition) + check.str_param(input_name, 'input_name') + check.inst_param(source_config, 'source_config', config.Source) + + input_def = solid.input_def_named(input_name) + source_def = input_def.source_of_type(source_config.name) + + error_context_str = 'source type {source}'.format(source=source_def.source_type) + + arg_dict = validate_args( + source_def.argument_def_dict, + source_config.args, + error_context_str, + ) + + return SingleSyncOutputComputeNode( + friendly_name=f'{solid.name}.{input_name}.source.{source_config.name}', + node_inputs=[], + node_outputs=[ + ComputeNodeOutput( + name=SOURCE_OUTPUT, + dagster_type=input_def.dagster_type, + ), + ], + arg_dict=arg_dict, + sync_compute_fn=lambda context, _inputs: ComputeResult( + output_name=SOURCE_OUTPUT, + value=source_def.source_fn(context, arg_dict) + ), + tag=ComputeNodeTag.SOURCE, + solid=solid, + ) + +class LogicalSolidOutput(namedtuple('_LogicalSolidOutput', 'solid_name output_name')): + def __new__(cls, solid_name, output_name): + return super(LogicalSolidOutput, cls).__new__( + cls, + check.str_param(solid_name, 'solid_name'), + check.str_param(output_name, 'output_name'), + ) + +class LogicalSolidInput(namedtuple('_SourceCnDictKey', 'solid_name input_name')): + def __new__(cls, solid_name, input_name): + return super(LogicalSolidInput, cls).__new__( + cls, + check.str_param(solid_name, 'solid_name'), + check.str_param(input_name, 'input_name'), + ) + + +def create_source_compute_node_dict_from_environment(pipeline, environment): + check.inst_param(pipeline, 'pipeline', PipelineDefinition) + check.inst_param(environment, 'environment', config.Environment) + + source_cn_dict = {} + for solid_name, sources_by_input in environment.sources.items(): + solid = pipeline.solid_named(solid_name) + for input_name, source_config in sources_by_input.items(): + compute_node = create_compute_node_from_source_config(solid, input_name, source_config) + source_cn_dict[LogicalSolidInput(solid_name, input_name)] = compute_node + + return source_cn_dict + + +def get_lambda(output_name, value): + return lambda _context, _args: ComputeResult(output_name, value) + +SOURCE_OUTPUT = 'source_output' + +def create_source_compute_node_dict_from_input_values(pipeline, input_values): + check.inst_param(pipeline, 'pipeline', PipelineDefinition) + check.dict_param(input_values, 'input_values', key_type=str) + + source_cn_dict = {} + for solid_name, sources_by_input in input_values.items(): + for input_name, input_value in sources_by_input.items(): + source_cn_dict[LogicalSolidInput(solid_name, input_name)] = SingleSyncOutputComputeNode( + friendly_name=f'{solid_name}.{input_name}.stub', + node_inputs=[], + # This is just a stub of a pre-existing value, so we are not + # going to make any type guarantees + node_outputs=[ComputeNodeOutput(SOURCE_OUTPUT, Any)], + arg_dict={}, + sync_compute_fn=get_lambda(SOURCE_OUTPUT, input_value), + tag=ComputeNodeTag.INPUTSTUB, + solid=pipeline.solid_named(solid_name), + ) + + return source_cn_dict + + +def print_graph(graph, printer=print): + check.inst_param(graph, 'graph', ComputeNodeGraph) + printer = IndentingPrinter(printer=printer) + + for node in graph.topological_nodes(): + with printer.with_indent(f'Node {node.friendly_name} Id: {node.guid}'): + for node_input in node.node_inputs: + with printer.with_indent(f'Input: {node_input.name}'): + printer.line(f'Type: {node_input.dagster_type.name}') + printer.line(f'From: {node_input.prev_output_handle}') + for node_output in node.node_outputs: + with printer.with_indent(f'Output: {node_output.name}'): + printer.line(f'Type: {node_output.dagster_type.name}') + + +class ComputeNodeGraph: + def __init__(self, cn_dict, deps): + self.cn_dict = cn_dict + self.deps = deps + self.nodes = list(cn_dict.values()) + + def topological_nodes(self): + cn_guids_sorted = toposort.toposort_flatten(self.deps) + for cn_guid in cn_guids_sorted: + yield self.cn_dict[cn_guid] + + +def create_compute_node_graph_from_env(pipeline, env): + import dagster.core.execution + if isinstance(env, dagster.core.execution.ConfigEnv): + return create_compute_node_graph_from_environment( + pipeline, + env.environment, + ) + elif isinstance(env, dagster.core.execution.InMemoryEnv): + return create_compute_node_graph_from_input_values( + pipeline, + env.input_values, + from_solids=env.from_solids, + through_solids=env.through_solids, + evaluate_expectations=env.evaluate_expectations, + ) + else: + check.not_implemented('unsupported') + + +def create_compute_node_graph_from_input_values( + pipeline, + input_values, + from_solids=None, + through_solids=None, + evaluate_expectations=True, +): + source_cn_dict = create_source_compute_node_dict_from_input_values(pipeline, input_values) + return create_compute_node_graph_from_source_dict( + pipeline, + source_cn_dict, + from_solids=from_solids, + through_solids=through_solids, + evaluate_expectations=evaluate_expectations, + ) + + +def create_compute_node_graph_from_environment(pipeline, environment): + source_cn_dict = create_source_compute_node_dict_from_environment(pipeline, environment) + + return create_compute_node_graph_from_source_dict( + pipeline, + source_cn_dict, + materializations=environment.materializations, + from_solids=environment.execution.from_solids, + through_solids=environment.execution.through_solids, + evaluate_expectations=environment.expectations.evaluate, + ) + + + +def create_expectation_cn(solid, expectation_def, friendly_name, tag, prev_node_output_handle): + check.inst_param(solid, 'solid', SolidDefinition) + check.inst_param(expectation_def, 'input_expct_def', ExpectationDefinition) + check.inst_param(prev_node_output_handle, 'prev_node_output_handle', ComputeNodeOutputHandle) + + return SingleSyncOutputComputeNode( + friendly_name=friendly_name, + node_inputs=[ + ComputeNodeInput( + name=EXPECTATION_INPUT, + dagster_type=solid.output.dagster_type, + prev_output_handle=prev_node_output_handle, + ) + ], + node_outputs=[ + ComputeNodeOutput( + name=EXPECTATION_VALUE_OUTPUT, + dagster_type=solid.output.dagster_type + ), + ], + arg_dict={}, + sync_compute_fn=_create_expectation_lambda( + solid, + expectation_def, + EXPECTATION_VALUE_OUTPUT, + ), + tag=tag, + solid=solid + ) + + +ExpectationsComputeNodeGraph = namedtuple( + 'ExpectationsComputeNodeGraph', + 'nodes terminal_cn_output_handle', +) + + +def create_expectations_cn_graph(solid, inout_def, prev_node_output_handle, tag): + check.inst_param(solid, 'solid', SolidDefinition) + check.inst_param(inout_def, 'inout_def', (InputDefinition, OutputDefinition)) + check.inst_param(prev_node_output_handle, 'prev_node_output_handle', ComputeNodeOutputHandle) + check.inst_param(tag, 'tag', ComputeNodeTag) + + compute_nodes = [] + input_expect_nodes = [] + for expectation_def in inout_def.expectations: + expect_compute_node = create_expectation_cn( + solid=solid, + expectation_def=expectation_def, + friendly_name=f'{solid.name}.{inout_def.name}.expectation.{expectation_def.name}', + tag=tag, + prev_node_output_handle=prev_node_output_handle, + ) + input_expect_nodes.append(expect_compute_node) + compute_nodes.append(expect_compute_node) + + join_cn = _create_join_node(solid, input_expect_nodes, EXPECTATION_VALUE_OUTPUT) + + output_name = join_cn.node_outputs[0].name + return ExpectationsComputeNodeGraph( + compute_nodes + [join_cn], + create_cn_output_handle(join_cn, output_name), + ) + +LogicalSolidOutput = namedtuple('LogicalSolidOutput', 'solid_name output_name') + +def _prev_node_handle(dep_structure, solid, input_def, source_cn_dict, logical_output_mapper): + check.inst_param(dep_structure, 'dep_structure', PipelineDependencyStructure) + check.inst_param(solid, 'solid', SolidDefinition) + check.inst_param(input_def, 'input_def', InputDefinition) + check.dict_param( + source_cn_dict, + 'source_cn_dict', + key_type=LogicalSolidInput, + value_type=ComputeNode, + ) + + check.inst_param(logical_output_mapper, 'mapper', LogicalSolidOutputMapper) + + logical_solid_input = LogicalSolidInput(solid.name, input_def.name) + + if logical_solid_input in source_cn_dict: + return create_cn_output_handle(source_cn_dict[logical_solid_input], SOURCE_OUTPUT) + else: + check.invariant(dep_structure.input_has_dep(logical_solid_input)) + logical_solid_output = dep_structure.get_dep(logical_solid_input) + return logical_output_mapper.get_handle_for_logical_output(logical_solid_output) + +def create_cn_output_handle(compute_node, cn_output_name): + check.inst_param(compute_node, 'compute_node', ComputeNode) + check.str_param(cn_output_name, 'cn_output_name') + return ComputeNodeOutputHandle(compute_node, cn_output_name) + + +class PipelineDependencyStructure: + def __init__(self, pipeline): + check.inst_param(pipeline, 'pipeline', PipelineDefinition) + + edges = {} + for solid in pipeline.solids: + for input_def in solid.inputs: + if pipeline.dependency_structure.has_dep(solid.name, input_def.name): + logical_solid_input = LogicalSolidInput(solid.name, input_def.name) + + dep_target = pipeline.dependency_structure.get_dep_target( + solid.name, + input_def.name, + ) + + logical_solid_output = LogicalSolidOutput( + dep_target.solid_name, + dep_target.output_name, + ) + + edges[logical_solid_input] = logical_solid_output + + self.edges = edges + + def input_has_dep(self, logical_solid_input): + check.inst_param(logical_solid_input, 'logical_solid_input', LogicalSolidInput) + return logical_solid_input in self.edges + + def get_dep(self, logical_solid_input): + check.inst_param(logical_solid_input, 'logical_solid_input', LogicalSolidInput) + return self.edges[logical_solid_input] + + +class LogicalSolidOutputMapper: + def __init__(self): + self._output_handles = {} + + def set_mapping(self, logical_solid_output, cn_output_handle): + check.inst_param(logical_solid_output, 'logical_solid_output', LogicalSolidOutput) + check.inst_param(cn_output_handle, 'cn_output_handle', ComputeNodeOutputHandle) + self._output_handles[logical_solid_output] = cn_output_handle + + def get_handle_for_logical_output(self, logical_solid_output): + check.inst_param(logical_solid_output, 'logical_solid_output', LogicalSolidOutput) + return self._output_handles[logical_solid_output] + +def create_compute_node_graph_from_source_dict( + pipeline, + source_cn_dict, + materializations=None, + from_solids=None, + through_solids=None, + evaluate_expectations=True, +): + + check.inst_param(pipeline, 'pipeline', PipelineDefinition) + + check.dict_param( + source_cn_dict, + 'source_cn_dict', + key_type=LogicalSolidInput, + value_type=ComputeNode, + ) + + materializations = check.opt_list_param( + materializations, + 'materializations', + of_type=config.Materialization, + ) + + dep_structure = PipelineDependencyStructure(pipeline) + + check.bool_param(evaluate_expectations, 'evaluate_expectations') + + compute_nodes = list(source_cn_dict.values()) + + logical_output_mapper = LogicalSolidOutputMapper() + + subgraph = create_subgraph( + pipeline, + check.opt_list_param(from_solids, 'from_solid', of_type=str), + check.opt_list_param(through_solids, 'through_solid', of_type=str), + ) + + for topo_solid in subgraph.topological_solids: + cn_inputs = [] + + for input_def in topo_solid.inputs: + prev_cn_output_handle = _prev_node_handle( + dep_structure, + topo_solid, + input_def, + source_cn_dict, + logical_output_mapper, + ) + + check.inst(prev_cn_output_handle, ComputeNodeOutputHandle) + + # jam in input expectations here + + if evaluate_expectations and input_def.expectations: + expectations_graph = create_expectations_cn_graph( + topo_solid, + input_def, + prev_cn_output_handle, + tag=ComputeNodeTag.INPUT_EXPECTATION, + ) + compute_nodes = compute_nodes + expectations_graph.nodes + + check.inst(expectations_graph.terminal_cn_output_handle, ComputeNodeOutputHandle) + + cn_output_handle = expectations_graph.terminal_cn_output_handle + else: + cn_output_handle = prev_cn_output_handle + + cn_inputs.append( + ComputeNodeInput( + input_def.name, input_def.dagster_type, cn_output_handle + ) + ) + + solid_transform_cn = create_compute_node_from_solid_transform(topo_solid, cn_inputs) + + for output_def in topo_solid.outputs: + if evaluate_expectations and topo_solid.output.expectations: + expectations_graph = create_expectations_cn_graph( + topo_solid, + output_def, + create_cn_output_handle(solid_transform_cn, output_def.name), + tag=ComputeNodeTag.OUTPUT_EXPECTATION + ) + compute_nodes = compute_nodes + expectations_graph.nodes + logical_output_mapper.set_mapping( + LogicalSolidOutput(topo_solid.name, output_def.name), + expectations_graph.terminal_cn_output_handle, + ) + else: + logical_output_mapper.set_mapping( + LogicalSolidOutput(topo_solid.name, output_def.name), + create_cn_output_handle(solid_transform_cn, output_def.name), + ) + + + compute_nodes.append(solid_transform_cn) + + for materialization in materializations: + mat_cn = _construct_materialization_cn( + pipeline, + materialization, + logical_output_mapper.get_handle_for_logical_output( + LogicalSolidOutput( + materialization.solid, + materialization.output_name + ) + ), + ) + compute_nodes.append(mat_cn) + + cn_dict = {} + for cn in compute_nodes: + cn_dict[cn.guid] = cn + + deps = defaultdict(set) + + for cn in compute_nodes: + deps[cn.guid] = set() + for cn_input in cn.node_inputs: + deps[cn.guid].add(cn_input.prev_output_handle.compute_node.guid) + + return ComputeNodeGraph(cn_dict, deps) + + +def _create_join_node(solid, prev_nodes, prev_output_name): + check.inst_param(solid, 'solid', SolidDefinition) + check.list_param(prev_nodes, 'prev_nodes', of_type=ComputeNode) + check.invariant(len(prev_nodes) > 0) + check.str_param(prev_output_name, 'output_name') + + node_inputs = [] + seen_dagster_type = None + for prev_node in prev_nodes: + prev_node_output = prev_node.output_named(prev_output_name) + + if seen_dagster_type is None: + seen_dagster_type = prev_node_output.dagster_type + else: + check.invariant(seen_dagster_type == prev_node_output.dagster_type) + + output_handle = create_cn_output_handle(prev_node, prev_output_name) + + node_inputs.append( + ComputeNodeInput(prev_node.guid, prev_node_output.dagster_type, output_handle) + ) + + return SingleSyncOutputComputeNode( + friendly_name='join', + node_inputs=node_inputs, + node_outputs=[ComputeNodeOutput(JOIN_OUTPUT, seen_dagster_type)], + arg_dict={}, + sync_compute_fn=_create_join_lambda, + tag=ComputeNodeTag.JOIN, + solid=solid, + ) + + +ExpectationExecutionInfo = namedtuple('ExpectationExecutionInfo', 'solid expectation_def') + +def _create_join_lambda(_context, inputs): + return ComputeResult(output_name=JOIN_OUTPUT, value=list(inputs.values())[0]) + +def _create_expectation_lambda(solid, expectation_def, output_name): + check.inst_param(solid, 'solid', SolidDefinition) + check.inst_param(expectation_def, 'expectations_def', ExpectationDefinition) + check.str_param(output_name, 'output_name') + + def _do_expectation(context, inputs): + expt_result = expectation_def.expectation_fn( + context, + ExpectationExecutionInfo(solid, expectation_def), + inputs[EXPECTATION_INPUT], + ) + if expt_result.success: + return ComputeResult(output_name=output_name, value=inputs[EXPECTATION_INPUT]) + + raise DagsterExpectationFailedError(None) # for now + + return _do_expectation + + +def _construct_materialization_cn(pipeline, materialization, prev_output_handle): + check.inst_param(pipeline, 'pipeline', PipelineDefinition) + check.inst_param(materialization, 'materialization', config.Materialization) + check.inst_param(prev_output_handle, 'prev_output_handle', ComputeNodeOutputHandle) + + solid = pipeline.solid_named(materialization.solid) + mat_def = solid.output.materialization_of_type(materialization.name) + + error_context_str = 'source type {mat}'.format(mat=mat_def.name) + + arg_dict = validate_args( + mat_def.argument_def_dict, + materialization.args, + error_context_str, + ) + + return SingleSyncOutputComputeNode( + friendly_name=f'{solid.name}.materialization.{mat_def.name}', + node_inputs=[ + ComputeNodeInput( + name=MATERIALIZATION_INPUT, + dagster_type=solid.output.dagster_type, + prev_output_handle=prev_output_handle, + ) + ], + node_outputs=[], + arg_dict=arg_dict, + sync_compute_fn=_create_materialization_lambda(mat_def, materialization, 'TODO_REMOVE'), + tag=ComputeNodeTag.MATERIALIZATION, + solid=solid, + ) + + +def _create_materialization_lambda(mat_def, materialization, output_name): + check.inst_param(mat_def, 'mat_def', MaterializationDefinition) + check.inst_param(materialization, 'materialization', config.Materialization) + check.str_param(output_name, 'output_name') + + return lambda context, inputs: ComputeResult( + output_name=output_name, + value=mat_def.materialization_fn( + context, + materialization.args, + inputs[MATERIALIZATION_INPUT], + ), + ) + +def create_compute_node_from_solid_transform(solid, node_inputs): + check.inst_param(solid, 'solid', SolidDefinition) + check.list_param(node_inputs, 'node_inputs', of_type=ComputeNodeInput) + check.invariant(len(solid.outputs) == 1) + + output_name = solid.outputs[0].name + + return SingleSyncOutputComputeNode( + friendly_name=f'{solid.name}.transform', + node_inputs=node_inputs, + node_outputs=[ + ComputeNodeOutput(name=output.name, dagster_type=output.dagster_type) + for output in solid.outputs + ], + arg_dict={}, + sync_compute_fn=lambda context, inputs: ComputeResult( + output_name=output_name, + value=_execute_core_transform( + context, + solid.transform_fn, + inputs, + ), + ), + tag=ComputeNodeTag.TRANSFORM, + solid=solid, + ) diff --git a/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py b/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py new file mode 100644 index 0000000000000..a61da23de13eb --- /dev/null +++ b/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py @@ -0,0 +1,83 @@ +from dagster import ( + ExecutionContext, + InputDefinition, + PipelineContextDefinition, + PipelineDefinition, + config, + solid, + source, + OutputDefinition, +) + +from dagster.core.compute_nodes import create_compute_node_graph_from_environment + + +@solid(name='noop', inputs=[], output=OutputDefinition()) +def noop_solid(): + return 'foo' + + +def silencing_default_context(): + return { + 'default': + PipelineContextDefinition( + argument_def_dict={}, + context_fn=lambda _pipeline, _args: ExecutionContext(), + ) + } + + +@source() +def load_value_source(): + return 'value' + + +@solid( + inputs=[InputDefinition(name='some_input', sources=[load_value_source])], + output=OutputDefinition() +) +def solid_with_source(some_input): + return some_input + + +def silencing_pipeline(solids): + return PipelineDefinition(solids=solids, context_definitions=silencing_default_context()) + + +def test_compute_noop_node(): + pipeline = silencing_pipeline(solids=[ + noop_solid, + ]) + + environment = config.Environment(sources={}) + + compute_node_graph = create_compute_node_graph_from_environment(pipeline, environment) + + assert len(compute_node_graph.nodes) == 1 + + context = ExecutionContext() + + outputs = list(compute_node_graph.nodes[0].execute(context, {})) + + assert outputs[0].success_data.value == 'foo' + + +def test_compute_node_with_source(): + pipeline = silencing_pipeline(solids=[solid_with_source]) + environment = config.Environment( + sources={ + 'solid_with_source': { + 'some_input': config.Source(name='load_value_source', args={}) + } + } + ) + + compute_node_graph = create_compute_node_graph_from_environment(pipeline, environment) + assert len(compute_node_graph.nodes) == 2 + + context = ExecutionContext() + + node_list = list(compute_node_graph.topological_nodes()) + + assert list(node_list[0].execute(context, {}))[0].success_data.value == 'value' + assert list(node_list[1].execute(context, {'some_input': 'bar'}))[0].success_data.value == 'bar' diff --git a/python_modules/dagster/dagster/core/core_tests/test_core_execution.py b/python_modules/dagster/dagster/core/core_tests/test_core_execution.py deleted file mode 100644 index 5be8e41ea80d6..0000000000000 --- a/python_modules/dagster/dagster/core/core_tests/test_core_execution.py +++ /dev/null @@ -1,208 +0,0 @@ -import pytest - -from dagster.core import types - -from dagster.core.definitions import ( - SolidDefinition, - OutputDefinition, - MaterializationDefinition, - SourceDefinition, - ArgumentDefinition, -) - -from dagster.core.execution import ( - _execute_core_transform, - DagsterTypeError, - ExecutionContext, - _execute_materialization, - _read_source, -) - -from dagster.utils.compatability import create_custom_source_input - - -def create_test_context(): - return ExecutionContext() - - -def _read_new_single_source_input(context, new_input, arg_dict): - assert len(new_input.sources) == 1 - return _read_source(context, new_input.sources[0], arg_dict) - - -def test_read_source(): - expected_output = [{'data_key': 'data_value'}] - some_input = create_custom_source_input( - name='some_input', - source_fn=lambda context, arg_dict: expected_output, - argument_def_dict={} - ) - - output = _read_source(create_test_context(), some_input.sources[0], {}) - - assert output == expected_output - - -def test_source_arg_mismiatch(): - extra_arg_source = create_custom_source_input( - name='some_input', source_fn=lambda context, arg_dict: [], argument_def_dict={} - ).sources[0] - - with pytest.raises(DagsterTypeError): - _read_source(create_test_context(), extra_arg_source, {'extra_arg': None}) - - some_input_with_arg = create_custom_source_input( - name='some_input_with_arg', - source_fn=lambda context, arg_dict: [], - argument_def_dict={'in_arg': ArgumentDefinition(types.String)}, - ) - - with pytest.raises(DagsterTypeError): - _read_new_single_source_input(create_test_context(), some_input_with_arg, {}) - - -def test_source_arg_type_mismatch(): - some_input_with_arg = create_custom_source_input( - name='some_input_with_arg', - source_fn=lambda context, arg_dict: [], - argument_def_dict={'in_arg': ArgumentDefinition(types.String)}, - ) - - with pytest.raises(DagsterTypeError): - _read_new_single_source_input(create_test_context(), some_input_with_arg, {'in_arg': 1}) - - -def test_source_int_type(): - int_arg_source = SourceDefinition( - source_type='SOMETHING', - source_fn=lambda _context, _args: True, - argument_def_dict={'an_int': ArgumentDefinition(types.Int)}, - ) - assert _read_source(create_test_context(), int_arg_source, {'an_int': 0}) - assert _read_source(create_test_context(), int_arg_source, {'an_int': 1}) - assert _read_source(create_test_context(), int_arg_source, {'an_int': None}) - - with pytest.raises(DagsterTypeError): - _read_source(create_test_context(), int_arg_source, {'an_int': 'not_an_int'}) - - -def test_source_bool_type(): - bool_arg_source = SourceDefinition( - source_type='SOMETHING', - source_fn=lambda _context, _args: True, - argument_def_dict={'an_bool': ArgumentDefinition(types.Bool)}, - ) - - assert _read_source(create_test_context(), bool_arg_source, {'an_bool': True}) - assert _read_source(create_test_context(), bool_arg_source, {'an_bool': False}) - assert _read_source(create_test_context(), bool_arg_source, {'an_bool': None}) - - with pytest.raises(DagsterTypeError): - _read_source(create_test_context(), bool_arg_source, {'an_bool': 'not_an_bool'}) - - with pytest.raises(DagsterTypeError): - _read_source(create_test_context(), bool_arg_source, {'an_bool': 0}) - - -def noop_output(): - return OutputDefinition() - - -def test_materialize_output(): - some_input = create_custom_source_input( - name='some_input', - source_fn=lambda context, arg_dict: [{'data_key': 'data_value'}], - argument_def_dict={}, - ) - - def tranform_fn_inst(_context, args): - args['some_input'][0]['data_key'] = 'new_value' - return args['some_input'] - - single_solid = SolidDefinition( - name='some_node', - inputs=[some_input], - transform_fn=tranform_fn_inst, - output=noop_output(), - ) - - value = _read_new_single_source_input(create_test_context(), some_input, {}) - - output = _execute_core_transform( - create_test_context(), - single_solid.transform_fn, - {'some_input': value}, - ) - - assert output == [{'data_key': 'new_value'}] - - -def test_materialize_input_with_args(): - some_input = create_custom_source_input( - name='some_input', - source_fn=lambda context, arg_dict: [{'key': arg_dict['str_arg']}], - argument_def_dict={'str_arg': ArgumentDefinition(types.String)}, - ) - - output = _read_new_single_source_input( - create_test_context(), some_input, {'str_arg': 'passed_value'} - ) - expected_output = [{'key': 'passed_value'}] - assert output == expected_output - - -def test_execute_output_with_args(): - test_output = {} - - def materialization_fn_inst(context, arg_dict, value): - assert isinstance(context, ExecutionContext) - assert isinstance(arg_dict, dict) - test_output['thedata'] = value - test_output['thearg'] = arg_dict['out_arg'] - - materialization = MaterializationDefinition( - name='CUSTOM', - materialization_fn=materialization_fn_inst, - argument_def_dict={'out_arg': ArgumentDefinition(types.String)} - ) - - _execute_materialization( - create_test_context(), materialization, {'out_arg': 'the_out_arg'}, [{ - 'key': 'value' - }] - ) - - -def test_execute_materialization_arg_mismatch(): - materialization = MaterializationDefinition( - name='CUSTOM', - materialization_fn=lambda out, dict: [], - argument_def_dict={'out_arg': ArgumentDefinition(types.String)} - ) - - with pytest.raises(DagsterTypeError): - _execute_materialization(create_test_context(), materialization, arg_dict={}, value=[{}]) - - with pytest.raises(DagsterTypeError): - _execute_materialization( - create_test_context(), - materialization, - arg_dict={ - 'out_arg': 'foo', - 'extra_arg': 'bar' - }, - value=[{}] - ) - - -def test_execute_materialization_arg_type_mismatch(): - custom_output = MaterializationDefinition( - name='CUSTOM', - materialization_fn=lambda out, dict: [], - argument_def_dict={'out_arg': ArgumentDefinition(types.String)} - ) - - with pytest.raises(DagsterTypeError): - _execute_materialization( - create_test_context(), custom_output, arg_dict={'out_arg': 1}, value=[{}] - ) diff --git a/python_modules/dagster/dagster/core/core_tests/test_error_handling.py b/python_modules/dagster/dagster/core/core_tests/test_error_handling.py deleted file mode 100644 index 6f53f94b1aaf8..0000000000000 --- a/python_modules/dagster/dagster/core/core_tests/test_error_handling.py +++ /dev/null @@ -1,59 +0,0 @@ -import pytest - -from dagster import check - -from dagster.utils.compatability import create_custom_source_input - -from dagster.core.execution import ( - _read_source, - _execute_core_transform, - _execute_materialization, - ExecutionContext, - MaterializationDefinition, -) - -from dagster.core.errors import DagsterUserCodeExecutionError - - -def create_test_context(): - return ExecutionContext() - - -def test_basic_source_runtime_error_handling(): - def source_fn_inst(_context, _arg_dict): - raise Exception('a user error') - - erroring_input = create_custom_source_input( - name='some_input', source_fn=source_fn_inst, argument_def_dict={} - ) - - with pytest.raises(DagsterUserCodeExecutionError): - _read_source(create_test_context(), erroring_input.sources[0], {}) - - -def test_basic_core_transform_error_handling(): - def transform_fn(an_input): - check.str_param(an_input, 'an_input') - raise Exception('exception during core transform') - - with pytest.raises(DagsterUserCodeExecutionError): - _execute_core_transform( - create_test_context(), - transform_fn, - {'an_input': 'value'}, - ) - - -def test_basic_materialization_runtime_error_handling(): - def materialization_fn_inst(_data, arg_dict): - assert arg_dict == {} - raise Exception('error during output') - - materialization_def = MaterializationDefinition( - name='CUSTOM', materialization_fn=materialization_fn_inst, argument_def_dict={} - ) - - with pytest.raises(DagsterUserCodeExecutionError): - _execute_materialization( - create_test_context(), materialization_def, arg_dict={}, value='whatever' - ) diff --git a/python_modules/dagster/dagster/core/core_tests/test_execute_solid.py b/python_modules/dagster/dagster/core/core_tests/test_execute_solid.py index 2e75976641977..839d8bb848544 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_execute_solid.py +++ b/python_modules/dagster/dagster/core/core_tests/test_execute_solid.py @@ -13,7 +13,7 @@ from dagster.core.execution import ( output_single_solid, - SolidExecutionResult, + ExecutionResultBase, DagsterExecutionFailureReason, ExecutionContext, execute_single_solid, @@ -153,9 +153,6 @@ def materialization_fn(context, arg_dict, data): arg_dict={} ) - if output_result.exception: - raise output_result.exception - assert output_result.success assert 'called' in output_events @@ -179,9 +176,6 @@ def test_execute_solid_with_args(): arg_dict={}, ) - if result.exception: - raise result.exception - assert result.success assert test_output['thedata'][0]['key'] == 'an_input_arg' @@ -199,9 +193,9 @@ def test_execute_solid_with_failed_input_expectation_non_throwing(): throw_on_error=False, ) - assert isinstance(solid_execution_result, SolidExecutionResult) + assert isinstance(solid_execution_result, ExecutionResultBase) assert solid_execution_result.success is False - assert solid_execution_result.reason == DagsterExecutionFailureReason.EXPECTATION_FAILURE + # assert solid_execution_result.reason == DagsterExecutionFailureReason.EXPECTATION_FAILURE def test_execute_solid_with_failed_input_expectation_throwing(): @@ -254,9 +248,9 @@ def test_execute_solid_with_failed_output_expectation_non_throwing(): throw_on_error=False ) - assert isinstance(solid_execution_result, SolidExecutionResult) + assert isinstance(solid_execution_result, ExecutionResultBase) assert solid_execution_result.success is False - assert solid_execution_result.reason == DagsterExecutionFailureReason.EXPECTATION_FAILURE + # assert solid_execution_result.reason == DagsterExecutionFailureReason.EXPECTATION_FAILURE def test_execute_solid_with_failed_output_expectation_throwing(): diff --git a/python_modules/dagster/dagster/core/core_tests/test_input_expectations.py b/python_modules/dagster/dagster/core/core_tests/test_input_expectations.py deleted file mode 100644 index 446f45d181f63..0000000000000 --- a/python_modules/dagster/dagster/core/core_tests/test_input_expectations.py +++ /dev/null @@ -1,100 +0,0 @@ -import pytest - -import dagster - -from dagster.core.definitions import ( - ExpectationDefinition, - ExpectationResult, - ArgumentDefinition, -) -from dagster.core.execution import ( - _execute_input_expectation, DagsterUserCodeExecutionError, ExecutionContext -) -from dagster.core import types - -from dagster.utils.compatability import create_custom_source_input - - -def create_test_context(): - return ExecutionContext() - - -def test_basic_failing_input_expectation(): - def failing_expectation(_context, _info, _some_input): - return ExpectationResult(success=False, message='Some failure') - - some_input = create_custom_source_input( - name='some_input', - source_fn=lambda arg_dict: [{'key': arg_dict['str_arg']}], - argument_def_dict={'str_arg': ArgumentDefinition(types.String)}, - expectations=[ - ExpectationDefinition(name='failing', expectation_fn=failing_expectation) - ] - ) - - expectation_def = some_input.expectations[0] - - info = create_dummy_info(expectation_def) - - result = _execute_input_expectation(create_test_context(), info, 'some_value') - - assert isinstance(result, ExpectationResult) - assert not result.success - assert result.message == 'Some failure' - - -def create_dummy_info(expectation_def): - info = dagster.InputExpectationInfo( - solid=dagster.SolidDefinition( - name='dummy', - inputs=[], - transform_fn=lambda _context, _args: None, - output=dagster.OutputDefinition(), - ), - input_def=dagster.InputDefinition(name='dummy_input'), - expectation_def=expectation_def, - ) - return info - - -def test_basic_passing_input_expectation(): - def passing_expectation(_context, _info, _some_input): - return ExpectationResult(success=True, message='yayayaya') - - some_input = create_custom_source_input( - name='some_input', - source_fn=lambda arg_dict: [{'key': arg_dict['str_arg']}], - argument_def_dict={'str_arg': ArgumentDefinition(types.String)}, - expectations=[ - ExpectationDefinition(name='passing', expectation_fn=passing_expectation) - ] - ) - - result = _execute_input_expectation( - create_test_context(), create_dummy_info(some_input.expectations[0]), 'some_value' - ) - - assert isinstance(result, ExpectationResult) - assert result.success - assert result.message == 'yayayaya' - - -def test_input_expectation_user_error(): - def throwing(_context, _info, _something): - raise Exception('nope') - - failing_during_expectation_input = create_custom_source_input( - name='failing_during_expectation', - source_fn=lambda arg_dict: [{'key': arg_dict['str_arg']}], - argument_def_dict={'str_arg': ArgumentDefinition(types.String)}, - expectations=[ - ExpectationDefinition(name='passing', expectation_fn=throwing) - ] - ) - - with pytest.raises(DagsterUserCodeExecutionError): - _execute_input_expectation( - create_test_context(), - create_dummy_info(failing_during_expectation_input.expectations[0]), - 'some_value', - ) diff --git a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py index 759063cd84b1c..acb1ad85d1bd4 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py +++ b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py @@ -6,6 +6,7 @@ InputDefinition, SourceDefinition, ArgumentDefinition, + DependencyDefinition, ) from dagster.core.execution import (ExecutionContext, execute_single_solid, execute_pipeline) @@ -120,14 +121,24 @@ def test_execute_dep_solid_different_input_name(): inputs=[ InputDefinition( name='a_dependency', - depends_on=first_solid, + + # depends_on=first_solid, ), ], transform_fn=lambda context, args: args['a_dependency'] + args['a_dependency'], output=dagster.OutputDefinition(), ) - pipeline = dagster.PipelineDefinition(solids=[first_solid, second_solid]) + pipeline = dagster.PipelineDefinition( + solids=[first_solid, second_solid], + dependencies=[ + DependencyDefinition( + from_solid='second_solid', + from_input='a_dependency', + to_solid='first_solid', + ) + ] + ) result = dagster.execute_pipeline( pipeline, environment=config.Environment( @@ -213,7 +224,6 @@ def s_fn(arg_dict, executed, key): 'table_one': config.Source(name='TABLE', args={'name': 'table_one_instance'}), }, 'table_two': { - 'table_one': config.Source(name='TABLE', args={'name': 'table_one_instance'}), 'table_two': config.Source(name='TABLE', args={'name': 'table_two_instance'}), }, } @@ -240,7 +250,12 @@ def s_fn(arg_dict, executed, key): executed['s2_t2_source'] = False second_only_env = config.Environment( - sources=sources, + sources={ + 'table_two': { + 'table_one': config.Source(name='TABLE', args={'name': 'table_one_instance'}), + 'table_two': config.Source(name='TABLE', args={'name': 'table_two_instance'}), + }, + }, execution=config.Execution(from_solids=['table_two']), ) @@ -265,7 +280,12 @@ def s_fn(arg_dict, executed, key): executed['s2_t2_source'] = False first_only_env = config.Environment( - sources=sources, + # sources=sources, + sources={ + 'table_one': { + 'table_one': config.Source(name='TABLE', args={'name': 'table_one_instance'}), + }, + }, execution=config.Execution(through_solids=['table_one']), ) diff --git a/python_modules/dagster/dagster/core/core_tests/test_output_expectations.py b/python_modules/dagster/dagster/core/core_tests/test_output_expectations.py deleted file mode 100644 index 2df0fa2ad2032..0000000000000 --- a/python_modules/dagster/dagster/core/core_tests/test_output_expectations.py +++ /dev/null @@ -1,73 +0,0 @@ -import pytest - -import dagster -from dagster.core.definitions import ( - ExpectationDefinition, - ExpectationResult, -) -from dagster.core.execution import ( - _execute_output_expectation, DagsterUserCodeExecutionError, ExecutionContext -) - - -def create_test_context(): - return ExecutionContext() - - -def create_dummy_output_info(expect_def): - return dagster.OutputExpectationInfo( - solid=dagster.SolidDefinition( - name='dummy', - inputs=[], - transform_fn=lambda _context, _args: None, - output=dagster.OutputDefinition(expectations=[expect_def]) - ), - expectation_def=expect_def - ) - - -def test_basic_failing_output_expectation(): - def failing(_context, _info, _output): - return ExpectationResult( - success=False, - message='some message', - ) - - result = _execute_output_expectation( - create_test_context(), create_dummy_output_info(ExpectationDefinition('failing', failing)), - 'not used' - ) - - assert isinstance(result, ExpectationResult) - assert not result.success - assert result.message == 'some message' - - -def test_basic_passing_output_expectation(): - def success(_context, _info, _output): - return ExpectationResult( - success=True, - message='yay', - ) - - expectation = ExpectationDefinition('success', success) - result = _execute_output_expectation( - create_test_context(), - create_dummy_output_info(expectation), - 'not used', - ) - - assert isinstance(result, ExpectationResult) - assert result.success - assert result.message == 'yay' - - -def test_output_expectation_user_error(): - def throwing(_output): - raise Exception('user error') - - with pytest.raises(DagsterUserCodeExecutionError): - _execute_output_expectation( - create_test_context(), - create_dummy_output_info(ExpectationDefinition('throwing', throwing)), 'not used' - ) diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py index 87d83c3196c5c..f091ee795cdbb 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py @@ -12,6 +12,7 @@ execute_pipeline, ExecutionContext, ) +from dagster.core.errors import DagsterUserCodeExecutionError from dagster.utils.compatability import ( create_custom_source_input, create_single_materialization_output ) @@ -143,7 +144,7 @@ def test_transform_failure_pipeline(): assert len(result_list) == 1 assert not result_list[0].success - assert result_list[0].exception + assert result_list[0].dagster_user_exception def test_input_failure_pipeline(): @@ -158,7 +159,7 @@ def test_input_failure_pipeline(): assert len(result_list) == 1 assert not result_list[0].success - assert result_list[0].exception + assert result_list[0].dagster_user_exception def test_output_failure_pipeline(): @@ -179,7 +180,7 @@ def test_output_failure_pipeline(): assert len(result_list) == 1 assert not result_list[0].success - assert result_list[0].exception + assert result_list[0].dagster_user_exception def test_failure_midstream(): @@ -222,4 +223,4 @@ def transform_fn(_context, args): assert result_list[0].success assert result_list[1].success assert not result_list[2].success - assert result_list[2].reason == DagsterExecutionFailureReason.USER_CODE_ERROR + assert isinstance(result_list[2].dagster_user_exception, DagsterUserCodeExecutionError) diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py index add096fe60307..b6f19815d0f06 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py @@ -2,13 +2,18 @@ from dagster import (check, config) -from dagster.core.definitions import (SolidDefinition, OutputDefinition, PipelineDefinition) +from dagster.core.definitions import ( + OutputDefinition, + PipelineDefinition, + SolidDefinition, + construct_dependency_structure_from_solids_only, +) from dagster.core.graph import (create_adjacency_lists, SolidGraph) from dagster.core.execution import ( execute_pipeline_iterator, execute_pipeline_iterator_in_memory, ExecutionContext, - SolidExecutionResult, + ExecutionResultBase, ) from dagster.utils.compatability import create_custom_source_input @@ -79,8 +84,13 @@ def root_transform(_context, args): ) +def _do_construct(solids): + return create_adjacency_lists(solids, construct_dependency_structure_from_solids_only(solids)) + + def test_empty_adjaceny_lists(): - forward_edges, backwards_edges = create_adjacency_lists([create_root_solid('a_node')]) + solids = [create_root_solid('a_node')] + forward_edges, backwards_edges = _do_construct(solids) assert forward_edges == {'a_node': set()} assert backwards_edges == {'a_node': set()} @@ -90,12 +100,16 @@ def test_single_dep_adjacency_lists(): node_a = create_root_solid('A') node_b = create_solid_with_deps('B', node_a) - forward_edges, backwards_edges = create_adjacency_lists([node_a, node_b]) + forward_edges, backwards_edges = _do_construct([node_a, node_b]) assert forward_edges == {'A': {'B'}, 'B': set()} assert backwards_edges == {'B': {'A'}, 'A': set()} +def graph_from_solids_only(solids): + return SolidGraph(solids, construct_dependency_structure_from_solids_only(solids)) + + def test_diamond_deps_adjaceny_lists(): # A <-- (B, C) <-- D @@ -104,7 +118,7 @@ def test_diamond_deps_adjaceny_lists(): node_c = create_solid_with_deps('C', node_a) node_d = create_solid_with_deps('D', node_b, node_c) - forward_edges, backwards_edges = create_adjacency_lists([node_a, node_b, node_c, node_d]) + forward_edges, backwards_edges = _do_construct([node_a, node_b, node_c, node_d]) assert forward_edges == {'A': {'B', 'C'}, 'B': {'D'}, 'C': {'D'}, 'D': set()} assert backwards_edges == {'D': {'B', 'C'}, 'B': {'A'}, 'C': {'A'}, 'A': set()} @@ -118,7 +132,7 @@ def test_disconnected_graphs_adjaceny_lists(): node_c = create_root_solid('C') node_d = create_solid_with_deps('D', node_c) - forward_edges, backwards_edges = create_adjacency_lists([node_a, node_b, node_c, node_d]) + forward_edges, backwards_edges = _do_construct([node_a, node_b, node_c, node_d]) assert forward_edges == {'A': {'B'}, 'B': set(), 'C': {'D'}, 'D': set()} assert backwards_edges == {'B': {'A'}, 'A': set(), 'D': {'C'}, 'C': set()} @@ -129,7 +143,7 @@ def create_diamond_graph(): node_c = create_solid_with_deps('C', node_a) node_d = create_solid_with_deps('D', node_b, node_c) - return SolidGraph([node_d, node_c, node_b, node_a]) + return graph_from_solids_only([node_d, node_c, node_b, node_a]) def test_diamond_toposort(): @@ -139,7 +153,7 @@ def test_diamond_toposort(): def test_single_node_unprovided_inputs(): node_a = create_root_solid('A') - solid_graph = SolidGraph(solids=[node_a]) + solid_graph = graph_from_solids_only(solids=[node_a]) assert solid_graph.compute_unprovided_inputs('A', ['A_input']) == set() assert solid_graph.compute_unprovided_inputs('A', []) == set(['A_input']) @@ -176,7 +190,7 @@ def test_diamond_toposort_unprovided_inputs(): def test_unprovided_input_param_invariants(): node_a = create_root_solid('A') - solid_graph = SolidGraph(solids=[node_a]) + solid_graph = graph_from_solids_only(solids=[node_a]) with pytest.raises(check.ParameterCheckError): solid_graph.compute_unprovided_inputs('B', []) @@ -187,7 +201,7 @@ def test_unprovided_input_param_invariants(): def test_execution_subgraph_one_node(): node_a = create_root_solid('A') - solid_graph = SolidGraph(solids=[node_a]) + solid_graph = graph_from_solids_only(solids=[node_a]) execution_graph = solid_graph.create_execution_subgraph( from_solids=['A'], @@ -230,8 +244,8 @@ def transform_called(name): def assert_equivalent_results(left, right): - check.inst_param(left, 'left', SolidExecutionResult) - check.inst_param(right, 'right', SolidExecutionResult) + check.inst_param(left, 'left', ExecutionResultBase) + check.inst_param(right, 'right', ExecutionResultBase) assert left.success == right.success assert left.name == right.name @@ -240,8 +254,8 @@ def assert_equivalent_results(left, right): def assert_all_results_equivalent(expected_results, result_results): - check.list_param(expected_results, 'expected_results', of_type=SolidExecutionResult) - check.list_param(result_results, 'result_results', of_type=SolidExecutionResult) + check.list_param(expected_results, 'expected_results', of_type=ExecutionResultBase) + check.list_param(result_results, 'result_results', of_type=ExecutionResultBase) assert len(expected_results) == len(result_results) for expected, result in zip(expected_results, result_results): assert_equivalent_results(expected, result) @@ -264,11 +278,11 @@ def test_pipeline_execution_graph_diamond_in_memory(): return _do_test(pipeline, lambda: execute_pipeline_iterator_in_memory( ExecutionContext(), pipeline, - input_values=input_values, + input_values={'A': input_values}, )) -def _do_test(pipeline, do_execute_pipeline_iter): +def _do_test(_pipeline, do_execute_pipeline_iter): solid_graph = create_diamond_graph() pipeline = PipelineDefinition(solids=solid_graph.solids) @@ -283,49 +297,38 @@ def _do_test(pipeline, do_execute_pipeline_iter): assert results[0].transformed_value == [input_set('A_input'), transform_called('A')] - expected_results = [ - SolidExecutionResult( - success=True, - solid=pipeline.solid_named('A'), - transformed_value=[ - input_set('A_input'), - transform_called('A'), - ], - exception=None, - ), - SolidExecutionResult( - success=True, - solid=pipeline.solid_named('B'), - transformed_value=[ - input_set('A_input'), - transform_called('A'), - transform_called('B'), - ], - exception=None, - ), - SolidExecutionResult( - success=True, - solid=pipeline.solid_named('C'), - transformed_value=[ - input_set('A_input'), - transform_called('A'), - transform_called('B'), - transform_called('C'), - ], - exception=None, - ), - SolidExecutionResult( - success=True, - solid=pipeline.solid_named('D'), - transformed_value=[ - input_set('A_input'), - transform_called('A'), - transform_called('B'), - transform_called('C'), - transform_called('D'), - ], - exception=None, - ), + assert results[1].transformed_value == [ + input_set('A_input'), + transform_called('A'), + transform_called('C'), + ] or results[1].transformed_value == [ + input_set('A_input'), + transform_called('A'), + transform_called('B'), + ] + + assert results[2].transformed_value == [ + input_set('A_input'), + transform_called('A'), + transform_called('C'), + transform_called('B'), + ] or [ + input_set('A_input'), + transform_called('A'), + transform_called('B'), + transform_called('C'), ] - assert_all_results_equivalent(expected_results, results) + assert results[3].transformed_value == [ + input_set('A_input'), + transform_called('A'), + transform_called('C'), + transform_called('B'), + transform_called('D'), + ] or [ + input_set('A_input'), + transform_called('A'), + transform_called('B'), + transform_called('C'), + transform_called('D'), + ] diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_meta.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_meta.py index ccd547aa2462b..0a9d0bf1b8519 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_meta.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_meta.py @@ -12,8 +12,3 @@ def step_one_no_external_source(): def step_two(foo_bar): foo_bar['foo'] = foo_bar['foo'] + foo_bar['foo'] return foo_bar - - -def test_pipeline_meta(): - pipeline = dagster.PipelineDefinition(solids=[step_one_no_external_source, step_two]) - assert list(pipeline.external_inputs) == [] diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index 04ec217a3a6a4..ee21af58d0490 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -1,13 +1,18 @@ +from collections import (defaultdict, namedtuple) import copy import keyword import re from dagster import check from dagster.core import types -from dagster.utils.logging import level_from_string +from dagster.utils.logging import ( + level_from_string, + define_colored_console_logger, +) from .errors import DagsterInvalidDefinitionError -from .graph import SolidGraph + +DEFAULT_OUTPUT = 'result' DISALLOWED_NAMES = set( [ @@ -60,17 +65,11 @@ def __init__(self, *, argument_def_dict, context_fn, description=None): def _default_pipeline_context_definitions(): def _default_context_fn(_pipeline, args): - # This has a circular dependency between execution and definition - # The likely solution is to move the ExecutionContext to definitions.py - # -- schrockn (07-28-18) import dagster.core.execution - import dagster.utils.logging log_level = level_from_string(args['log_level']) context = dagster.core.execution.ExecutionContext( - loggers=[ - dagster.utils.logging.define_colored_console_logger('dagster', level=log_level) - ] + loggers=[define_colored_console_logger('dagster', level=log_level)] ) return context @@ -84,8 +83,70 @@ def _default_context_fn(_pipeline, args): return {'default': default_context_def} +class DependencyDefinition: + def __init__(self, from_solid, from_input, to_solid, to_output=DEFAULT_OUTPUT): + self.from_solid = check.str_param(from_solid, 'from_solid') + self.from_input = check.str_param(from_input, 'from_input') + self.to_solid = check.str_param(to_solid, 'to_solid') + self.to_output = check.str_param(to_output, 'to_output') + + +DepTarget = namedtuple('DepTarget', 'solid_name output_name') + + +class DependencyStructure: + def __init__(self, deps): + self.deps = check.list_param(deps, 'deps', of_type=DependencyDefinition) + + self._dep_lookup = defaultdict(dict) + for dep in deps: + self._dep_lookup[dep.from_solid][dep.from_input] = DepTarget( + solid_name=dep.to_solid, + output_name=dep.to_output, + ) + + def has_dep(self, solid_name, input_name): + check.str_param(solid_name, 'solid_name') + check.str_param(input_name, 'input_name') + return input_name in self._dep_lookup.get(solid_name, {}) + + def deps_of_solid(self, solid_name): + check.str_param(solid_name, 'solid_name') + return list(self._dep_lookup[solid_name].values()) + + def get_dep_target(self, solid_name, input_name): + check.str_param(solid_name, 'solid_name') + check.str_param(input_name, 'input_name') + return self._dep_lookup[solid_name][input_name] + + +def construct_dependency_structure_from_solids_only(solids): + check.list_param(solids, 'solids', SolidDefinition) + + deps = [] + for solid in solids: + for input_def in solid.inputs: + if input_def.depends_on: + + to_solid = input_def.depends_on + check.invariant(len(to_solid.outputs) == 1) + + deps.append( + DependencyDefinition( + from_solid=solid.name, + from_input=input_def.name, + to_solid=to_solid.name, + to_output=to_solid.outputs[0].name + ) + ) + + return DependencyStructure(deps) + + class PipelineDefinition: - def __init__(self, solids, name=None, description=None, context_definitions=None): + def __init__( + self, solids, name=None, description=None, context_definitions=None, dependencies=None + ): self.description = check.opt_str_param(description, 'description') self.name = check.opt_str_param(name, 'name') @@ -122,16 +183,18 @@ def __init__(self, solids, name=None, description=None, context_definitions=None {solid_names}''' ) - self.solid_graph = SolidGraph(solids=solids) + # self.solid_graph = SolidGraph(solids=solids) + + # TEMPORARY FOR MIGRATION + if dependencies is None: + self.dependency_structure = construct_dependency_structure_from_solids_only(solids) + else: + self.dependency_structure = DependencyStructure(dependencies) @property def solid_names(self): return [solid.name for solid in self.solids] - @property - def input_names(self): - return set([input_def.name for input_def in self.all_inputs]) - def get_input(self, solid_name, input_name): for solid in self.solids: if solid.name != solid_name: @@ -141,20 +204,6 @@ def get_input(self, solid_name, input_name): return input_def check.failed('not found') - @property - def external_inputs(self): - for input_def in self.all_inputs: - if input_def.is_external: - yield input_def - - @property - def externally_sourced_solids(self): - for solid in self.solids: - for input_def in solid.inputs: - if input_def.is_external: - yield solid - break - def has_solid(self, name): check.str_param(name, 'name') for solid in self.solids: @@ -162,12 +211,6 @@ def has_solid(self, name): return True return False - @property - def all_inputs(self): - for solid in self.solids: - for input_def in solid.inputs: - yield input_def - def solid_named(self, name): check.str_param(name, 'name') for solid in self.solids: @@ -175,18 +218,6 @@ def solid_named(self, name): return solid check.failed('Could not find solid named ' + name) - @property - def all_depended_on_solids(self): - for input_def in self.all_inputs: - if input_def.depends_on: - yield input_def.depends_on - - @property - def all_sink_solids(self): - all_names = set([solid.name for solid in self.solids]) - all_depended_on_names = set([solid.name for solid in self.all_depended_on_solids]) - return all_names.difference(all_depended_on_names) - class ExpectationResult: def __init__(self, success, solid=None, message=None, result_context=None): @@ -280,10 +311,6 @@ def __init__( self.input_callback = check.opt_callable_param(input_callback, 'input_callback') self.description = check.opt_str_param(description, 'description') - @property - def is_external(self): - return self.depends_on is None - def source_of_type(self, source_type): check.str_param(source_type, 'source_type') for source in self.sources: @@ -333,12 +360,14 @@ class OutputDefinition: # runtime type info def __init__( self, + # name=None, dagster_type=None, materializations=None, expectations=None, output_callback=None, description=None ): + self.name = DEFAULT_OUTPUT self.dagster_type = check.opt_inst_param( dagster_type, 'dagster_type', types.DagsterType, types.Any @@ -370,12 +399,21 @@ def materialization_of_type(self, name): # The output class SolidDefinition: def __init__(self, name, inputs, transform_fn, output, description=None): + # if output: + # check.invariant(outputs is None) + # self.outputs = [output] + # else: + # check.invariant(outputs is not None) + # self.outputs = check.list_param(outputs, 'outputs', of_type=OutputDefinition) + self.name = check_valid_name(name) self.inputs = check.list_param(inputs, 'inputs', InputDefinition) - self.output = check.inst_param(output, 'output', OutputDefinition) self.transform_fn = check.callable_param(transform_fn, 'transform') + self.output = check.inst_param(output, 'output', OutputDefinition) self.description = check.opt_str_param(description, 'description') + self.outputs = [output] + # Notes to self # Input Definitions diff --git a/python_modules/dagster/dagster/core/errors.py b/python_modules/dagster/dagster/core/errors.py index 7c482a6dd38ac..88037768a733a 100644 --- a/python_modules/dagster/dagster/core/errors.py +++ b/python_modules/dagster/dagster/core/errors.py @@ -57,8 +57,8 @@ def __init__(self, execution_result, *args, **kwargs): # Probable fix is to move all "execution result" objects # to definitions import dagster.core.execution - self.execution_result = check.inst_param( + self.execution_result = check.opt_inst_param( execution_result, 'execution_result', - dagster.core.execution.SolidExecutionResult, + dagster.core.execution.ExecutionResultBase, ) diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index 9376a2c15552c..6f9ccce1920d9 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -21,170 +21,36 @@ # too many lines # pylint: disable=C0302 -from collections import (namedtuple, OrderedDict) from contextlib import contextmanager import copy -import sys -import uuid import six from dagster import check, config -from dagster.utils.logging import (CompositeLogger, get_formatted_stack_trace) -from dagster.utils.timing import time_execution_scope from .definitions import ( - SolidDefinition, ExpectationResult, SourceDefinition, - MaterializationDefinition, PipelineDefinition, PipelineContextDefinition + SolidDefinition, + PipelineDefinition, + PipelineContextDefinition, ) from .errors import ( - DagsterUserCodeExecutionError, DagsterTypeError, DagsterExecutionFailureReason, - DagsterExpectationFailedError, DagsterInvariantViolationError + DagsterUserCodeExecutionError, + DagsterExecutionFailureReason, + DagsterExpectationFailedError, + DagsterInvariantViolationError, ) from .argument_handling import validate_args -Metric = namedtuple('Metric', 'context_dict metric_name value') - - -class ExecutionContext: - ''' - A context object flowed through the entire scope of single execution of a - pipeline of solids. This is used by both framework and user code to log - messages and metrics. It also maintains a stack of context values so that - logs, metrics, and any future reporting are reported with a minimal, consistent - level of context so that developers do not have to repeatedly log well-known - information (e.g. the name of the solid, the name of the pipeline, etc) when - logging. Additionally tool author may add their own context values to assist - reporting. - - - resources is an arbitrary user-defined object that can be passed in - by a user and then access during pipeline execution. This exists so that - a user does not have to subclass ExecutionContext - ''' - - def __init__(self, loggers=None, resources=None): - self._logger = CompositeLogger(loggers=loggers) - self._context_dict = OrderedDict() - self._metrics = [] - self.resources = resources - - def _maybe_quote(self, val): - str_val = str(val) - if ' ' in str_val: - return '"{val}"'.format(val=str_val) - return str_val - - def _kv_message(self, extra=None): - extra = check.opt_dict_param(extra, 'extra') - return ' '.join( - [ - '{key}={value}'.format(key=key, value=self._maybe_quote(value)) - for key, value in [*self._context_dict.items(), *extra.items()] - ] - ) - - def _log(self, method, msg, kwargs): - check.str_param(method, 'method') - check.str_param(msg, 'msg') - - check.invariant('extra' not in kwargs, 'do not allow until explicit support is handled') - check.invariant('exc_info' not in kwargs, 'do not allow until explicit support is handled') - - check.invariant('log_message' not in kwargs, 'log_message_id reserved value') - check.invariant('log_message_id' not in kwargs, 'log_message_id reserved value') - - - full_message = 'message="{message}" {kv_message}'.format( - message=msg, kv_message=self._kv_message(kwargs) - ) - - log_props = copy.copy(self._context_dict) - - log_props['log_message'] = msg - log_props['log_message_id'] = str(uuid.uuid4()) - - getattr(self._logger, method)(full_message, extra={**log_props, **kwargs}) - - def debug(self, msg, **kwargs): - return self._log('debug', msg, kwargs) - - def info(self, msg, **kwargs): - return self._log('info', msg, kwargs) - - def warning(self, msg, **kwargs): - return self._log('warning', msg, kwargs) - - def error(self, msg, **kwargs): - return self._log('error', msg, kwargs) - - def critical(self, msg, **kwargs): - return self._log('critical', msg, kwargs) - - # FIXME: Actually make this work - # def exception(self, e): - # check.inst_param(e, 'e', Exception) - - # # this is pretty lame right. should embellish with more data (stack trace?) - # return self._log('error', str(e)) - - @contextmanager - def value(self, key, value): - check.str_param(key, 'key') - check.not_none_param(value, 'value') - - check.invariant(not key in self._context_dict, 'Should not be in context') - - self._context_dict[key] = value - - yield - - self._context_dict.pop(key) - - def metric(self, metric_name, value): - check.str_param(metric_name, 'metric_name') - check.not_none_param(value, 'value') - - keys = list(self._context_dict.keys()) - keys.append(metric_name) - if isinstance(value, float): - format_string = 'metric:{metric_name}={value:.3f} {kv_message}' - else: - format_string = 'metric:{metric_name}={value} {kv_message}' - - self._logger.info( - format_string.format( - metric_name=metric_name, value=value, kv_message=self._kv_message() - ), - extra=self._context_dict - ) - - self._metrics.append( - Metric( - context_dict=copy.copy(self._context_dict), metric_name=metric_name, value=value - ) - ) - - def _dict_covers(self, needle_dict, haystack_dict): - for key, value in needle_dict.items(): - if not key in haystack_dict: - return False - if value != haystack_dict[key]: - return False - return True - - def metrics_covering_context(self, needle_dict): - for metric in self._metrics: - if self._dict_covers(needle_dict, metric.context_dict): - yield metric +from .compute_nodes import ( + ComputeNodeTag, + create_compute_node_graph_from_env, + execute_compute_nodes, +) - def metrics_matching_context(self, needle_dict): - for metric in self._metrics: - if needle_dict == metric.context_dict: - yield metric +from .execution_context import ExecutionContext class DagsterPipelineExecutionResult: def __init__( @@ -194,7 +60,7 @@ def __init__( ): self.context = check.inst_param(context, 'context', ExecutionContext) self.result_list = check.list_param( - result_list, 'result_list', of_type=SolidExecutionResult + result_list, 'result_list', of_type=ExecutionResultBase ) @property @@ -209,7 +75,41 @@ def result_named(self, name): check.failed('Did not find result {name} in pipeline execution result'.format(name=name)) -class SolidExecutionResult: +class ExecutionResultBase: + def __init__(self, success, context, transformed_value): + self.success = check.bool_param(success, 'success') + self.context = context + self.transformed_value = transformed_value + + + +class BridgeSolidExecutionResult(ExecutionResultBase): + def __init__(self, success, context, transformed_value, name, dagster_user_exception, solid, tag): + super().__init__(success, context, transformed_value) + self.name = name + self.dagster_user_exception = dagster_user_exception + self.solid = solid + self.tag = tag + + def copy(self): + ''' This must be used instead of copy.deepcopy() because exceptions cannot + be deepcopied''' + return BridgeSolidExecutionResult( + name=self.name, + solid=self.solid, + success=self.success, + transformed_value=copy.deepcopy(self.transformed_value), + context=self.context, + dagster_user_exception=self.dagster_user_exception, + tag=self.tag, + ) + + def reraise_user_error(self): + check.inst(self.dagster_user_exception, DagsterUserCodeExecutionError) + six.reraise(*self.dagster_user_exception.original_exc_info) + + +class LegacySolidExecutionResult(ExecutionResultBase): ''' A class to represent the result of the execution of a single solid. Pipeline commands return iterators or lists of these results. @@ -224,40 +124,27 @@ def __init__( transformed_value, reason=None, exception=None, - input_expectation_results=None, - output_expectation_results=None, + _input_expectation_results=None, + _output_expectation_results=None, context=None, ): - self.success = check.bool_param(success, 'success') + super().__init__(success, context, transformed_value) if not success: check.param_invariant( isinstance(reason, DagsterExecutionFailureReason), 'reason', 'Must provide a reason is result is a failure' ) - self.transformed_value = transformed_value + # self.transformed_value = transformed_value self.solid = check.inst_param(solid, 'solid', SolidDefinition) self.reason = reason self.exception = check.opt_inst_param(exception, 'exception', Exception) - self.input_expectation_results = check.opt_inst_param( - input_expectation_results, - 'input_expectation_results', - InputExpectationResults - ) - - self.output_expectation_results = check.opt_inst_param( - output_expectation_results, - 'output_expectation_results', - OutputExpectationResults, - ) - if reason == DagsterExecutionFailureReason.USER_CODE_ERROR: check.inst(exception, DagsterUserCodeExecutionError) self.user_exception = exception.user_exception else: self.user_exception = None - self.context = context def reraise_user_error(self): check.invariant(self.reason == DagsterExecutionFailureReason.USER_CODE_ERROR) @@ -271,17 +158,13 @@ def name(self): def copy(self): ''' This must be used instead of copy.deepcopy() because exceptions cannot be deepcopied''' - return SolidExecutionResult( + return LegacySolidExecutionResult( success=self.success, solid=self.solid, transformed_value=copy.deepcopy(self.transformed_value), context=self.context, reason=self.reason, exception=self.exception, - input_expectation_results=self.input_expectation_results.copy() - if self.input_expectation_results else None, - output_expectation_results=self.output_expectation_results.copy() - if self.output_expectation_results else None, ) def copy_result_list(result_list): @@ -299,340 +182,6 @@ def copy_result_dict(result_dict): new_dict[input_name] = result.copy() return new_dict -@contextmanager -def _user_code_error_boundary(context, msg, **kwargs): - ''' - Wraps the execution of user-space code in an error boundary. This places a uniform - policy around an user code invoked by the framework. This ensures that all user - errors are wrapped in the SolidUserCodeExecutionError, and that the original stack - trace of the user error is preserved, so that it can be reported without confusing - framework code in the stack trace, if a tool author wishes to do so. This has - been especially help in a notebooking context. - ''' - check.inst_param(context, 'context', ExecutionContext) - check.str_param(msg, 'msg') - - try: - yield - except Exception as e: - stack_trace = get_formatted_stack_trace(e) - context.error(str(e), stack_trace=stack_trace) - raise DagsterUserCodeExecutionError( - msg.format(**kwargs), e, user_exception=e, original_exc_info=sys.exc_info() - ) - -def _read_source(context, source_definition, arg_dict): - ''' - Check to ensure that the arguments to a particular input are valid, and then - execute the input functions. Wraps that execution in appropriate logging, metrics tracking, - and a user-code error boundary. - ''' - check.inst_param(context, 'context', ExecutionContext) - check.inst_param(source_definition, 'source_definition', SourceDefinition) - check.dict_param(arg_dict, 'arg_dict', key_type=str) - - with context.value('source_type', source_definition.source_type): - error_context_str = 'source type {source}'.format(source=source_definition.source_type) - args_to_pass = validate_args( - source_definition.argument_def_dict, - arg_dict, - error_context_str, - ) - error_str = 'Error occured while loading source "{source_type}"' - with _user_code_error_boundary( - context, - error_str, - source_type=source_definition.source_type, - ): - context.info('Entering input implementation') - - with time_execution_scope() as timer_result: - value = source_definition.source_fn(context, args_to_pass) - - context.metric('input_load_time_ms', timer_result.millis) - - return value - - -InputExpectationInfo = namedtuple('InputExpectionInfo', 'solid input_def expectation_def') - -def _execute_input_expectation(context, info, value): - ''' - Execute one user-specified input expectation on an input that has been instantiated in memory - Wraps computation in an error boundary and performs all necessary logging and metrics tracking - (TODO: actually log and track metrics!) - ''' - check.inst_param(context, 'context', ExecutionContext) - check.inst_param(info, 'info', InputExpectationInfo) - - error_str = 'Error occured while evaluation expectation "{expectation_name}" in input' - with _user_code_error_boundary(context, error_str, expectation_name=info.expectation_def.name): - expectation_result = info.expectation_def.expectation_fn(context, info, value) - - if not isinstance(expectation_result, ExpectationResult): - raise DagsterInvariantViolationError( - 'Must return ExpectationResult from expectation function' - ) - return expectation_result - - -OutputExpectationInfo = namedtuple('OutputExpectationInfo', 'solid expectation_def') - -def _execute_output_expectation(context, info, transformed_value): - ''' - Execute one user-specified output expectation on an instantiated result of the core transform. - Wraps computation in an error boundary and performs all necessary logging and metrics tracking - (TODO: actually log and track metrics!) - ''' - check.inst_param(context, 'context', ExecutionContext) - # check.inst_param(expectation_def, 'expectation_def', ExpectationDefinition) - check.inst_param(info, 'info', OutputExpectationInfo) - - error_str = 'Error occured while evaluation expectation "{expectation_name}" in output' - expectation_def = info.expectation_def - with _user_code_error_boundary(context, error_str, expectation_name=expectation_def.name): - expectation_result = expectation_def.expectation_fn(context, info, transformed_value) - - if not isinstance(expectation_result, ExpectationResult): - - raise DagsterInvariantViolationError( - 'Must return ExpectationResult from expectation function' - ) - - return expectation_result - - -def _execute_core_transform(context, solid_transform_fn, values_dict): - ''' - Execute the user-specified transform for the solid. Wrap in an error boundary and do - all relevant logging and metrics tracking - ''' - check.inst_param(context, 'context', ExecutionContext) - check.callable_param(solid_transform_fn, 'solid_transform_fn') - check.dict_param(values_dict, 'values_dict', key_type=str) - - error_str = 'Error occured during core transform' - with _user_code_error_boundary(context, error_str): - with time_execution_scope() as timer_result: - transformed_value = solid_transform_fn(context, values_dict) - - context.metric('core_transform_time_ms', timer_result.millis) - - return transformed_value - - -def _execute_materialization(context, materialiation_def, arg_dict, value): - ''' - Execute a single output, calling into user-specified code. Check validity - of arguments into the output, do appropriate loggina and metrics tracking, and - actually execute the output function with an appropriate error boundary. - ''' - check.inst_param(context, 'context', ExecutionContext) - check.inst_param(materialiation_def, 'materialization', MaterializationDefinition) - check.dict_param(arg_dict, 'arg_dict', key_type=str) - - expected_args = set(materialiation_def.argument_def_dict.keys()) - received_args = set(arg_dict.keys()) - - if expected_args != received_args: - raise DagsterTypeError( - 'Argument mismatch in output. Expected {expected} got {received}'.format( - expected=repr(expected_args), - received=repr(received_args), - ) - ) - - for arg_name, arg_value in arg_dict.items(): - arg_def_type = materialiation_def.argument_def_dict[arg_name] - if not arg_def_type.dagster_type.is_python_valid_value(arg_value): - raise DagsterTypeError( - 'Expected type {typename} for arg {arg_name} in output but got {arg_value}'.format( - typename=arg_def_type.dagster_type.name, - arg_name=arg_name, - arg_value=repr(arg_value), - ) - ) - - error_str = 'Error during execution of materialization' - with _user_code_error_boundary(context, error_str): - context.info('Entering materialization implementation') - materialiation_def.materialization_fn(context, arg_dict, value) - - -class InputExpectationResult: - def __init__(self, input_name, all_results): - self.input_name = check.str_param(input_name, 'input_name') - self.all_results = check.list_param(all_results, 'all_results', ExpectationResult) - - @property - def fails(self): - for result in self.all_results: - if not result.success: - yield result - - @property - def passes(self): - for result in self.all_results: - if result.success: - yield result - - def copy(self): - return InputExpectationResult( - input_name=self.input_name, - all_results=copy_result_list(self.all_results), - ) - -class InputExpectationResults: - def __init__(self, result_dict): - check.dict_param( - result_dict, - 'result_dict', - key_type=str, - value_type=InputExpectationResult - ) - - all_passes = [] - all_fails = [] - for run_results in result_dict.values(): - all_passes.extend(run_results.passes) - all_fails.extend(run_results.fails) - - self.all_passes = all_passes - self.all_fails = all_fails - self.result_dict = result_dict - - def copy(self): - return InputExpectationResults(copy_result_dict(self.result_dict)) - - @property - def success(self): - return not self.all_fails - - -def _execute_all_input_expectations(context, input_manager, solid, values_dict): - check.inst_param(context, 'context', ExecutionContext) - check.dict_param(values_dict, 'values_dict', key_type=str) - - result_dict = {} - - if not input_manager.evaluate_expectations: - return InputExpectationResults(result_dict) - - for input_name in values_dict.keys(): - input_def = solid.input_def_named(input_name) - value = values_dict[input_name] - - results = [] - - for input_expectation_def in input_def.expectations: - user_expectation_result = _execute_input_expectation( - context, InputExpectationInfo(solid, input_def, input_expectation_def), value, - ) - results.append(user_expectation_result) - - input_result = InputExpectationResult( - input_name=input_name, - all_results=results, - ) - result_dict[input_name] = input_result - - return InputExpectationResults(result_dict) - -class OutputExpectationResults: - def __init__(self, results): - self.results = check.list_param(results, 'results', ExpectationResult) - - @property - def success(self): - for result in self.results: - if not result.success: - return False - return True - - def copy(self): - return OutputExpectationResults(copy_result_list(self.results)) - -def _pipeline_solid_in_memory(context, input_manager, solid, transform_values_dict): - ''' - Given inputs that are already in memory. Evaluation all inputs expectations, - execute the core transform, and then evaluate all output expectations. - - This is the core of the solid execution that does not touch any extenralized state, whether - it be inputs or outputs. - ''' - check.inst_param(context, 'context', ExecutionContext) - check.inst_param(solid, 'solid', SolidDefinition) - check.dict_param(transform_values_dict, 'transform_values_dict', key_type=str) - - input_expectation_results = _execute_all_input_expectations( - context, - input_manager, - solid, - transform_values_dict - ) - - if not input_expectation_results.success: - return SolidExecutionResult( - success=False, - transformed_value=None, - solid=solid, - context=context, - reason=DagsterExecutionFailureReason.EXPECTATION_FAILURE, - input_expectation_results=input_expectation_results - ) - - context.info('Executing core transform') - - transformed_value = _execute_core_transform(context, solid.transform_fn, transform_values_dict) - - if not solid.output.dagster_type.is_python_valid_value(transformed_value): - raise DagsterInvariantViolationError(f'''Solid {solid.name} output {transformed_value} - which does not match the type for Dagster Type {solid.output.dagster_type.name}''') - - if solid.output.output_callback: - solid.output.output_callback(context, transformed_value) - - output_expectation_results = _execute_output_expectations( - context, - input_manager, - solid, - transformed_value, - ) - - if not output_expectation_results.success: - return SolidExecutionResult( - success=False, - transformed_value=None, - solid=solid, - context=context, - reason=DagsterExecutionFailureReason.EXPECTATION_FAILURE, - input_expectation_results=input_expectation_results, - output_expectation_results=output_expectation_results, - ) - - return SolidExecutionResult( - success=True, - transformed_value=transformed_value, - solid=solid, - context=context, - input_expectation_results=input_expectation_results, - output_expectation_results=output_expectation_results, - ) - - -def _execute_output_expectations(context, input_manager, solid, transformed_value): - if not input_manager.evaluate_expectations: - return OutputExpectationResults([]) - - output_expectation_result_list = [] - - for output_expectation_def in solid.output.expectations: - info = OutputExpectationInfo(solid=solid, expectation_def=output_expectation_def) - output_expectation_result = _execute_output_expectation(context, info, transformed_value) - output_expectation_result_list.append(output_expectation_result) - - return OutputExpectationResults(results=output_expectation_result_list) - def _create_passthrough_context_definition(context): check.inst_param(context, 'context', ExecutionContext) @@ -674,8 +223,11 @@ def execute_single_solid(context, solid, environment, throw_on_error=True): def _do_throw_on_error(execution_result): - check.inst_param(execution_result, 'execution_result', SolidExecutionResult) - if not execution_result.success: + check.inst_param(execution_result, 'execution_result', ExecutionResultBase) + if execution_result.success: + return + + if isinstance(execution_result, LegacySolidExecutionResult): if execution_result.reason == DagsterExecutionFailureReason.EXPECTATION_FAILURE: raise DagsterExpectationFailedError(execution_result) elif execution_result.reason == DagsterExecutionFailureReason.USER_CODE_ERROR: @@ -684,6 +236,13 @@ def _do_throw_on_error(execution_result): check.invariant(execution_result.exception) raise execution_result.exception + if isinstance(execution_result, BridgeSolidExecutionResult): + if isinstance(execution_result.dagster_user_exception, DagsterUserCodeExecutionError): + execution_result.reraise_user_error() + + raise execution_result.dagster_user_exception + + def output_single_solid( context, @@ -719,6 +278,17 @@ def output_single_solid( ) ) + for result in results: + if not result.success: + if throw_on_error: + _do_throw_on_error(result) + else: + return result + + for result in results: + if result.name == solid.name and result.tag == ComputeNodeTag.TRANSFORM: + return result + check.invariant(len(results) == 1, 'must be one result got ' + str(len(results))) execution_result = results[0] @@ -730,79 +300,7 @@ def output_single_solid( return execution_result - -def _gather_input_values(context, solid, input_manager): - check.inst_param(context, 'context', ExecutionContext) - check.inst_param(solid, 'solid', SolidDefinition) - check.inst_param(input_manager, 'input_manager', InputManager) - - context.info('About to instantiate and gather all inputs') - - input_values = {} - for input_def in solid.inputs: - with context.value('input', input_def.name): - input_value = input_manager.get_input_value(context, solid, input_def) - - if not input_def.dagster_type.is_python_valid_value(input_value): - raise DagsterInvariantViolationError(f'''Solid {solid.name} input {input_def.name} - received value {input_value} which does not match the type for Dagster type - {input_def.dagster_type.name}''' - ) - - input_values[input_def.name] = input_value - if input_def.input_callback: - input_def.input_callback(context, input_values[input_def.name]) - return input_values - - -def _execute_pipeline_solid_step(context, solid, input_manager): - check.inst_param(context, 'context', ExecutionContext) - check.inst_param(solid, 'solid', SolidDefinition) - check.inst_param(input_manager, 'input_manager', InputManager) - - # The value produce by an inputs is potentially different per solid. - # This is allowed so that two solids that do different instantiation of the - # same exact input (e.g. the same file) don't have to create additional solids - # to account for this. - - input_values = _gather_input_values(context, solid, input_manager) - - # This call does all input and output expectations, as well as the core transform - execution_result = _pipeline_solid_in_memory(context, input_manager, solid, input_values) - - if not execution_result.success: - return execution_result - - check.invariant( - solid.name not in input_manager.intermediate_values, - 'should be not in intermediate values' - ) - - transformed_value = execution_result.transformed_value - - context.debug( - 'About to set {output} for {name}'.format( - output=repr(transformed_value), - name=solid.name, - ) - ) - - input_manager.intermediate_values[solid.name] = transformed_value - - return SolidExecutionResult( - success=True, - solid=solid, - context=context, - transformed_value=input_manager.intermediate_values[solid.name], - input_expectation_results=execution_result.input_expectation_results, - output_expectation_results=execution_result.output_expectation_results, - exception=None - ) - -class InputManager: - def __init__(self): - self.intermediate_values = {} - +class DagsterEnv: @contextmanager def yield_context(self): check.not_implemented('must implement in subclass') @@ -823,17 +321,6 @@ def from_solids(self): def through_solids(self): check.not_implemented('must implement in subclass') - def get_input_value(self, context, solid, input_def): - if input_def.depends_on and input_def.depends_on.name in self.intermediate_values: - # grab value from dependency - return self.intermediate_values[input_def.depends_on.name] - else: - # must get value from source - return self._get_sourced_input_value(context, solid.name, input_def.name) - - def _get_sourced_input_value(self, _context, _solid_name, _input_name): - check.not_implemented('must implement in subclass') - def _wrap_in_yield(thing): if isinstance(thing, ExecutionContext): def _wrap(): @@ -844,11 +331,12 @@ def _wrap(): return thing -class InMemoryInputManager(InputManager): - def __init__(self, context, input_values, from_solids=None, through_solids=None): +class InMemoryEnv(DagsterEnv): + def __init__(self, context, pipeline, input_values, from_solids=None, through_solids=None): super().__init__() - self.input_values = check.dict_param(input_values, 'input_values', key_type=str) self.context = check.inst_param(context, 'context', ExecutionContext) + self.pipeline = check.inst_param(pipeline, 'pipeline', PipelineDefinition) + self.input_values = check.dict_param(input_values, 'input_values', key_type=str) self._from_solids = check.opt_list_param(from_solids, from_solids, of_type=str) self._through_solids = check.opt_list_param(through_solids, through_solids, of_type=str) @@ -860,9 +348,6 @@ def from_solids(self): def through_solids(self): return self._through_solids - def _get_sourced_input_value(self, _context, _solid_name, input_name): - return self.input_values[input_name] - @contextmanager def yield_context(self): return _wrap_in_yield(self.context) @@ -875,6 +360,7 @@ def materializations(self): def evaluate_expectations(self): return True + def _validate_environment(environment, pipeline): for solid_name, input_configs in environment.sources.items(): if not pipeline.has_solid(solid_name): @@ -888,7 +374,7 @@ def _validate_environment(environment, pipeline): if not solid_inst.has_input(input_name): raise DagsterInvariantViolationError( f'Input "{input_name}" not found in the pipeline on solid "{solid_name}".' + \ - f'Input must be one of {repr(pipeline.input_names)}' + f'Input must be one of {repr([inp.name for inp in solid_inst.inputs])}' ) context_name = environment.context.name @@ -900,7 +386,7 @@ def _validate_environment(environment, pipeline): ) -class EnvironmentInputManager(InputManager): +class ConfigEnv(DagsterEnv): def __init__(self, pipeline, environment): super().__init__() # This is not necessarily the best spot for these calls @@ -933,19 +419,6 @@ def yield_context(self): thing = context_definition.context_fn(self.pipeline, args_to_pass) return _wrap_in_yield(thing) - def _get_sourced_input_value(self, context, solid_name, input_name): - source_config = self.environment.sources[solid_name][input_name] - input_def = self.pipeline.get_input(solid_name, input_name) - source_def = input_def.source_of_type(source_config.name) - return _read_source( - context, source_def, self.args_for_input(solid_name, input_def.name) - ) - - def args_for_input(self, solid_name, input_name): - check.str_param(solid_name, 'solid_name') - check.str_param(input_name, 'input_name') - return self.environment.sources[solid_name][input_name].args - @property def materializations(self): return self.environment.materializations @@ -958,12 +431,12 @@ def execute_pipeline_iterator(pipeline, environment): check.inst_param(pipeline, 'pipeline', PipelineDefinition) check.inst_param(environment, 'enviroment', config.Environment) - input_manager = EnvironmentInputManager(pipeline, environment) - with input_manager.yield_context() as context: + env = ConfigEnv(pipeline, environment) + with env.yield_context() as context: return _execute_pipeline_iterator( context, pipeline, - EnvironmentInputManager(pipeline, environment) + ConfigEnv(pipeline, environment) ) def execute_pipeline_iterator_in_memory( @@ -979,120 +452,46 @@ def execute_pipeline_iterator_in_memory( return _execute_pipeline_iterator( context, pipeline, - InMemoryInputManager(context, input_values, from_solids, through_solids), + InMemoryEnv(context, pipeline, input_values, from_solids, through_solids), ) -def _execute_pipeline_iterator( - context, - pipeline, - input_manager, -): - ''' - This is the core workhorse function of this module, iterating over the pipeline execution - in topological order. This allow a tool consuming this API to execute a solid one at a time - and then make decisions based upon the result. - - If you do not specify "through_solids" it executes all the solids specified entire pipeline. - If through_solids is specified, it will stop executing once all of those solids in - through_solids have been executed. - - If you want to actually output the results of the transform see output_pipeline_iterator - - execute_pipeline is the "synchronous" version of this function and returns a list of results - once the entire pipeline has been executed. - ''' +def _execute_pipeline_iterator(context, pipeline, env): check.inst_param(context, 'context', ExecutionContext) check.inst_param(pipeline, 'pipeline', PipelineDefinition) - check.inst_param(input_manager, 'input_manager', InputManager) - - through_solids = input_manager.through_solids - from_solids = input_manager.from_solids - - pipeline_context_value = pipeline.name if pipeline.name else 'unnamed' - - materialization_args = MaterializationArgs(pipeline, input_manager.materializations) - - if through_solids is None: - through_solids = materialization_args.through_solids - - with context.value('pipeline', pipeline_context_value): - input_manager = input_manager - - if not through_solids: - through_solids = pipeline.all_sink_solids - - if not from_solids: - all_deps = set() - for through_solid in through_solids: - all_deps.union(pipeline.solid_graph.transitive_dependencies_of(through_solid)) - - from_solids = list(all_deps) - - # TODO provide meaningful error messages when the wrong inputs are provided for a particular - # execution subgraph - - # for through_solid_name in through_solids: - # unprovided_inputs = pipeline.solid_graph.compute_unprovided_inputs( - # input_names=sourced_input_names, solid_name=through_solid_name - # ) - # if unprovided_inputs: - # check.failed( - # 'Failed to provide inputs {unprovided_inputs} for solid {name}'.format( - # unprovided_inputs=unprovided_inputs, name=through_solid_name - # ) - # ) + check.inst_param(env, 'env', DagsterEnv) - execution_graph = pipeline.solid_graph.create_execution_subgraph( - from_solids, list(through_solids) - ) - - for solid in execution_graph.topological_solids: - - try: - with context.value('solid', solid.name): - result = _execute_pipeline_solid_step(context, solid, input_manager) - if not result.success: - yield result - break + cn_graph = create_compute_node_graph_from_env(pipeline, env) - if not materialization_args.should_materialize(result.name): - yield result - continue - - _execute_materializations( - context, - solid, - materialization_args.materializations_for_solid(solid.name), - result.transformed_value, - ) + cn_nodes = list(cn_graph.topological_nodes()) - yield result + check.invariant(len(cn_nodes[0].node_inputs) == 0) - except DagsterUserCodeExecutionError as see: - yield SolidExecutionResult( - success=False, - reason=DagsterExecutionFailureReason.USER_CODE_ERROR, - solid=solid, - context=context, - transformed_value=None, - exception=see, - ) - break - -def _execute_materializations( - context, - solid, - materializations, - transformed_value, -): - for materialization in materializations: - arg_dict = materialization.args - name = materialization.name - with context.value('materialization_name', name): - mat_def = solid.output.materialization_of_type(name) - _execute_materialization(context, mat_def, arg_dict, transformed_value) + for cn_result in execute_compute_nodes(context, cn_nodes): + cn_node = cn_result.compute_node + if not cn_result.success: + yield BridgeSolidExecutionResult( + success=False, + context=context, + transformed_value=None, + name=cn_node.solid.name, + dagster_user_exception=cn_result.failure_data.dagster_user_exception, + solid=cn_node.solid, + tag=cn_result.tag, + ) + return + + if cn_node.tag == ComputeNodeTag.TRANSFORM: + yield BridgeSolidExecutionResult( + success=True, + context=context, + transformed_value=cn_result.success_data.value, + name=cn_node.solid.name, + dagster_user_exception=None, + solid=cn_node.solid, + tag=cn_node.tag, + ) def execute_pipeline( pipeline, @@ -1103,7 +502,7 @@ def execute_pipeline( check.inst_param(environment, 'environment', config.Environment) return _execute_pipeline( pipeline, - EnvironmentInputManager(pipeline, environment), + ConfigEnv(pipeline, environment), throw_on_error, ) @@ -1119,33 +518,33 @@ def execute_pipeline_in_memory( check.dict_param(input_values, 'input_values', key_type=str) return _execute_pipeline( pipeline, - InMemoryInputManager(context, input_values, from_solids, through_solids), + InMemoryEnv(context, pipeline, input_values, from_solids, through_solids), throw_on_error, ) def _execute_pipeline( pipeline, - input_manager, + env, throw_on_error=True, ): ''' "Synchronous" version of execute_pipeline_iteator. throw_on_error makes the function throw when an error is encoutered rather than returning - the SolidExecutionResult in an error-state. + the LegacySolidExecutionResult in an error-state. Note: throw_on_error is very useful in testing contexts when not testing for error conditions ''' check.inst_param(pipeline, 'pipeline', PipelineDefinition) - check.inst_param(input_manager, 'input_manager', InputManager) + check.inst_param(env, 'env', DagsterEnv) check.bool_param(throw_on_error, 'throw_on_error') results = [] - with input_manager.yield_context() as context: + with env.yield_context() as context: for result in _execute_pipeline_iterator( context, pipeline, - input_manager=input_manager, + env=env, ): if throw_on_error: if not result.success: @@ -1153,21 +552,3 @@ def _execute_pipeline( results.append(result.copy()) return DagsterPipelineExecutionResult(context, results) - - -class MaterializationArgs: - def __init__(self, pipeline, materializations): - check.inst_param(pipeline, 'pipeline', PipelineDefinition) - check.list_param(materializations, 'materializations', of_type=config.Materialization) - - self.pipeline = pipeline - self.materializations = list(materializations) - self.through_solids = [materialization.solid for materialization in self.materializations] - - def should_materialize(self, solid_name): - return solid_name in self.through_solids - - def materializations_for_solid(self, solid_name): - for materialization in self.materializations: - if materialization.solid == solid_name: - yield materialization diff --git a/python_modules/dagster/dagster/core/execution_context.py b/python_modules/dagster/dagster/core/execution_context.py new file mode 100644 index 0000000000000..8c4c5e189df7a --- /dev/null +++ b/python_modules/dagster/dagster/core/execution_context.py @@ -0,0 +1,137 @@ +import copy +from collections import OrderedDict, namedtuple +from contextlib import contextmanager + +from dagster import check +from dagster.utils.logging import CompositeLogger + +Metric = namedtuple('Metric', 'context_dict metric_name value') + + +class ExecutionContext: + ''' + A context object flowed through the entire scope of single execution of a + pipeline of solids. This is used by both framework and user code to log + messages and metrics. It also maintains a stack of context values so that + logs, metrics, and any future reporting are reported with a minimal, consistent + level of context so that developers do not have to repeatedly log well-known + information (e.g. the name of the solid, the name of the pipeline, etc) when + logging. Additionally tool author may add their own context values to assist + reporting. + + + resources is an arbitrary user-defined object that can be passed in + by a user and then access during pipeline execution. This exists so that + a user does not have to subclass ExecutionContext + ''' + + def __init__(self, loggers=None, resources=None): + self._logger = CompositeLogger(loggers=loggers) + self._context_dict = OrderedDict() + self._metrics = [] + self.resources = resources + + def _maybe_quote(self, val): + str_val = str(val) + if ' ' in str_val: + return '"{val}"'.format(val=str_val) + return str_val + + def _kv_message(self, extra=None): + extra = check.opt_dict_param(extra, 'extra') + return ' '.join( + [ + '{key}={value}'.format(key=key, value=self._maybe_quote(value)) + for key, value in [*self._context_dict.items(), *extra.items()] + ] + ) + + def _log(self, method, msg, **kwargs): + check.str_param(method, 'method') + check.str_param(msg, 'msg') + + full_message = 'message="{message}" {kv_message}'.format( + message=msg, kv_message=self._kv_message(kwargs) + ) + + log_props = copy.copy(self._context_dict) + log_props['log_message'] = msg + + getattr(self._logger, method)(full_message, extra={**log_props, **kwargs}) + + def debug(self, msg, **kwargs): + return self._log('debug', msg, **kwargs) + + def info(self, msg, **kwargs): + return self._log('info', msg, **kwargs) + + def warning(self, msg, **kwargs): + return self._log('warning', msg, **kwargs) + + def error(self, msg, **kwargs): + return self._log('error', msg, **kwargs) + + def critical(self, msg, **kwargs): + return self._log('critical', msg, **kwargs) + + # FIXME: Actually make this work + # def exception(self, e): + # check.inst_param(e, 'e', Exception) + + # # this is pretty lame right. should embellish with more data (stack trace?) + # return self._log('error', str(e)) + + @contextmanager + def value(self, key, value): + check.str_param(key, 'key') + check.not_none_param(value, 'value') + + check.invariant(not key in self._context_dict, 'Should not be in context') + + self._context_dict[key] = value + + yield + + self._context_dict.pop(key) + + def metric(self, metric_name, value): + check.str_param(metric_name, 'metric_name') + check.not_none_param(value, 'value') + + keys = list(self._context_dict.keys()) + keys.append(metric_name) + if isinstance(value, float): + format_string = 'metric:{metric_name}={value:.3f} {kv_message}' + else: + format_string = 'metric:{metric_name}={value} {kv_message}' + + self._logger.info( + format_string.format( + metric_name=metric_name, value=value, kv_message=self._kv_message() + ), + extra=self._context_dict + ) + + self._metrics.append( + Metric( + context_dict=copy.copy(self._context_dict), metric_name=metric_name, value=value + ) + ) + + def _dict_covers(self, needle_dict, haystack_dict): + for key, value in needle_dict.items(): + if not key in haystack_dict: + return False + if value != haystack_dict[key]: + return False + return True + + def metrics_covering_context(self, needle_dict): + for metric in self._metrics: + if self._dict_covers(needle_dict, metric.context_dict): + yield metric + + def metrics_matching_context(self, needle_dict): + for metric in self._metrics: + if needle_dict == metric.context_dict: + yield metric diff --git a/python_modules/dagster/dagster/core/graph.py b/python_modules/dagster/dagster/core/graph.py index 6a81fbc8a29ec..841f9fc23d16c 100644 --- a/python_modules/dagster/dagster/core/graph.py +++ b/python_modules/dagster/dagster/core/graph.py @@ -1,47 +1,55 @@ from toposort import toposort_flatten -import dagster - from dagster import check -from dagster.core import types -from dagster.utils import logging - -import dagster.core.definitions -# from .definitions import (SolidDefinition, PipelineContextDefinition) +from .definitions import ( + DependencyStructure, + PipelineDefinition, + SolidDefinition, + construct_dependency_structure_from_solids_only, +) -def create_adjacency_lists(solids): - check.list_param(solids, 'solids', of_type=dagster.core.definitions.SolidDefinition) +def create_adjacency_lists(solids, dep_structure): + check.list_param(solids, 'solids', of_type=SolidDefinition) + check.inst_param(dep_structure, 'dep_structure', DependencyStructure) visit_dict = {s.name: False for s in solids} forward_edges = {s.name: set() for s in solids} backward_edges = {s.name: set() for s in solids} - def visit(solid): - if visit_dict[solid.name]: + def visit(solid_name): + if visit_dict[solid_name]: return - visit_dict[solid.name] = True + visit_dict[solid_name] = True - for inp in solid.inputs: - if inp.depends_on is not None: - from_node = inp.depends_on.name - to_node = solid.name - if from_node in forward_edges: - forward_edges[from_node].add(to_node) - backward_edges[to_node].add(from_node) - visit(inp.depends_on) + for dep_target in dep_structure.deps_of_solid(solid_name): + forward_node = dep_target.solid_name + backward_node = solid_name + if forward_node in forward_edges: + forward_edges[forward_node].add(backward_node) + backward_edges[backward_node].add(forward_node) + visit(forward_node) for s in solids: - visit(s) + visit(s.name) return (forward_edges, backward_edges) class SolidGraph: - def __init__(self, solids): - check.list_param(solids, 'solids', of_type=dagster.core.definitions.SolidDefinition) + def __init__(self, solids, dependency_structure): + + solids = check.list_param(solids, 'solids', of_type=SolidDefinition) + + if dependency_structure is None: + self.dep_structure = construct_dependency_structure_from_solids_only(solids) + else: + self.dep_structure = check.inst_param( + dependency_structure, 'dependency_structure', DependencyStructure + ) + self._solid_dict = {solid.name: solid for solid in solids} solid_names = set([solid.name for solid in solids]) @@ -55,7 +63,7 @@ def __init__(self, solids): all_inputs[input_def.name] = input_def self._all_inputs = all_inputs - self.forward_edges, self.backward_edges = create_adjacency_lists(solids) + self.forward_edges, self.backward_edges = create_adjacency_lists(solids, self.dep_structure) self.topological_order = toposort_flatten(self.backward_edges, sort=True) self._transitive_deps = {} @@ -76,9 +84,10 @@ def transitive_dependencies_of(self, solid_name): trans_deps = set() for inp in self._solid_dict[solid_name].inputs: - if inp.depends_on: - trans_deps.add(inp.depends_on.name) - trans_deps.union(self.transitive_dependencies_of(inp.depends_on.name)) + if self.dep_structure.has_dep(solid_name, inp.name): + dep_target = self.dep_structure.get_dep_target(solid_name, inp.name) + trans_deps.add(dep_target.solid_name) + trans_deps.union(self.transitive_dependencies_of(dep_target.solid_name)) self._transitive_deps[solid_name] = trans_deps return self._transitive_deps[solid_name] @@ -126,10 +135,11 @@ def visit(solid): if inp.name in input_set: continue - if inp.is_external: - unprovided_inputs.add(inp.name) + if self.dep_structure.has_dep(solid.name, inp.name): + dep_target = self.dep_structure.get_dep_target(solid.name, inp.name) + visit(self._solid_dict[dep_target.solid_name]) else: - visit(inp.depends_on) + unprovided_inputs.add(inp.name) visit(output_solid) @@ -148,15 +158,56 @@ def visit(solid): involved_solids.add(solid.name) for input_def in solid.inputs: - if input_def.is_external: + if not self.dep_structure.has_dep(solid.name, input_def.name): continue - if input_def.depends_on.name in from_solid_set: + from_solid = self.dep_structure.get_dep_target( + solid.name, input_def.name + ).solid_name + + if from_solid in from_solid_set: continue - visit(input_def.depends_on) + visit(self._solid_dict[from_solid]) for to_solid in to_solids: visit(self._solid_dict[to_solid]) - return SolidGraph([self._solid_dict[name] for name in involved_solids]) + return SolidGraph([self._solid_dict[name] for name in involved_solids], self.dep_structure) + + +def all_depended_on_solids(pipeline): + check.inst_param(pipeline, 'pipeline', PipelineDefinition) + dep_struct = pipeline.dependency_structure + for solid in pipeline.solids: + for input_def in solid.inputs: + if dep_struct.has_dep(solid.name, input_def.name): + dep_target = dep_struct.get_dep_target(solid.name, input_def.name) + yield pipeline.solid_named(dep_target.solid_name) + + +def all_sink_solids(pipeline): + check.inst_param(pipeline, 'pipeline', PipelineDefinition) + all_names = set([solid.name for solid in pipeline.solids]) + all_depended_on_names = set([solid.name for solid in all_depended_on_solids(pipeline)]) + return all_names.difference(all_depended_on_names) + + +def create_subgraph(pipeline, from_solids, through_solids): + check.inst_param(pipeline, 'pipeline', PipelineDefinition) + check.opt_list_param(from_solids, 'from_solids', of_type=str) + check.opt_list_param(through_solids, 'through_solids', of_type=str) + + solid_graph = SolidGraph(pipeline.solids, pipeline.dependency_structure) + + if not through_solids: + through_solids = list(all_sink_solids(pipeline)) + + if not from_solids: + all_deps = set() + for through_solid in through_solids: + all_deps.union(solid_graph.transitive_dependencies_of(through_solid)) + + from_solids = list(all_deps) + + return solid_graph.create_execution_subgraph(from_solids, through_solids) diff --git a/python_modules/dagster/dagster/core/validation.py b/python_modules/dagster/dagster/core/validation.py new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/python_modules/dagster/dagster/graphviz.py b/python_modules/dagster/dagster/graphviz.py index d1f354a7d93a8..fd033a07fca9f 100644 --- a/python_modules/dagster/dagster/graphviz.py +++ b/python_modules/dagster/dagster/graphviz.py @@ -23,7 +23,10 @@ def build_graphviz_graph(pipeline): scoped_name = solid.name + '.' + input_def.name graphviz_graph.edge(scoped_name, solid.name) - if input_def.depends_on: - graphviz_graph.edge(input_def.depends_on.name, scoped_name) + if pipeline.dependency_structure.has_dep(solid.name, input_def.name): + dep_target = pipeline.dependency_structure.get_dep_target( + solid.name, input_def.name + ) + graphviz_graph.edge(dep_target.solid_name, scoped_name) return graphviz_graph diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py index 0160d399e9932..899bea584eb66 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py @@ -1,20 +1,25 @@ import os +import pytest import pandas as pd import dagster from dagster import ( - check, config, InputDefinition, OutputDefinition, SolidDefinition, ArgumentDefinition + check, + config, + InputDefinition, + OutputDefinition, + SolidDefinition, + ArgumentDefinition, + PipelineDefinition, ) from dagster.core import types from dagster.core.decorators import solid from dagster.core.execution import ( - InMemoryInputManager, + InMemoryEnv, ExecutionContext, - _read_source, execute_pipeline_iterator, output_single_solid, - _pipeline_solid_in_memory, execute_pipeline, execute_single_solid, ) @@ -57,17 +62,6 @@ def create_test_context(): return ExecutionContext() -def test_pandas_input(): - csv_input = InputDefinition('num_csv', dagster_pd.DataFrame) - df = _read_source( - create_test_context(), csv_input.source_of_type('CSV'), - {'path': script_relative_path('num.csv')} - ) - - assert isinstance(df, pd.DataFrame) - assert df.to_dict('list') == {'num1': [1, 3], 'num2': [2, 4]} - - def test_pandas_solid(): csv_input = InputDefinition('num_csv', dagster_pd.DataFrame) @@ -202,7 +196,7 @@ def sum_sq_table_renamed_input(sum_table_renamed): return sum_table_renamed -def create_sum_sq_table(sum_table_solid): +def create_mult_table(sum_table_solid): def transform(_context, args): sum_df = args['sum_table'] sum_df['sum_squared'] = sum_df['sum'] * sum_df['sum'] @@ -237,15 +231,23 @@ def test_pandas_csv_in_memory(): assert df.to_dict('list') == {'num1': [1, 3], 'num2': [2, 4], 'sum': [3, 7]} +@pytest.mark.skip('do not use _pipeline_solid_in_memory') def test_two_step_pipeline_in_memory(): sum_table_solid = create_sum_table() - mult_table_solid = create_sum_sq_table(sum_table_solid) + mult_table_solid = create_mult_table(sum_table_solid) + pipeline = PipelineDefinition(solids=[sum_table_solid, mult_table_solid]) context = create_test_context() df = get_solid_transformed_value(context, sum_table_solid, get_num_csv_environment('sum_table')) input_values = {'sum_table': df} - input_manager = InMemoryInputManager(context, input_values) + env = InMemoryEnv( + context, + pipeline, + {'mult_table': input_values}, + from_solids=['mult_table'], + through_solids=['mult_table'], + ) mult_df = _pipeline_solid_in_memory( - context, input_manager, mult_table_solid, input_values + context, env, mult_table_solid, input_values ).transformed_value assert mult_df.to_dict('list') == { 'num1': [1, 3], @@ -255,12 +257,20 @@ def test_two_step_pipeline_in_memory(): } +def _sum_only_pipeline(): + return PipelineDefinition(solids=[sum_table, sum_sq_table]) + + +@pytest.mark.skip('do not use _pipeline_solid_in_memory') def test_two_step_pipeline_in_memory_decorator_style(): context = create_test_context() df = get_solid_transformed_value(context, sum_table, get_num_csv_environment('sum_table')) input_values = {'sum_df': df} mult_df = _pipeline_solid_in_memory( - context, InMemoryInputManager(context, input_values), sum_sq_table, input_values + context, + InMemoryEnv(context, _sum_only_pipeline(), {'sum_sq_table': input_values}), + sum_sq_table, + input_values, ).transformed_value assert mult_df.to_dict('list') == { 'num1': [1, 3], @@ -374,8 +384,12 @@ def sum_mult_transform(_context, args): return (num_table_solid, sum_table_solid, mult_table_solid, sum_mult_table_solid) +@pytest.mark.skip('do not use _pipeline_solid_in_memory for these') def test_diamond_dag_run(): - num_table_solid, sum_table_solid, mult_table_solid, sum_mult_table_solid = create_diamond_dag() + solids = create_diamond_dag() + num_table_solid, sum_table_solid, mult_table_solid, sum_mult_table_solid = solids + + pipeline = PipelineDefinition(solids=list(solids)) context = create_test_context() @@ -388,14 +402,26 @@ def test_diamond_dag_run(): input_values = {'num_table': num_table_df} sum_df = _pipeline_solid_in_memory( - context, InMemoryInputManager(context, input_values), sum_table_solid, input_values + context, + InMemoryEnv( + context, + pipeline, + {'sum_table': input_values}, + from_solids=['sum_table'], + through_solids=['sum_table'], + ), + sum_table_solid, + input_values, ).transformed_value assert sum_df.to_dict('list') == {'num1': [1, 3], 'num2': [2, 4], 'sum': [3, 7]} input_values = {'num_table': num_table_df} mult_df = _pipeline_solid_in_memory( - context, InMemoryInputManager(context, input_values), mult_table_solid, input_values + context, + InMemoryEnv(context, pipeline, input_values), + mult_table_solid, + input_values, ).transformed_value assert mult_df.to_dict('list') == {'num1': [1, 3], 'num2': [2, 4], 'mult': [2, 12]} @@ -403,7 +429,7 @@ def test_diamond_dag_run(): input_values = {'sum_table': sum_df, 'mult_table': mult_df} sum_mult_df = _pipeline_solid_in_memory( - context, InMemoryInputManager(context, input_values), sum_mult_table_solid, input_values + context, InMemoryEnv(context, pipeline, input_values), sum_mult_table_solid, input_values ).transformed_value assert sum_mult_df.to_dict('list') == { From 01152114a4c8085a833647e777a559fe117e6533 Mon Sep 17 00:00:00 2001 From: Nick Schrock Date: Sat, 18 Aug 2018 08:31:56 -0700 Subject: [PATCH 002/103] Move dependencies out of SolidDefinition This moves dependencies from a solid-level concern to a pipeline-wide concern. This will make solid reuse much more straightforward, as well as other things. --- python_modules/dagster/dagster/__init__.py | 1 + .../dagster/dagster/core/compute_nodes.py | 283 ++++++++---------- .../core/core_tests/test_naming_collisions.py | 29 +- .../core/core_tests/test_pipeline_errors.py | 22 +- .../core_tests/test_pipeline_execution.py | 83 +++-- .../core/core_tests/test_pipeline_meta.py | 14 - .../test_transform_only_pipeline.py | 47 ++- .../dagster/dagster/core/definitions.py | 163 ++++++---- python_modules/dagster/dagster/core/graph.py | 48 +-- .../test_sql_project_pipeline.py | 41 ++- .../pandas_hello_world/pipeline.py | 20 +- .../sql_hello_world/pipeline.py | 19 +- .../sql_project_example/pipelines.py | 73 +++-- python_modules/dagster/dagster/graphviz.py | 6 +- ...est_pandas_hello_world_no_library_slide.py | 28 +- .../pandas_kernel_tests/test_pandas_solids.py | 70 +++-- .../test_basic_solid.py | 26 +- .../test_isolated_sql_tests.py | 33 +- .../test_isolated_templated_sql_tests.py | 79 +++-- .../subquery_builder_experimental.py | 6 - .../dagster/sqlalchemy_kernel/templated.py | 10 +- .../dagster/dagster/utils/compatability.py | 2 - 22 files changed, 634 insertions(+), 469 deletions(-) delete mode 100644 python_modules/dagster/dagster/core/core_tests/test_pipeline_meta.py diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index 9fabc3594dcbb..8816935342e44 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -6,6 +6,7 @@ from dagster.core.definitions import ( ArgumentDefinition, + DependencyDefinition, ExpectationDefinition, ExpectationResult, InputDefinition, diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index 42afb90bf5d0c..439f596f9090a 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -9,26 +9,34 @@ from dagster import (check, config) from dagster.utils.indenting_printer import IndentingPrinter -from dagster.utils.logging import (get_formatted_stack_trace, define_colored_console_logger, DEBUG) +from dagster.utils.logging import ( + get_formatted_stack_trace, + define_colored_console_logger, + ERROR, +) + from dagster.utils.timing import time_execution_scope from .argument_handling import validate_args from .definitions import ( - SolidDefinition, + DependencyStructure, + ExpectationDefinition, InputDefinition, + MaterializationDefinition, OutputDefinition, PipelineDefinition, - ExpectationDefinition, - MaterializationDefinition, + SolidDefinition, + SolidInputHandle, + SolidOutputHandle, ) from .execution_context import ExecutionContext from .errors import ( + DagsterExpectationFailedError, DagsterInvariantViolationError, DagsterUserCodeExecutionError, - DagsterExpectationFailedError, ) from .graph import create_subgraph @@ -38,7 +46,87 @@ DagsterType, ) -LOG_LEVEL = DEBUG +class ComputeResult(namedtuple('_ComputeResult', 'output_name value')): + def __new__(cls, output_name, value): + return super(ComputeResult, cls).__new__( + cls, + check.str_param(output_name, 'output_name'), + value, + ) + + +class ComputeNodeOutputHandle(namedtuple('_ComputeNodeOutputHandle', 'compute_node output_name')): + def __new__(cls, compute_node, output_name): + return super(ComputeNodeOutputHandle, cls).__new__( + cls, + compute_node=check.inst_param(compute_node, 'compute_node', ComputeNode), + output_name=check.str_param(output_name, 'output_name'), + ) + + # Make this hashable so it be a key in a dictionary + + def __str__(self): + return f'ComputeNodeOutputHandle(guid="{self.compute_node.guid}", output_name="{self.output_name}")' + + def __hash__(self): + return hash(self.compute_node.guid + self.output_name) + + def __eq__(self, other): + return ( + self.compute_node.guid == other.compute_node.guid + and self.output_name == other.output_name + ) + + +class ComputeNodeSuccessData(namedtuple('_ComputeNodeSuccessData', 'output_name value')): + def __new__(cls, output_name, value): + return super(ComputeNodeSuccessData, cls).__new__( + cls, + output_name=check.str_param(output_name, 'output_name'), + value=value, + ) + + +class ComputeNodeFailureData(namedtuple('_ComputeNodeFailureData', 'dagster_user_exception')): + def __new__(cls, dagster_user_exception): + return super(ComputeNodeFailureData, cls).__new__( + cls, + dagster_user_exception=check.inst_param( + dagster_user_exception, + 'dagster_user_exception', + DagsterUserCodeExecutionError, + ), + ) + + +class ComputeNodeResult( + namedtuple( + '_ComputeNodeResult', + 'success compute_node tag success_data failure_data ', + ) +): + @staticmethod + def success_result(compute_node, tag, success_data): + return ComputeNodeResult( + success=True, + compute_node=check.inst_param(compute_node, 'compute_node', ComputeNode), + tag=check.inst_param(tag, 'tag', ComputeNodeTag), + success_data=check.inst_param(success_data, 'success_data', ComputeNodeSuccessData), + failure_data=None, + ) + + @staticmethod + def failure_result(compute_node, tag, failure_data): + return ComputeNodeResult( + success=False, + compute_node=check.inst_param(compute_node, 'compute_node', ComputeNode), + tag=check.inst_param(tag, 'tag', ComputeNodeTag), + success_data=None, + failure_data=check.inst_param(failure_data, 'failure_data', ComputeNodeFailureData), + ) + + +LOG_LEVEL = ERROR logger = define_colored_console_logger('dagster-compute-nodes', LOG_LEVEL) @contextmanager @@ -98,28 +186,6 @@ def _execute_core_transform(context, solid_transform_fn, values_dict): return transformed_value -class ComputeNodeOutputHandle(namedtuple('_ComputeNodeOutputHandle', 'compute_node output_name')): - def __new__(cls, compute_node, output_name): - return super(ComputeNodeOutputHandle, cls).__new__( - cls, - compute_node=check.inst_param(compute_node, 'compute_node', ComputeNode), - output_name=check.str_param(output_name, 'output_name'), - ) - - # Make this hashable so it be a key in a dictionary - - def __str__(self): - return f'ComputeNodeOutputHandle(guid="{self.compute_node.guid}", output_name="{self.output_name}")' - - def __hash__(self): - return hash(self.compute_node.guid + self.output_name) - - def __eq__(self, other): - return ( - self.compute_node.guid == other.compute_node.guid - and self.output_name == other.output_name - ) - class ComputeNodeInput: def __init__(self, name, dagster_type, prev_output_handle): @@ -235,51 +301,6 @@ def output_named(self, name): check.failed(f'output {name} not found') -class ComputeNodeSuccessData(namedtuple('_ComputeNodeSuccessData', 'output_name value')): - def __new__(cls, output_name, value): - return super(ComputeNodeSuccessData, cls).__new__( - cls, - output_name=check.str_param(output_name, 'output_name'), - value=value, - ) - -class ComputeNodeFailureData(namedtuple('_ComputeNodeFailureData', 'dagster_user_exception')): - def __new__(cls, dagster_user_exception): - return super(ComputeNodeFailureData, cls).__new__( - cls, - dagster_user_exception=check.inst_param( - dagster_user_exception, - 'dagster_user_exception', - DagsterUserCodeExecutionError, - ), - ) - -class ComputeNodeResult( - namedtuple( - '_ComputeNodeResult', - 'success compute_node tag success_data failure_data ', - ) -): - - @staticmethod - def success_result(compute_node, tag, success_data): - return ComputeNodeResult( - success=True, - compute_node=check.inst_param(compute_node, 'compute_node', ComputeNode), - tag=check.inst_param(tag, 'tag', ComputeNodeTag), - success_data=check.inst_param(success_data, 'success_data', ComputeNodeSuccessData), - failure_data=None, - ) - - @staticmethod - def failure_result(compute_node, tag, failure_data): - return ComputeNodeResult( - success=False, - compute_node=check.inst_param(compute_node, 'compute_node', ComputeNode), - tag=check.inst_param(tag, 'tag', ComputeNodeTag), - success_data=None, - failure_data=check.inst_param(failure_data, 'failure_data', ComputeNodeFailureData), - ) def execute_compute_nodes(context, compute_nodes): @@ -316,13 +337,6 @@ def __init__(self, *, sync_compute_fn, **kwargs): super().__init__(compute_fn=_yieldify(sync_compute_fn), **kwargs) -class ComputeResult(namedtuple('_ComputeResult', 'output_name value')): - def __new__(cls, output_name, value): - return super(ComputeResult, cls).__new__( - cls, - check.str_param(output_name, 'output_name'), - value, - ) def create_compute_node_from_source_config(solid, input_name, source_config): check.inst_param(solid, 'solid', SolidDefinition) @@ -357,24 +371,6 @@ def create_compute_node_from_source_config(solid, input_name, source_config): tag=ComputeNodeTag.SOURCE, solid=solid, ) - -class LogicalSolidOutput(namedtuple('_LogicalSolidOutput', 'solid_name output_name')): - def __new__(cls, solid_name, output_name): - return super(LogicalSolidOutput, cls).__new__( - cls, - check.str_param(solid_name, 'solid_name'), - check.str_param(output_name, 'output_name'), - ) - -class LogicalSolidInput(namedtuple('_SourceCnDictKey', 'solid_name input_name')): - def __new__(cls, solid_name, input_name): - return super(LogicalSolidInput, cls).__new__( - cls, - check.str_param(solid_name, 'solid_name'), - check.str_param(input_name, 'input_name'), - ) - - def create_source_compute_node_dict_from_environment(pipeline, environment): check.inst_param(pipeline, 'pipeline', PipelineDefinition) check.inst_param(environment, 'environment', config.Environment) @@ -384,8 +380,7 @@ def create_source_compute_node_dict_from_environment(pipeline, environment): solid = pipeline.solid_named(solid_name) for input_name, source_config in sources_by_input.items(): compute_node = create_compute_node_from_source_config(solid, input_name, source_config) - source_cn_dict[LogicalSolidInput(solid_name, input_name)] = compute_node - + source_cn_dict[solid.input_handle(input_name)] = compute_node return source_cn_dict @@ -400,8 +395,10 @@ def create_source_compute_node_dict_from_input_values(pipeline, input_values): source_cn_dict = {} for solid_name, sources_by_input in input_values.items(): + solid = pipeline.solid_named(solid_name) for input_name, input_value in sources_by_input.items(): - source_cn_dict[LogicalSolidInput(solid_name, input_name)] = SingleSyncOutputComputeNode( + input_handle = solid.input_handle(input_name) + source_cn_dict[input_handle] = SingleSyncOutputComputeNode( friendly_name=f'{solid_name}.{input_name}.stub', node_inputs=[], # This is just a stub of a pre-existing value, so we are not @@ -557,29 +554,31 @@ def create_expectations_cn_graph(solid, inout_def, prev_node_output_handle, tag) create_cn_output_handle(join_cn, output_name), ) -LogicalSolidOutput = namedtuple('LogicalSolidOutput', 'solid_name output_name') - def _prev_node_handle(dep_structure, solid, input_def, source_cn_dict, logical_output_mapper): - check.inst_param(dep_structure, 'dep_structure', PipelineDependencyStructure) + check.inst_param(dep_structure, 'dep_structure', DependencyStructure) check.inst_param(solid, 'solid', SolidDefinition) check.inst_param(input_def, 'input_def', InputDefinition) check.dict_param( source_cn_dict, 'source_cn_dict', - key_type=LogicalSolidInput, + key_type=SolidInputHandle, value_type=ComputeNode, ) check.inst_param(logical_output_mapper, 'mapper', LogicalSolidOutputMapper) - logical_solid_input = LogicalSolidInput(solid.name, input_def.name) + input_handle = solid.input_handle(input_def.name) - if logical_solid_input in source_cn_dict: - return create_cn_output_handle(source_cn_dict[logical_solid_input], SOURCE_OUTPUT) + if input_handle in source_cn_dict: + return create_cn_output_handle(source_cn_dict[input_handle], SOURCE_OUTPUT) else: - check.invariant(dep_structure.input_has_dep(logical_solid_input)) - logical_solid_output = dep_structure.get_dep(logical_solid_input) - return logical_output_mapper.get_handle_for_logical_output(logical_solid_output) + check.invariant( + dep_structure.has_dep(input_handle), + f'{input_handle} not found in dependency structure', + ) + + solid_output_handle = dep_structure.get_dep(input_handle) + return logical_output_mapper.get_cn_output_handle(solid_output_handle) def create_cn_output_handle(compute_node, cn_output_name): check.inst_param(compute_node, 'compute_node', ComputeNode) @@ -587,51 +586,19 @@ def create_cn_output_handle(compute_node, cn_output_name): return ComputeNodeOutputHandle(compute_node, cn_output_name) -class PipelineDependencyStructure: - def __init__(self, pipeline): - check.inst_param(pipeline, 'pipeline', PipelineDefinition) - - edges = {} - for solid in pipeline.solids: - for input_def in solid.inputs: - if pipeline.dependency_structure.has_dep(solid.name, input_def.name): - logical_solid_input = LogicalSolidInput(solid.name, input_def.name) - - dep_target = pipeline.dependency_structure.get_dep_target( - solid.name, - input_def.name, - ) - - logical_solid_output = LogicalSolidOutput( - dep_target.solid_name, - dep_target.output_name, - ) - - edges[logical_solid_input] = logical_solid_output - - self.edges = edges - - def input_has_dep(self, logical_solid_input): - check.inst_param(logical_solid_input, 'logical_solid_input', LogicalSolidInput) - return logical_solid_input in self.edges - - def get_dep(self, logical_solid_input): - check.inst_param(logical_solid_input, 'logical_solid_input', LogicalSolidInput) - return self.edges[logical_solid_input] - - class LogicalSolidOutputMapper: def __init__(self): self._output_handles = {} - def set_mapping(self, logical_solid_output, cn_output_handle): - check.inst_param(logical_solid_output, 'logical_solid_output', LogicalSolidOutput) + def set_mapping(self, solid_output_handle, cn_output_handle): + check.inst_param(solid_output_handle, 'solid_output_handle', SolidOutputHandle) check.inst_param(cn_output_handle, 'cn_output_handle', ComputeNodeOutputHandle) - self._output_handles[logical_solid_output] = cn_output_handle + self._output_handles[solid_output_handle] = cn_output_handle + + def get_cn_output_handle(self, solid_output_handle): + check.inst_param(solid_output_handle, 'solid_output_handle', SolidOutputHandle) + return self._output_handles[solid_output_handle] - def get_handle_for_logical_output(self, logical_solid_output): - check.inst_param(logical_solid_output, 'logical_solid_output', LogicalSolidOutput) - return self._output_handles[logical_solid_output] def create_compute_node_graph_from_source_dict( pipeline, @@ -647,7 +614,7 @@ def create_compute_node_graph_from_source_dict( check.dict_param( source_cn_dict, 'source_cn_dict', - key_type=LogicalSolidInput, + key_type=SolidInputHandle, value_type=ComputeNode, ) @@ -657,7 +624,7 @@ def create_compute_node_graph_from_source_dict( of_type=config.Materialization, ) - dep_structure = PipelineDependencyStructure(pipeline) + dep_structure = pipeline.dependency_structure check.bool_param(evaluate_expectations, 'evaluate_expectations') @@ -720,12 +687,12 @@ def create_compute_node_graph_from_source_dict( ) compute_nodes = compute_nodes + expectations_graph.nodes logical_output_mapper.set_mapping( - LogicalSolidOutput(topo_solid.name, output_def.name), + topo_solid.output_handle(output_def.name), expectations_graph.terminal_cn_output_handle, ) else: logical_output_mapper.set_mapping( - LogicalSolidOutput(topo_solid.name, output_def.name), + topo_solid.output_handle(output_def.name), create_cn_output_handle(solid_transform_cn, output_def.name), ) @@ -733,14 +700,12 @@ def create_compute_node_graph_from_source_dict( compute_nodes.append(solid_transform_cn) for materialization in materializations: + mat_solid = pipeline.solid_named(materialization.solid) mat_cn = _construct_materialization_cn( pipeline, materialization, - logical_output_mapper.get_handle_for_logical_output( - LogicalSolidOutput( - materialization.solid, - materialization.output_name - ) + logical_output_mapper.get_cn_output_handle( + mat_solid.output_handle(materialization.output_name) ), ) compute_nodes.append(mat_cn) diff --git a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py index acb1ad85d1bd4..fbed4f3f2be63 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py +++ b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py @@ -120,25 +120,24 @@ def test_execute_dep_solid_different_input_name(): 'second_solid', inputs=[ InputDefinition( - name='a_dependency', + name='an_input', - # depends_on=first_solid, ), ], - transform_fn=lambda context, args: args['a_dependency'] + args['a_dependency'], + transform_fn=lambda context, args: args['an_input'] + args['an_input'], output=dagster.OutputDefinition(), ) + pipeline = dagster.PipelineDefinition( solids=[first_solid, second_solid], - dependencies=[ - DependencyDefinition( - from_solid='second_solid', - from_input='a_dependency', - to_solid='first_solid', - ) - ] + dependencies={ + 'second_solid': { + 'an_input': DependencyDefinition('first_solid') + } + } ) + result = dagster.execute_pipeline( pipeline, environment=config.Environment( @@ -199,7 +198,6 @@ def s_fn(arg_dict, executed, key): argument_def_dict={'name': ArgumentDefinition(types.String)}, ), ], - depends_on=table_one, ), InputDefinition( name='table_two', @@ -217,7 +215,14 @@ def s_fn(arg_dict, executed, key): output=dagster.OutputDefinition(), ) - pipeline = dagster.PipelineDefinition(solids=[table_one, table_two]) + pipeline = dagster.PipelineDefinition( + solids=[table_one, table_two], + dependencies={ + 'table_two' : { + 'table_one' : DependencyDefinition('table_one') + } + } + ) sources = { 'table_one': { diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py index f091ee795cdbb..dedbce41e5e25 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py @@ -6,6 +6,7 @@ from dagster.core.definitions import ( SolidDefinition, InputDefinition, + DependencyDefinition, ) from dagster.core.execution import ( DagsterExecutionFailureReason, @@ -28,9 +29,11 @@ def silencing_default_context(): } -def silencing_pipeline(solids): +def silencing_pipeline(solids, dependencies=None): return dagster.PipelineDefinition( - solids=solids, context_definitions=silencing_default_context() + solids=solids, + dependencies=dependencies, + context_definitions=silencing_default_context(), ) @@ -193,10 +196,7 @@ def transform_fn(_context, args): solid_c = SolidDefinition( name='C', - inputs=[ - InputDefinition(name='A', depends_on=solid_a), - InputDefinition(name='B', depends_on=solid_b), - ], + inputs=[InputDefinition(name='A'), InputDefinition(name='B')], transform_fn=transform_fn, output=dagster.OutputDefinition(), ) @@ -211,7 +211,15 @@ def transform_fn(_context, args): } } ) - pipeline = silencing_pipeline(solids=[solid_a, solid_b, solid_c]) + pipeline = silencing_pipeline( + solids=[solid_a, solid_b, solid_c], + dependencies={ + 'C': { + 'A': DependencyDefinition(solid_a.name), + 'B': DependencyDefinition(solid_b.name), + } + } + ) pipeline_result = execute_pipeline( pipeline, environment=environment, diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py index b6f19815d0f06..3eaef2b3e0b72 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py @@ -1,13 +1,16 @@ import pytest -from dagster import (check, config) - -from dagster.core.definitions import ( +from dagster import ( + DependencyDefinition, OutputDefinition, PipelineDefinition, SolidDefinition, - construct_dependency_structure_from_solids_only, + check, + config, ) + +from dagster.core.definitions import DependencyStructure + from dagster.core.graph import (create_adjacency_lists, SolidGraph) from dagster.core.execution import ( execute_pipeline_iterator, @@ -44,7 +47,6 @@ def create_solid_with_deps(name, *solid_deps): solid_dep.name, source_fn=create_dep_input_fn(solid_dep.name), argument_def_dict={}, - depends_on=solid_dep, ) for solid_dep in solid_deps ] @@ -84,13 +86,14 @@ def root_transform(_context, args): ) -def _do_construct(solids): - return create_adjacency_lists(solids, construct_dependency_structure_from_solids_only(solids)) +def _do_construct(solids, dependencies): + dependency_structure = DependencyStructure(solids, dependencies) + return create_adjacency_lists(solids, dependency_structure) def test_empty_adjaceny_lists(): solids = [create_root_solid('a_node')] - forward_edges, backwards_edges = _do_construct(solids) + forward_edges, backwards_edges = _do_construct(solids, {}) assert forward_edges == {'a_node': set()} assert backwards_edges == {'a_node': set()} @@ -100,14 +103,21 @@ def test_single_dep_adjacency_lists(): node_a = create_root_solid('A') node_b = create_solid_with_deps('B', node_a) - forward_edges, backwards_edges = _do_construct([node_a, node_b]) + forward_edges, backwards_edges = _do_construct( + [node_a, node_b], + { + 'B': { + 'A': DependencyDefinition('A'), + }, + }, + ) assert forward_edges == {'A': {'B'}, 'B': set()} assert backwards_edges == {'B': {'A'}, 'A': set()} -def graph_from_solids_only(solids): - return SolidGraph(solids, construct_dependency_structure_from_solids_only(solids)) +def graph_from_solids_only(solids, dependencies): + return SolidGraph(solids, DependencyStructure(solids, dependencies)) def test_diamond_deps_adjaceny_lists(): @@ -118,11 +128,30 @@ def test_diamond_deps_adjaceny_lists(): node_c = create_solid_with_deps('C', node_a) node_d = create_solid_with_deps('D', node_b, node_c) - forward_edges, backwards_edges = _do_construct([node_a, node_b, node_c, node_d]) + forward_edges, backwards_edges = _do_construct( + [node_a, node_b, node_c, node_d], + diamond_deps(), + ) + assert forward_edges == {'A': {'B', 'C'}, 'B': {'D'}, 'C': {'D'}, 'D': set()} assert backwards_edges == {'D': {'B', 'C'}, 'B': {'A'}, 'C': {'A'}, 'A': set()} +def diamond_deps(): + return { + 'B': { + 'A': DependencyDefinition('A') + }, + 'C': { + 'A': DependencyDefinition('A') + }, + 'D': { + 'B': DependencyDefinition('B'), + 'C': DependencyDefinition('C'), + } + } + + def test_disconnected_graphs_adjaceny_lists(): # A <-- B # C <-- D @@ -132,18 +161,30 @@ def test_disconnected_graphs_adjaceny_lists(): node_c = create_root_solid('C') node_d = create_solid_with_deps('D', node_c) - forward_edges, backwards_edges = _do_construct([node_a, node_b, node_c, node_d]) + forward_edges, backwards_edges = _do_construct( + [node_a, node_b, node_c, node_d], { + 'B': { + 'A': DependencyDefinition('A') + }, + 'D': { + 'C': DependencyDefinition('C'), + } + } + ) assert forward_edges == {'A': {'B'}, 'B': set(), 'C': {'D'}, 'D': set()} assert backwards_edges == {'B': {'A'}, 'A': set(), 'D': {'C'}, 'C': set()} -def create_diamond_graph(): +def create_diamond_solids(): node_a = create_root_solid('A') node_b = create_solid_with_deps('B', node_a) node_c = create_solid_with_deps('C', node_a) node_d = create_solid_with_deps('D', node_b, node_c) + return [node_d, node_c, node_b, node_a] - return graph_from_solids_only([node_d, node_c, node_b, node_a]) + +def create_diamond_graph(): + return graph_from_solids_only(create_diamond_solids(), diamond_deps()) def test_diamond_toposort(): @@ -153,7 +194,7 @@ def test_diamond_toposort(): def test_single_node_unprovided_inputs(): node_a = create_root_solid('A') - solid_graph = graph_from_solids_only(solids=[node_a]) + solid_graph = graph_from_solids_only([node_a], {}) assert solid_graph.compute_unprovided_inputs('A', ['A_input']) == set() assert solid_graph.compute_unprovided_inputs('A', []) == set(['A_input']) @@ -190,7 +231,7 @@ def test_diamond_toposort_unprovided_inputs(): def test_unprovided_input_param_invariants(): node_a = create_root_solid('A') - solid_graph = graph_from_solids_only(solids=[node_a]) + solid_graph = graph_from_solids_only([node_a], {}) with pytest.raises(check.ParameterCheckError): solid_graph.compute_unprovided_inputs('B', []) @@ -201,7 +242,7 @@ def test_unprovided_input_param_invariants(): def test_execution_subgraph_one_node(): node_a = create_root_solid('A') - solid_graph = graph_from_solids_only(solids=[node_a]) + solid_graph = graph_from_solids_only([node_a], {}) execution_graph = solid_graph.create_execution_subgraph( from_solids=['A'], @@ -262,8 +303,7 @@ def assert_all_results_equivalent(expected_results, result_results): def test_pipeline_execution_graph_diamond(): - solid_graph = create_diamond_graph() - pipeline = PipelineDefinition(solids=solid_graph.solids) + pipeline = PipelineDefinition(solids=create_diamond_solids(), dependencies=diamond_deps()) environment = config.Environment(sources={'A': {'A_input': config.Source('CUSTOM', {})}}) return _do_test(pipeline, lambda: execute_pipeline_iterator( pipeline, @@ -272,8 +312,7 @@ def test_pipeline_execution_graph_diamond(): def test_pipeline_execution_graph_diamond_in_memory(): - solid_graph = create_diamond_graph() - pipeline = PipelineDefinition(solids=solid_graph.solids) + pipeline = PipelineDefinition(solids=create_diamond_solids(), dependencies=diamond_deps()) input_values = {'A_input': [{'A_input': 'input_set'}]} return _do_test(pipeline, lambda: execute_pipeline_iterator_in_memory( ExecutionContext(), diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_meta.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_meta.py deleted file mode 100644 index 0a9d0bf1b8519..0000000000000 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_meta.py +++ /dev/null @@ -1,14 +0,0 @@ -import dagster -from dagster.core.definitions import InputDefinition -from dagster.core.decorators import solid - - -@solid() -def step_one_no_external_source(): - return {'foo': 'bar'} - - -@solid(inputs=[InputDefinition(name='foo_bar', depends_on=step_one_no_external_source)]) -def step_two(foo_bar): - foo_bar['foo'] = foo_bar['foo'] + foo_bar['foo'] - return foo_bar diff --git a/python_modules/dagster/dagster/core/core_tests/test_transform_only_pipeline.py b/python_modules/dagster/dagster/core/core_tests/test_transform_only_pipeline.py index 0d0319698f7c6..2935296518861 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_transform_only_pipeline.py +++ b/python_modules/dagster/dagster/core/core_tests/test_transform_only_pipeline.py @@ -1,7 +1,13 @@ -import dagster -from dagster import check -from dagster import config -from dagster.core.definitions import SolidDefinition +from dagster import ( + DependencyDefinition, + InputDefinition, + OutputDefinition, + PipelineDefinition, + SolidDefinition, + check, + config, + execute_pipeline, +) from dagster.utils.compatability import create_custom_source_input @@ -17,28 +23,34 @@ def test_execute_solid_with_dep_only_inputs_no_api(): name='step_one_solid', inputs=[], transform_fn=lambda context, args: _set_key_value(did_run_dict, 'step_one', True), - output=dagster.OutputDefinition(), + output=OutputDefinition(), ) only_dep_input = create_custom_source_input( name='step_one_solid', source_fn=lambda arg_dict: check.not_implemented('should not get here'), argument_def_dict={}, - depends_on=step_one_solid ) step_two_solid = SolidDefinition( name='step_two_solid', inputs=[only_dep_input], transform_fn=lambda context, args: _set_key_value(did_run_dict, 'step_two', True), - output=dagster.OutputDefinition(), + output=OutputDefinition(), ) - pipeline = dagster.PipelineDefinition(solids=[step_one_solid, step_two_solid]) + pipeline = PipelineDefinition( + solids=[step_one_solid, step_two_solid], + dependencies={ + 'step_two_solid': { + 'step_one_solid': DependencyDefinition('step_one_solid') + }, + }, + ) # from dagster.utils import logging - pipeline_result = dagster.execute_pipeline(pipeline, environment=config.Environment.empty()) + pipeline_result = execute_pipeline(pipeline, environment=config.Environment.empty()) assert pipeline_result.success @@ -56,19 +68,26 @@ def test_execute_solid_with_dep_only_inputs_with_api(): name='step_one_solid', inputs=[], transform_fn=lambda context, args: _set_key_value(did_run_dict, 'step_one', True), - output=dagster.OutputDefinition(), + output=OutputDefinition(), ) step_two_solid = SolidDefinition( name='step_two_solid', transform_fn=lambda context, args: _set_key_value(did_run_dict, 'step_two', True), - inputs=[dagster.InputDefinition(step_one_solid.name, depends_on=step_one_solid)], - output=dagster.OutputDefinition(), + inputs=[InputDefinition(step_one_solid.name)], + output=OutputDefinition(), ) - pipeline = dagster.PipelineDefinition(solids=[step_one_solid, step_two_solid]) + pipeline = PipelineDefinition( + solids=[step_one_solid, step_two_solid], + dependencies={ + 'step_two_solid': { + step_one_solid.name: DependencyDefinition(step_one_solid.name) + } + }, + ) - pipeline_result = dagster.execute_pipeline(pipeline, environment=config.Environment.empty()) + pipeline_result = execute_pipeline(pipeline, environment=config.Environment.empty()) for result in pipeline_result.result_list: assert result.success diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index ee21af58d0490..aaafa2d50435a 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -84,63 +84,53 @@ def _default_context_fn(_pipeline, args): class DependencyDefinition: - def __init__(self, from_solid, from_input, to_solid, to_output=DEFAULT_OUTPUT): - self.from_solid = check.str_param(from_solid, 'from_solid') - self.from_input = check.str_param(from_input, 'from_input') - self.to_solid = check.str_param(to_solid, 'to_solid') - self.to_output = check.str_param(to_output, 'to_output') + def __init__(self, solid, output=DEFAULT_OUTPUT, description=None): + self.solid = check.str_param(solid, 'solid') + self.output = check.str_param(output, 'output') + self.description = check.opt_str_param(description, 'description') DepTarget = namedtuple('DepTarget', 'solid_name output_name') class DependencyStructure: - def __init__(self, deps): - self.deps = check.list_param(deps, 'deps', of_type=DependencyDefinition) + def __init__(self, solids, dep_dict): + check.list_param(solids, 'solids', of_type=SolidDefinition) + check.dict_param(dep_dict, 'dep_dict', key_type=str, value_type=dict) + + def _solid_named(name): + for solid in solids: + if solid.name == name: + return solid + check.failed('not fouhnd') self._dep_lookup = defaultdict(dict) - for dep in deps: - self._dep_lookup[dep.from_solid][dep.from_input] = DepTarget( - solid_name=dep.to_solid, - output_name=dep.to_output, + + for solid_name, input_dict in dep_dict.items(): + check.dict_param( + input_dict, + 'input_dict', + key_type=str, + value_type=DependencyDefinition, ) - def has_dep(self, solid_name, input_name): - check.str_param(solid_name, 'solid_name') - check.str_param(input_name, 'input_name') - return input_name in self._dep_lookup.get(solid_name, {}) + for input_name, dep in input_dict.items(): + dep_solid = _solid_named(dep.solid) + output_def = dep_solid.output_def_named(dep.output) + self._dep_lookup[solid_name][input_name] = SolidOutputHandle(dep_solid, output_def) + + def has_dep(self, solid_input_handle): + check.inst_param(solid_input_handle, 'solid_input_handle', SolidInputHandle) + input_name = solid_input_handle.input_def.name + return input_name in self._dep_lookup.get(solid_input_handle.solid.name, {}) def deps_of_solid(self, solid_name): check.str_param(solid_name, 'solid_name') return list(self._dep_lookup[solid_name].values()) - def get_dep_target(self, solid_name, input_name): - check.str_param(solid_name, 'solid_name') - check.str_param(input_name, 'input_name') - return self._dep_lookup[solid_name][input_name] - - -def construct_dependency_structure_from_solids_only(solids): - check.list_param(solids, 'solids', SolidDefinition) - - deps = [] - for solid in solids: - for input_def in solid.inputs: - if input_def.depends_on: - - to_solid = input_def.depends_on - check.invariant(len(to_solid.outputs) == 1) - - deps.append( - DependencyDefinition( - from_solid=solid.name, - from_input=input_def.name, - to_solid=to_solid.name, - to_output=to_solid.outputs[0].name - ) - ) - - return DependencyStructure(deps) + def get_dep(self, solid_input_handle): + check.inst_param(solid_input_handle, 'solid_input_handle', SolidInputHandle) + return self._dep_lookup[solid_input_handle.solid.name][solid_input_handle.input_def.name] class PipelineDefinition: @@ -171,25 +161,14 @@ def __init__( self.solids = check.list_param(solids, 'solids', of_type=SolidDefinition) - solid_names = set([solid.name for solid in self.solids]) - for solid in solids: - for input_def in solid.inputs: - if input_def.depends_on: - check.invariant( - input_def.depends_on.name in solid_names, - f'''The solid {input_def.depends_on.name} was specified - as a dependency, however it does not exist in your - pipeline definition. Only the following exist: - {solid_names}''' - ) - - # self.solid_graph = SolidGraph(solids=solids) - - # TEMPORARY FOR MIGRATION - if dependencies is None: - self.dependency_structure = construct_dependency_structure_from_solids_only(solids) - else: - self.dependency_structure = DependencyStructure(dependencies) + dependencies = check.opt_dict_param( + dependencies, + 'dependencies', + key_type=str, + value_type=dict, + ) + + self.dependency_structure = DependencyStructure(solids, dependencies) @property def solid_names(self): @@ -288,7 +267,6 @@ def __init__( name, dagster_type=None, sources=None, - depends_on=None, expectations=None, input_callback=None, description=None @@ -299,7 +277,6 @@ def __init__( sources = dagster_type.default_sources self.sources = check.opt_list_param(sources, 'sources', of_type=SourceDefinition) - self.depends_on = check.opt_inst_param(depends_on, 'depends_on', SolidDefinition) self.dagster_type = check.opt_inst_param( dagster_type, 'dagster_type', types.DagsterType, types.Any @@ -394,6 +371,42 @@ def materialization_of_type(self, name): check.failed('Did not find materialization {type}'.format(type=name)) +class SolidInputHandle(namedtuple('_SolidInputHandle', 'solid input_def')): + def __new__(cls, solid, input_def): + return super(SolidInputHandle, cls).__new__( + cls, + check.inst_param(solid, 'solid', SolidDefinition), + check.inst_param(input_def, 'input_def', InputDefinition), + ) + + def __str__(self): + return f'SolidInputHandle(solid="{self.solid.name}", input_name="{self.input_def.name}")' + + def __hash__(self): + return hash((self.solid.name, self.input_def.name)) + + def __eq__(self, other): + return self.solid.name == other.solid.name and self.input_def.name == other.input_def.name + + +class SolidOutputHandle(namedtuple('_SolidOutputHandle', 'solid output')): + def __new__(cls, solid, output): + return super(SolidOutputHandle, cls).__new__( + cls, + check.inst_param(solid, 'solid', SolidDefinition), + check.inst_param(output, 'output', OutputDefinition), + ) + + def __str__(self): + return f'SolidOutputHandle(solid="{self.solid.name}", output.name="{self.output.name}")' + + def __hash__(self): + return hash((self.solid.name, self.output.name)) + + def __eq__(self, other): + return self.solid.name == other.solid.name and self.output.name == other.output.name + + # One or more inputs # The core computation in the native kernel abstraction # The output @@ -414,6 +427,14 @@ def __init__(self, name, inputs, transform_fn, output, description=None): self.outputs = [output] + input_handles = {} + for inp in self.inputs: + input_handles[inp.name] = SolidInputHandle(self, inp) + + self.input_handles = input_handles + + self.output_handles = {output.name: SolidOutputHandle(self, output)} + # Notes to self # Input Definitions @@ -433,6 +454,14 @@ def __init__(self, name, inputs, transform_fn, output, description=None): # - Compute (value, args) => Result # - Expectations + def input_handle(self, name): + check.str_param(name, 'name') + return self.input_handles[name] + + def output_handle(self, name): + check.str_param(name, 'name') + return self.output_handles[name] + @property def input_names(self): return [inp.name for inp in self.inputs] @@ -452,6 +481,14 @@ def input_def_named(self, name): check.failed('input {name} not found'.format(name=name)) + def output_def_named(self, name): + check.str_param(name, 'name') + for output in self.outputs: + if output.name == name: + return output + + check.failed('output {name} not found'.format(name=name)) + class __ArgumentValueSentinel: pass diff --git a/python_modules/dagster/dagster/core/graph.py b/python_modules/dagster/dagster/core/graph.py index 841f9fc23d16c..4a5cc19ab199b 100644 --- a/python_modules/dagster/dagster/core/graph.py +++ b/python_modules/dagster/dagster/core/graph.py @@ -6,7 +6,6 @@ DependencyStructure, PipelineDefinition, SolidDefinition, - construct_dependency_structure_from_solids_only, ) @@ -24,8 +23,8 @@ def visit(solid_name): visit_dict[solid_name] = True - for dep_target in dep_structure.deps_of_solid(solid_name): - forward_node = dep_target.solid_name + for output_handle in dep_structure.deps_of_solid(solid_name): + forward_node = output_handle.solid.name backward_node = solid_name if forward_node in forward_edges: forward_edges[forward_node].add(backward_node) @@ -43,12 +42,9 @@ def __init__(self, solids, dependency_structure): solids = check.list_param(solids, 'solids', of_type=SolidDefinition) - if dependency_structure is None: - self.dep_structure = construct_dependency_structure_from_solids_only(solids) - else: - self.dep_structure = check.inst_param( - dependency_structure, 'dependency_structure', DependencyStructure - ) + self.dep_structure = check.inst_param( + dependency_structure, 'dependency_structure', DependencyStructure + ) self._solid_dict = {solid.name: solid for solid in solids} @@ -83,11 +79,13 @@ def transitive_dependencies_of(self, solid_name): return self._transitive_deps[solid_name] trans_deps = set() - for inp in self._solid_dict[solid_name].inputs: - if self.dep_structure.has_dep(solid_name, inp.name): - dep_target = self.dep_structure.get_dep_target(solid_name, inp.name) - trans_deps.add(dep_target.solid_name) - trans_deps.union(self.transitive_dependencies_of(dep_target.solid_name)) + solid = self._solid_dict[solid_name] + for inp in solid.inputs: + input_handle = solid.input_handle(inp.name) + if self.dep_structure.has_dep(input_handle): + output_handle = self.dep_structure.get_dep(input_handle) + trans_deps.add(output_handle.solid.name) + trans_deps.union(self.transitive_dependencies_of(output_handle.solid.name)) self._transitive_deps[solid_name] = trans_deps return self._transitive_deps[solid_name] @@ -135,9 +133,11 @@ def visit(solid): if inp.name in input_set: continue - if self.dep_structure.has_dep(solid.name, inp.name): - dep_target = self.dep_structure.get_dep_target(solid.name, inp.name) - visit(self._solid_dict[dep_target.solid_name]) + input_handle = solid.input_handle(inp.name) + + if self.dep_structure.has_dep(input_handle): + output_handle = self.dep_structure.get_dep(input_handle) + visit(self._solid_dict[output_handle.solid.name]) else: unprovided_inputs.add(inp.name) @@ -158,12 +158,11 @@ def visit(solid): involved_solids.add(solid.name) for input_def in solid.inputs: - if not self.dep_structure.has_dep(solid.name, input_def.name): + input_handle = solid.input_handle(input_def.name) + if not self.dep_structure.has_dep(input_handle): continue - from_solid = self.dep_structure.get_dep_target( - solid.name, input_def.name - ).solid_name + from_solid = self.dep_structure.get_dep(input_handle).solid.name if from_solid in from_solid_set: continue @@ -181,9 +180,10 @@ def all_depended_on_solids(pipeline): dep_struct = pipeline.dependency_structure for solid in pipeline.solids: for input_def in solid.inputs: - if dep_struct.has_dep(solid.name, input_def.name): - dep_target = dep_struct.get_dep_target(solid.name, input_def.name) - yield pipeline.solid_named(dep_target.solid_name) + input_handle = solid.input_handle(input_def.name) + if dep_struct.has_dep(input_handle): + output_handle = dep_struct.get_dep(input_handle) + yield pipeline.solid_named(output_handle.solid.name) def all_sink_solids(pipeline): diff --git a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py index 3103fd83aa560..c996453446e72 100644 --- a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py +++ b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py @@ -1,6 +1,6 @@ import sqlalchemy as sa import dagster -from dagster import (config, InputDefinition) +from dagster import (config, InputDefinition, DependencyDefinition) import dagster.sqlalchemy_kernel as dagster_sa from dagster.utils.test import script_relative_path @@ -26,7 +26,7 @@ def create_persisted_context(): return dagster_sa.create_sql_alchemy_context_from_engine(engine=engine) -def create_mem_sql_pipeline_context_tuple(solids): +def create_mem_sql_pipeline_context_tuple(solids, dependencies=None): default_def = dagster.PipelineContextDefinition( argument_def_dict={}, context_fn=lambda _pipeline, _args: in_mem_context(), @@ -36,10 +36,12 @@ def create_mem_sql_pipeline_context_tuple(solids): context_fn=lambda _pipeline, _args: create_persisted_context(), ) return dagster.PipelineDefinition( - solids=solids, context_definitions={ + solids=solids, + dependencies=dependencies, + context_definitions={ 'default': default_def, 'persisted': persisted_def - } + }, ) @@ -68,14 +70,16 @@ def test_sql_populate_tables(): create_all_tables_solids = _get_project_solid('create_all_tables') populate_num_table_solid = _get_project_solid( - 'populate_num_table', - inputs=[ - InputDefinition(create_all_tables_solids.name, depends_on=create_all_tables_solids) - ] + 'populate_num_table', inputs=[InputDefinition(create_all_tables_solids.name)] ) pipeline = create_mem_sql_pipeline_context_tuple( - solids=[create_all_tables_solids, populate_num_table_solid] + solids=[create_all_tables_solids, populate_num_table_solid], + dependencies={ + populate_num_table_solid.name: { + create_all_tables_solids.name: DependencyDefinition(create_all_tables_solids.name) + } + } ) pipeline_result = dagster.execute_pipeline(pipeline, environment=config.Environment.empty()) @@ -92,19 +96,17 @@ def create_full_pipeline(): populate_num_table_solid = _get_project_solid( 'populate_num_table', - inputs=[InputDefinition('create_all_tables_solids', depends_on=create_all_tables_solids)], + inputs=[InputDefinition('create_all_tables_solids')], ) insert_into_sum_table_solid = _get_project_solid( 'insert_into_sum_table', - inputs=[InputDefinition('populate_num_table_solid', depends_on=populate_num_table_solid)], + inputs=[InputDefinition('populate_num_table_solid')], ) insert_into_sum_sq_table_solid = _get_project_solid( 'insert_into_sum_sq_table', - inputs=[ - InputDefinition('insert_into_sum_sq_table', depends_on=insert_into_sum_table_solid) - ], + inputs=[InputDefinition('insert_into_sum_sq_table')], ) return create_mem_sql_pipeline_context_tuple( @@ -114,6 +116,17 @@ def create_full_pipeline(): insert_into_sum_table_solid, insert_into_sum_sq_table_solid, ], + dependencies={ + populate_num_table_solid.name: { + 'create_all_tables_solids': DependencyDefinition(create_all_tables_solids.name) + }, + insert_into_sum_table_solid.name: { + 'populate_num_table_solid': DependencyDefinition(populate_num_table_solid.name) + }, + insert_into_sum_sq_table_solid.name: { + 'insert_into_sum_sq_table': DependencyDefinition(insert_into_sum_table_solid.name), + }, + } ) diff --git a/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py b/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py index 1016a33d6b407..99ba3c8f533b0 100644 --- a/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py +++ b/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py @@ -1,5 +1,11 @@ import dagster -from dagster import (solid, InputDefinition, OutputDefinition, PipelineDefinition) +from dagster import ( + DependencyDefinition, + InputDefinition, + OutputDefinition, + PipelineDefinition, + solid, +) import dagster.pandas_kernel as dagster_pd @@ -14,7 +20,7 @@ def sum_solid(num): @solid( - inputs=[InputDefinition('sum_df', dagster_pd.DataFrame, depends_on=sum_solid)], + inputs=[InputDefinition('sum_df', dagster_pd.DataFrame)], output=OutputDefinition(dagster_pd.DataFrame) ) def sum_sq_solid(sum_df): @@ -24,7 +30,7 @@ def sum_sq_solid(sum_df): @solid( - inputs=[InputDefinition('sum_sq_solid', dagster_pd.DataFrame, depends_on=sum_sq_solid)], + inputs=[InputDefinition('sum_sq_solid', dagster_pd.DataFrame)], output=OutputDefinition(dagster_pd.DataFrame) ) def always_fails_solid(**_kwargs): @@ -39,6 +45,14 @@ def define_pipeline(): sum_sq_solid, always_fails_solid, ], + dependencies={ + 'sum_sq_solid': { + 'sum_df': DependencyDefinition(sum_solid.name), + }, + 'always_fails_solid': { + 'sum_sq_solid': DependencyDefinition(sum_sq_solid.name), + } + } ) diff --git a/python_modules/dagster/dagster/dagster_examples/sql_hello_world/pipeline.py b/python_modules/dagster/dagster/dagster_examples/sql_hello_world/pipeline.py index 9d890b560b0eb..b1cb93242b8c9 100644 --- a/python_modules/dagster/dagster/dagster_examples/sql_hello_world/pipeline.py +++ b/python_modules/dagster/dagster/dagster_examples/sql_hello_world/pipeline.py @@ -1,9 +1,12 @@ import sqlalchemy as sa -import dagster +from dagster import ( + DependencyDefinition, + PipelineDefinition, +) + from dagster.sqlalchemy_kernel.subquery_builder_experimental import ( create_sql_solid, - create_table_input_dependency, create_table_expression_input, ) @@ -41,10 +44,16 @@ def define_pipeline(): sum_sq_table_solid = create_sql_solid( name='sum_sq_table', - inputs=[create_table_input_dependency(sum_table_solid)], + inputs=[create_table_expression_input(sum_table_solid)], sql_text='SELECT num1, num2, sum, sum * sum as sum_sq from ({sum_table})', ) - return dagster.PipelineDefinition( - name='sql_hello_world', solids=[sum_table_solid, sum_sq_table_solid] + return PipelineDefinition( + name='sql_hello_world', + solids=[sum_table_solid, sum_sq_table_solid], + dependencies={ + sum_sq_table_solid.name: { + sum_table_solid.name: DependencyDefinition(sum_table_solid.name) + } + }, ) diff --git a/python_modules/dagster/dagster/dagster_examples/sql_project_example/pipelines.py b/python_modules/dagster/dagster/dagster_examples/sql_project_example/pipelines.py index 876e6e81bc695..a16f5fb6c2b1f 100644 --- a/python_modules/dagster/dagster/dagster_examples/sql_project_example/pipelines.py +++ b/python_modules/dagster/dagster/dagster_examples/sql_project_example/pipelines.py @@ -2,7 +2,10 @@ import dagster import dagster.sqlalchemy_kernel as dagster_sa -from dagster import InputDefinition +from dagster import ( + DependencyDefinition, + InputDefinition, +) def _get_sql_script_path(name): @@ -18,17 +21,17 @@ def define_full_pipeline(): populate_num_table_solid = _get_project_solid( 'populate_num_table', - inputs=[ - InputDefinition(create_all_tables_solids.name, depends_on=create_all_tables_solids) - ], + inputs=[InputDefinition(create_all_tables_solids.name)], ) - insert_deps = [ - InputDefinition(populate_num_table_solid.name, depends_on=populate_num_table_solid) - ] + insert_into_sum_table_solid = _get_project_solid( + 'insert_into_sum_table', + inputs=[InputDefinition(populate_num_table_solid.name)], + ) - insert_into_sum_table_solid, insert_into_sum_sq_table_solid = get_insert_solids( - insert_deps=insert_deps + insert_into_sum_sq_table_solid = _get_project_solid( + 'insert_into_sum_sq_table', + inputs=[InputDefinition(insert_into_sum_table_solid.name)], ) return dagster.PipelineDefinition( @@ -40,26 +43,20 @@ def define_full_pipeline(): insert_into_sum_table_solid, insert_into_sum_sq_table_solid, ], + dependencies={ + populate_num_table_solid.name: { + create_all_tables_solids.name: DependencyDefinition(create_all_tables_solids.name) + }, + insert_into_sum_table_solid.name: { + populate_num_table_solid.name: DependencyDefinition(populate_num_table_solid.name), + }, + insert_into_sum_sq_table_solid.name: { + insert_into_sum_table_solid.name: DependencyDefinition(insert_into_sum_table_solid), + } + }, ) -def get_insert_solids(insert_deps): - insert_into_sum_table_solid = _get_project_solid( - 'insert_into_sum_table', - inputs=insert_deps, - ) - - insert_into_sum_sq_table_solid = _get_project_solid( - 'insert_into_sum_sq_table', - inputs=[ - InputDefinition( - insert_into_sum_table_solid.name, depends_on=insert_into_sum_table_solid - ) - ], - ) - return insert_into_sum_table_solid, insert_into_sum_sq_table_solid - - def define_truncate_pipeline(): truncate_solid = _get_project_solid('truncate_all_derived_tables') return dagster.PipelineDefinition( @@ -71,8 +68,13 @@ def define_truncate_pipeline(): def define_rerun_pipeline(): - insert_into_sum_table_solid, insert_into_sum_sq_table_solid = get_insert_solids( - insert_deps=None + insert_into_sum_table_solid = _get_project_solid( + 'insert_into_sum_table', + inputs=None, + ) + + insert_into_sum_sq_table_solid = _get_project_solid( + 'insert_into_sum_sq_table', inputs=[InputDefinition(insert_into_sum_table_solid.name)] ) return dagster.PipelineDefinition( @@ -83,6 +85,12 @@ def define_rerun_pipeline(): insert_into_sum_table_solid, insert_into_sum_sq_table_solid, ], + dependencies={ + insert_into_sum_sq_table_solid.name: { + insert_into_sum_table_solid.name: + DependencyDefinition(insert_into_sum_table_solid.name), + } + } ) @@ -91,9 +99,7 @@ def define_setup_pipeline(): populate_num_table_solid = _get_project_solid( 'populate_num_table', - inputs=[ - InputDefinition(create_all_tables_solids.name, depends_on=create_all_tables_solids) - ] + inputs=[InputDefinition(create_all_tables_solids.name)], ) return dagster.PipelineDefinition( @@ -103,4 +109,9 @@ def define_setup_pipeline(): create_all_tables_solids, populate_num_table_solid, ], + dependencies={ + populate_num_table_solid.name: { + create_all_tables_solids.name: DependencyDefinition(create_all_tables_solids.name), + } + } ) diff --git a/python_modules/dagster/dagster/graphviz.py b/python_modules/dagster/dagster/graphviz.py index fd033a07fca9f..59db3b79d1514 100644 --- a/python_modules/dagster/dagster/graphviz.py +++ b/python_modules/dagster/dagster/graphviz.py @@ -24,9 +24,7 @@ def build_graphviz_graph(pipeline): graphviz_graph.edge(scoped_name, solid.name) if pipeline.dependency_structure.has_dep(solid.name, input_def.name): - dep_target = pipeline.dependency_structure.get_dep_target( - solid.name, input_def.name - ) - graphviz_graph.edge(dep_target.solid_name, scoped_name) + output_handle = pipeline.dependency_structure.get_dep(solid.name, input_def.name) + graphviz_graph.edge(output_handle.solid_name, scoped_name) return graphviz_graph diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py index b4de9507b3b7c..d10e650d9df72 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py @@ -4,12 +4,13 @@ from dagster import config from dagster.core import types from dagster.core.definitions import ( + ArgumentDefinition, + DependencyDefinition, InputDefinition, MaterializationDefinition, OutputDefinition, SolidDefinition, SourceDefinition, - ArgumentDefinition, ) from dagster.core.execution import (ExecutionContext, execute_single_solid) from dagster.utils.test import script_relative_path @@ -88,20 +89,6 @@ def create_dataframe_input(name): ) -def create_dataframe_dependency(name, depends_on): - return InputDefinition( - name=name, - sources=[ - SourceDefinition( - source_type='CSV', - argument_def_dict={'path': ArgumentDefinition(types.Path)}, - source_fn=lambda context, arg_dict: pd.read_csv(arg_dict['path']), - ), - ], - depends_on=depends_on, - ) - - def create_dataframe_output(): def mat_fn(_context, arg_dict, df): df.to_csv(arg_dict['path'], index=False) @@ -176,12 +163,19 @@ def solid_two_transform(_context, args): solid_two = SolidDefinition( name='solid_two', - inputs=[create_dataframe_dependency(name='sum_df', depends_on=solid_one)], + inputs=[create_dataframe_input(name='sum_df')], transform_fn=solid_two_transform, output=create_dataframe_output(), ) - pipeline = dagster.PipelineDefinition(solids=[solid_one, solid_two]) + pipeline = dagster.PipelineDefinition( + solids=[solid_one, solid_two], + dependencies={ + 'solid_two': { + 'sum_df': DependencyDefinition('solid_one'), + }, + } + ) environment = config.Environment( sources={ diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py index 899bea584eb66..7e28bda25a920 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py @@ -3,15 +3,16 @@ import pandas as pd -import dagster from dagster import ( - check, - config, + ArgumentDefinition, + DependencyDefinition, + ExecutionContext, InputDefinition, OutputDefinition, - SolidDefinition, - ArgumentDefinition, PipelineDefinition, + SolidDefinition, + check, + config, ) from dagster.core import types from dagster.core.decorators import solid @@ -33,7 +34,7 @@ def _dataframe_solid(name, inputs, transform_fn): name=name, inputs=inputs, transform_fn=transform_fn, - output=dagster.OutputDefinition(dagster_pd.DataFrame), + output=OutputDefinition(dagster_pd.DataFrame), ) @@ -108,7 +109,7 @@ def test_pandas_csv_to_csv(): # just adding a second context arg to test that def transform(context, args): - check.inst_param(context, 'context', dagster.core.execution.ExecutionContext) + check.inst_param(context, 'context', ExecutionContext) num_csv = args['num_csv'] num_csv['sum'] = num_csv['num1'] + num_csv['num2'] return num_csv @@ -167,8 +168,8 @@ def transform(_context, args): @solid( - inputs=[dagster.InputDefinition('num_csv', dagster_pd.DataFrame)], - output=dagster.OutputDefinition(dagster_pd.DataFrame), + inputs=[InputDefinition('num_csv', dagster_pd.DataFrame)], + output=OutputDefinition(dagster_pd.DataFrame), ) def sum_table(num_csv): check.inst_param(num_csv, 'num_csv', pd.DataFrame) @@ -177,8 +178,8 @@ def sum_table(num_csv): @solid( - inputs=[dagster.InputDefinition('sum_df', dagster_pd.DataFrame, depends_on=sum_table)], - output=dagster.OutputDefinition(dagster_pd.DataFrame), + inputs=[InputDefinition('sum_df', dagster_pd.DataFrame)], + output=OutputDefinition(dagster_pd.DataFrame), ) def sum_sq_table(sum_df): sum_df['sum_squared'] = sum_df['sum'] * sum_df['sum'] @@ -187,9 +188,9 @@ def sum_sq_table(sum_df): @solid( inputs=[ - dagster.InputDefinition('sum_table_renamed', dagster_pd.DataFrame, depends_on=sum_table) + InputDefinition('sum_table_renamed', dagster_pd.DataFrame) ], - output=dagster.OutputDefinition(dagster_pd.DataFrame), + output=OutputDefinition(dagster_pd.DataFrame), ) def sum_sq_table_renamed_input(sum_table_renamed): sum_table_renamed['sum_squared'] = sum_table_renamed['sum'] * sum_table_renamed['sum'] @@ -205,7 +206,7 @@ def transform(_context, args): return _dataframe_solid( name='mult_table', inputs=[ - dagster.InputDefinition('sum_table', dagster_pd.DataFrame, depends_on=sum_table_solid) + InputDefinition('sum_table', dagster_pd.DataFrame) ], transform_fn=transform ) @@ -258,7 +259,7 @@ def test_two_step_pipeline_in_memory(): def _sum_only_pipeline(): - return PipelineDefinition(solids=[sum_table, sum_sq_table]) + return PipelineDefinition(solids=[sum_table, sum_sq_table], dependencies={},) @pytest.mark.skip('do not use _pipeline_solid_in_memory') @@ -324,9 +325,23 @@ def test_no_transform_solid(): def create_diamond_pipeline(): - return dagster.PipelineDefinition(solids=list(create_diamond_dag())) + return PipelineDefinition(solids=list(create_diamond_dag()), dependencies=create_diamond_deps(),) +def create_diamond_deps(): + return { + 'sum_table' : { + 'num_table' : DependencyDefinition('num_table'), + }, + 'mult_table' : { + 'num_table' : DependencyDefinition('num_table'), + }, + 'sum_mult_table' : { + 'sum_table' : DependencyDefinition('sum_table'), + 'mult_table' : DependencyDefinition('mult_table'), + } + } + def create_diamond_dag(): num_table_solid = _dataframe_solid( name='num_table', @@ -343,7 +358,7 @@ def sum_transform(_context, args): sum_table_solid = _dataframe_solid( name='sum_table', inputs=[ - dagster.InputDefinition('num_table', dagster_pd.DataFrame, depends_on=num_table_solid) + InputDefinition('num_table', dagster_pd.DataFrame) ], transform_fn=sum_transform, ) @@ -357,7 +372,7 @@ def mult_transform(_context, args): mult_table_solid = _dataframe_solid( name='mult_table', inputs=[ - dagster.InputDefinition('num_table', dagster_pd.DataFrame, depends_on=num_table_solid) + InputDefinition('num_table', dagster_pd.DataFrame) ], transform_fn=mult_transform, ) @@ -373,10 +388,8 @@ def sum_mult_transform(_context, args): sum_mult_table_solid = _dataframe_solid( name='sum_mult_table', inputs=[ - dagster.InputDefinition('sum_table', dagster_pd.DataFrame, depends_on=sum_table_solid), - dagster.InputDefinition( - 'mult_table', dagster_pd.DataFrame, depends_on=mult_table_solid - ), + InputDefinition('sum_table', dagster_pd.DataFrame), + InputDefinition('mult_table', dagster_pd.DataFrame), ], transform_fn=sum_mult_transform, ) @@ -389,7 +402,7 @@ def test_diamond_dag_run(): solids = create_diamond_dag() num_table_solid, sum_table_solid, mult_table_solid, sum_mult_table_solid = solids - pipeline = PipelineDefinition(solids=list(solids)) + pipeline = PipelineDefinition(solids=list(solids), dependencies=create_diamond_deps()) context = create_test_context() @@ -630,7 +643,7 @@ def transform_fn(_context, args): ], transform_fn=transform_fn ) - pipeline = dagster.PipelineDefinition(solids=[double_sum]) + pipeline = PipelineDefinition(solids=[double_sum]) output_df = execute_pipeline( pipeline, @@ -686,7 +699,14 @@ def test_pandas_multiple_outputs(): def test_rename_input(): result = execute_pipeline( - dagster.PipelineDefinition(solids=[sum_table, sum_sq_table_renamed_input]), + PipelineDefinition( + solids=[sum_table, sum_sq_table_renamed_input], + dependencies={ + sum_sq_table_renamed_input.name : { + 'sum_table_renamed' : DependencyDefinition(sum_table.name), + }, + }, + ), environment=get_num_csv_environment('sum_table'), ) diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_basic_solid.py b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_basic_solid.py index 422ae22bab272..c967b6d4edffd 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_basic_solid.py +++ b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_basic_solid.py @@ -1,6 +1,10 @@ import sqlalchemy as sa import dagster -from dagster import config +from dagster import ( + config, + DependencyDefinition, + PipelineDefinition, +) from dagster.core.execution import ( output_single_solid, execute_pipeline, @@ -9,13 +13,12 @@ from dagster.sqlalchemy_kernel.subquery_builder_experimental import ( create_sql_solid, create_table_expression_input, - create_table_input_dependency, ) from .math_test_db import in_mem_context -def pipeline_test_def(solids, context): - return dagster.PipelineDefinition( +def pipeline_test_def(solids, context, dependencies): + return PipelineDefinition( solids=solids, context_definitions={ 'default': @@ -23,7 +26,8 @@ def pipeline_test_def(solids, context): argument_def_dict={}, context_fn=lambda _pipeline, _args: context, ), - } + }, + dependencies=dependencies, ) @@ -84,11 +88,19 @@ def create_sum_sq_pipeline(context): sum_sq_solid = create_sql_solid( name='sum_sq_table', - inputs=[create_table_input_dependency(sum_solid)], + inputs=[create_table_expression_input(sum_solid.name)], sql_text='SELECT num1, num2, sum, sum * sum as sum_sq from {sum_table}', ) - return pipeline_test_def(solids=[sum_solid, sum_sq_solid], context=context) + return pipeline_test_def( + solids=[sum_solid, sum_sq_solid], + context=context, + dependencies={ + sum_sq_solid.name: { + sum_solid.name: DependencyDefinition(sum_solid.name), + }, + }, + ) def test_execute_sql_sum_sq_solid(): diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_sql_tests.py b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_sql_tests.py index 37dd577928e0d..78441b44d0110 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_sql_tests.py +++ b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_sql_tests.py @@ -1,5 +1,9 @@ import dagster -from dagster import config, InputDefinition +from dagster import ( + DependencyDefinition, + InputDefinition, + config, +) from dagster.core.execution import execute_single_solid from dagster.sqlalchemy_kernel.subquery_builder_experimental import ( create_sql_statement_solid, sql_file_solid @@ -9,7 +13,7 @@ from .math_test_db import in_mem_context -def pipeline_test_def(solids, context): +def pipeline_test_def(solids, context, dependencies=None): return dagster.PipelineDefinition( solids=solids, context_definitions={ @@ -18,7 +22,8 @@ def pipeline_test_def(solids, context): argument_def_dict={}, context_fn=lambda _pipeline, _args: context, ), - } + }, + dependencies=dependencies, ) @@ -51,10 +56,18 @@ def test_basic_pipeline(): sum_sq_sql_solid = create_sql_statement_solid( 'sum_sq_sql_solid', sum_sq_sql_text, - inputs=[InputDefinition(name=sum_sql_solid.name, depends_on=sum_sql_solid)] + inputs=[InputDefinition(name=sum_sql_solid.name)], ) - pipeline = pipeline_test_def(solids=[sum_sql_solid, sum_sq_sql_solid], context=in_mem_context()) + pipeline = pipeline_test_def( + solids=[sum_sql_solid, sum_sq_sql_solid], + context=in_mem_context(), + dependencies={ + 'sum_sq_sql_solid': { + sum_sql_solid.name: DependencyDefinition(sum_sql_solid.name), + } + }, + ) pipeline_result = dagster.execute_pipeline(pipeline, environment=config.Environment.empty()) @@ -81,11 +94,17 @@ def test_pipeline_from_files(): create_sum_sq_table_solid = sql_file_solid( script_relative_path('sql_files/create_sum_sq_table.sql'), - inputs=[InputDefinition(create_sum_table_solid.name, depends_on=create_sum_table_solid)], + inputs=[InputDefinition(create_sum_table_solid.name)], ) pipeline = pipeline_test_def( - solids=[create_sum_table_solid, create_sum_sq_table_solid], context=in_mem_context() + solids=[create_sum_table_solid, create_sum_sq_table_solid], + context=in_mem_context(), + dependencies={ + create_sum_sq_table_solid.name: { + create_sum_table_solid.name: DependencyDefinition(create_sum_table_solid.name), + } + }, ) pipeline_result = dagster.execute_pipeline(pipeline, environment=config.Environment.empty()) diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py index 21d6f044c33df..a9ce4958c16e8 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py +++ b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py @@ -1,11 +1,18 @@ -import dagster -from dagster import (config, check) -from dagster.core.definitions import ( +from dagster import ( + ArgumentDefinition, + DependencyDefinition, InputDefinition, + PipelineContextDefinition, + PipelineDefinition, + OutputDefinition, SolidDefinition, SourceDefinition, - ArgumentDefinition, + check, + config, + execute_pipeline, + types, ) + from dagster.sqlalchemy_kernel.templated import ( _create_templated_sql_transform_with_output, _render_template_string, @@ -24,16 +31,17 @@ def table_name_source(table_name): return config.Source('TABLENAME', args={'name': table_name}) -def pipeline_test_def(solids, context): - return dagster.PipelineDefinition( +def pipeline_test_def(solids, context, dependencies=None): + return PipelineDefinition( solids=solids, context_definitions={ 'default': - dagster.PipelineContextDefinition( + PipelineContextDefinition( argument_def_dict={}, context_fn=lambda _pipeline, _args: context, ), - } + }, + dependencies=dependencies, ) @@ -50,7 +58,7 @@ def test_single_templated_sql_solid_single_table_raw_api(): source_type='TABLENAME', source_fn=lambda context, arg_dict: arg_dict, argument_def_dict={ - 'name': ArgumentDefinition(dagster.core.types.String), + 'name': ArgumentDefinition(types.String), }, ) ] @@ -60,7 +68,7 @@ def test_single_templated_sql_solid_single_table_raw_api(): name='sum_table_transform', inputs=[sum_table_input], transform_fn=_create_templated_sql_transform_with_output(sql, 'sum_table'), - output=dagster.OutputDefinition(), + output=OutputDefinition(), ) pipeline = pipeline_test_def(solids=[sum_table_transform_solid], context=in_mem_context()) @@ -70,7 +78,7 @@ def test_single_templated_sql_solid_single_table_raw_api(): }}, ) - result = dagster.execute_pipeline(pipeline, environment=environment) + result = execute_pipeline(pipeline, environment=environment) assert result.success assert _load_table(result.context, sum_table_arg) == [(1, 2, 3), (3, 4, 7)] @@ -80,7 +88,7 @@ def test_single_templated_sql_solid_single_table_raw_api(): 'sum_table': table_name_source('another_table') }}, ) - result_no_source = dagster.execute_pipeline(pipeline, environment=environment_without_source) + result_no_source = execute_pipeline(pipeline, environment=environment_without_source) assert result_no_source.success assert _load_table(result_no_source.context, 'another_table') == [(1, 2, 3), (3, 4, 7)] @@ -108,7 +116,7 @@ def test_single_templated_sql_solid_single_table_with_api(): }} ) - result = dagster.execute_pipeline(pipeline, environment=environment) + result = execute_pipeline(pipeline, environment=environment) assert result.success assert _load_table(result.context, sum_table_arg) == [(1, 2, 3), (3, 4, 7)] @@ -128,7 +136,7 @@ def test_single_templated_sql_solid_double_table_raw_api(): source_type='TABLENAME', source_fn=lambda context, arg_dict: arg_dict, argument_def_dict={ - 'name': ArgumentDefinition(dagster.core.types.String), + 'name': ArgumentDefinition(types.String), }, ) ] @@ -141,7 +149,7 @@ def test_single_templated_sql_solid_double_table_raw_api(): source_type='TABLENAME', source_fn=lambda context, arg_dict: arg_dict, argument_def_dict={ - 'name': ArgumentDefinition(dagster.core.types.String), + 'name': ArgumentDefinition(types.String), }, ) ] @@ -154,7 +162,7 @@ def test_single_templated_sql_solid_double_table_raw_api(): sql, 'sum_table', ), - output=dagster.OutputDefinition(), + output=OutputDefinition(), ) pipeline = pipeline_test_def(solids=[sum_solid], context=in_mem_context(num_table_arg)) @@ -168,7 +176,7 @@ def test_single_templated_sql_solid_double_table_raw_api(): }, ) - result = dagster.execute_pipeline(pipeline, environment=environment) + result = execute_pipeline(pipeline, environment=environment) assert result.success assert _load_table(result.context, sum_table_arg) == [(1, 2, 3), (3, 4, 7)] @@ -199,7 +207,7 @@ def test_single_templated_sql_solid_double_table_with_api(): }, ) - result = dagster.execute_pipeline(pipeline, environment=environment) + result = execute_pipeline(pipeline, environment=environment) assert result.success assert _load_table(result.context, sum_table_arg) == [(1, 2, 3), (3, 4, 7)] @@ -224,12 +232,20 @@ def test_templated_sql_solid_pipeline(): sql=sum_sq_sql_template, table_arguments=['sum_sq_table'], output='sum_sq_table', - dependencies=[sum_solid], + table_deps=[sum_solid], ) context = in_mem_context() - pipeline = pipeline_test_def(solids=[sum_solid, sum_sq_solid], context=context) + pipeline = pipeline_test_def( + solids=[sum_solid, sum_sq_solid], + context=context, + dependencies={ + sum_sq_solid.name: { + sum_solid.name: DependencyDefinition(sum_solid.name), + }, + }, + ) first_sum_table = 'first_sum_table' first_sum_sq_table = 'first_sum_sq_table' @@ -243,7 +259,7 @@ def test_templated_sql_solid_pipeline(): }, } ) - first_result = dagster.execute_pipeline(pipeline, environment=environment_one) + first_result = execute_pipeline(pipeline, environment=environment_one) assert first_result.success assert len(first_result.result_list) == 2 @@ -270,7 +286,7 @@ def test_templated_sql_solid_pipeline(): execution=config.Execution.single_solid('sum_sq_table'), ) - second_result = dagster.execute_pipeline( + second_result = execute_pipeline( pipeline_two, environment=environment_two, ) @@ -298,7 +314,7 @@ def test_templated_sql_solid_with_api(): 'sum_table': table_name_source(sum_table_arg) }} ) - result = dagster.execute_pipeline(pipeline, environment=environment) + result = execute_pipeline(pipeline, environment=environment) assert result.success assert _load_table(result.context, sum_table_arg) == [(1, 2, 3), (3, 4, 7)] @@ -328,7 +344,7 @@ def test_with_from_through_specifying_all_solids(): execution=config.Execution(from_solids=all_solid_names, through_solids=all_solid_names) ) - pipeline_result = dagster.execute_pipeline(pipeline, environment=environment) + pipeline_result = execute_pipeline(pipeline, environment=environment) assert len(pipeline_result.result_list) == 3 assert _load_table(pipeline_result.context, first_sum_table) == [(1, 2, 3), (3, 4, 7)] assert _load_table(pipeline_result.context, first_mult_table) == [(1, 2, 2), (3, 4, 12)] @@ -356,7 +372,7 @@ def test_multi_input_partial_execution(): } ) - first_pipeline_result = dagster.execute_pipeline(pipeline, environment=environment) + first_pipeline_result = execute_pipeline(pipeline, environment=environment) assert first_pipeline_result.success assert len(first_pipeline_result.result_list) == 3 @@ -378,7 +394,7 @@ def test_multi_input_partial_execution(): execution=config.Execution.single_solid('sum_mult_table') ) - second_pipeline_result = dagster.execute_pipeline( + second_pipeline_result = execute_pipeline( pipeline, environment=environment_two, ) @@ -418,12 +434,19 @@ def create_multi_input_pipeline(): name='sum_mult_table', sql=sum_mult_join_template, table_arguments=['sum_mult_table'], - dependencies=[sum_solid, mult_solid], + table_deps=[sum_solid, mult_solid], output='sum_mult_table', ) pipeline = pipeline_test_def( - solids=[sum_solid, mult_solid, sum_mult_solid], context=in_mem_context() + solids=[sum_solid, mult_solid, sum_mult_solid], + context=in_mem_context(), + dependencies={ + sum_mult_solid.name: { + sum_solid.name: DependencyDefinition(sum_solid.name), + mult_solid.name: DependencyDefinition(mult_solid.name), + } + }, ) return pipeline diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/subquery_builder_experimental.py b/python_modules/dagster/dagster/sqlalchemy_kernel/subquery_builder_experimental.py index aae99a268a0c8..d3dcf490810e2 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/subquery_builder_experimental.py +++ b/python_modules/dagster/dagster/sqlalchemy_kernel/subquery_builder_experimental.py @@ -90,12 +90,6 @@ def create_table_expression_input(name): return InputDefinition(name=name, sources=[_table_name_source()]) -def create_table_input_dependency(solid): - check.inst_param(solid, 'solid', SolidDefinition) - - return InputDefinition(name=solid.name, sources=[_table_name_source()], depends_on=solid) - - def create_sql_transform(sql_text): def transform_fn(_context, args): sql_texts = {} diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/templated.py b/python_modules/dagster/dagster/sqlalchemy_kernel/templated.py index 3e6cb42c77a10..cba8eb1fb974d 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/templated.py +++ b/python_modules/dagster/dagster/sqlalchemy_kernel/templated.py @@ -12,7 +12,7 @@ from .common import execute_sql_text_on_context -def _create_table_input(name, depends_on=None): +def _create_table_input(name): return InputDefinition( name=name, sources=[ @@ -22,7 +22,6 @@ def _create_table_input(name, depends_on=None): argument_def_dict={'name': ArgumentDefinition(dagster.core.types.String)}, ) ], - depends_on=depends_on ) @@ -31,7 +30,7 @@ def create_templated_sql_transform_solid( sql, table_arguments, output, - dependencies=None, + table_deps=None, extra_inputs=None, ): ''' @@ -104,11 +103,12 @@ def create_templated_sql_transform_solid( check.str_param(sql, 'sql') check.list_param(table_arguments, 'table_arguments', of_type=str) check.str_param(output, 'output') - dependencies = check.opt_list_param(dependencies, 'dependencies', of_type=SolidDefinition) + table_deps = check.opt_list_param(table_deps, 'table_deps', of_type=SolidDefinition) extra_inputs = check.opt_list_param(extra_inputs, 'extra_inputs', of_type=InputDefinition) + dep_inputs = [_create_table_input(dep.name) for dep in table_deps] table_inputs = [_create_table_input(table) for table in table_arguments] - dep_inputs = [_create_table_input(dep.name, depends_on=dep) for dep in dependencies] + return SolidDefinition( name=name, inputs=table_inputs + dep_inputs + extra_inputs, diff --git a/python_modules/dagster/dagster/utils/compatability.py b/python_modules/dagster/dagster/utils/compatability.py index b20a89c53a6e6..269948f19836c 100644 --- a/python_modules/dagster/dagster/utils/compatability.py +++ b/python_modules/dagster/dagster/utils/compatability.py @@ -16,7 +16,6 @@ def create_custom_source_input( source_fn, *, argument_def_dict=None, - depends_on=None, expectations=None, source_type='CUSTOM' ): @@ -34,7 +33,6 @@ def create_custom_source_input( argument_def_dict=argument_def_dict, ) ], - depends_on=depends_on, expectations=check.opt_list_param( expectations, 'expectations', of_type=ExpectationDefinition ) From c0e4e3f6ef376eb021e8489342b79873b368dfe5 Mon Sep 17 00:00:00 2001 From: Nick Schrock Date: Sat, 18 Aug 2018 16:31:35 -0700 Subject: [PATCH 003/103] Dagster supports multiple outputs now Now the contract for the transform is that you yield a sequence of Result objects. Added single_output_transform helper for SolidDefinition to handle all legacy callers. --- .../dagster_ge_tests/test_pandas_ge.py | 6 +- python_modules/dagster/README.rst | 4 +- python_modules/dagster/dagster/__init__.py | 1 + .../dagster/dagster/cli/pipeline.py | 23 ++-- python_modules/dagster/dagster/config.py | 10 +- .../dagster/dagster/core/compute_nodes.py | 114 +++++++++--------- .../core/core_tests/test_execute_solid.py | 18 +-- .../core_tests/test_iterator_execution.py | 2 +- .../core/core_tests/test_multiple_outputs.py | 36 ++++++ .../core/core_tests/test_naming_collisions.py | 14 +-- .../core/core_tests/test_pipeline_errors.py | 10 +- .../core_tests/test_pipeline_execution.py | 14 +-- .../test_transform_only_pipeline.py | 8 +- .../dagster/dagster/core/decorators.py | 2 +- .../dagster/dagster/core/definitions.py | 33 +++-- python_modules/dagster/dagster/core/errors.py | 2 +- .../dagster/dagster/core/execution.py | 100 +++------------ .../pandas_hello_world/pipeline.py | 6 +- .../test_pandas_hello_world_library_slide.py | 6 +- ...est_pandas_hello_world_no_library_slide.py | 8 +- .../pandas_kernel_tests/test_pandas_solids.py | 14 +-- .../test_pandas_user_error.py | 8 +- .../test_isolated_templated_sql_tests.py | 4 +- .../subquery_builder_experimental.py | 4 +- .../dagster/sqlalchemy_kernel/templated.py | 2 +- 25 files changed, 219 insertions(+), 230 deletions(-) create mode 100644 python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py diff --git a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py index fed7beeb129f1..ef229ace8fc9e 100644 --- a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py +++ b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py @@ -27,7 +27,7 @@ def col_exists(name, col_name): 'num_df', dagster_pd.DataFrame, expectations=[col_exists('num1_exists', 'num1')] ) ], - output=OutputDefinition(dagster_pd.DataFrame) + output=OutputDefinition(dagster_type=dagster_pd.DataFrame) ) def sum_solid(num_df): return _sum_solid_impl(num_df) @@ -41,7 +41,7 @@ def sum_solid(num_df): expectations=[col_exists('failing', 'not_a_column')], ) ], - output=OutputDefinition(dagster_pd.DataFrame) + output=OutputDefinition(dagster_type=dagster_pd.DataFrame) ) def sum_solid_fails_input_expectation(num_df): return _sum_solid_impl(num_df) @@ -59,7 +59,7 @@ def sum_solid_fails_input_expectation(num_df): ], ), ], - output=OutputDefinition(dagster_pd.DataFrame) + output=OutputDefinition(dagster_type=dagster_pd.DataFrame) ) def sum_solid_expectations_config(num_df): return _sum_solid_impl(num_df) diff --git a/python_modules/dagster/README.rst b/python_modules/dagster/README.rst index c8e9799db453e..781b12beb3308 100644 --- a/python_modules/dagster/README.rst +++ b/python_modules/dagster/README.rst @@ -353,7 +353,7 @@ pandas kernel. (Note: the "kernel" terminology is not settled) num_df['sum'] = num_df['num1'] + num_df['num2'] return num_df - sum_solid = SolidDefinition( + sum_solid = SolidDefinition.single_output_transform( name='sum', description='This computes the sum of two numbers.' inputs=[dagster_pd.dataframe_csv_input(name='num_df')], @@ -448,7 +448,7 @@ and squared that value. (Fancy!) depends_on=sum_solid, ) - sum_sq_solid = SolidDefinition( + sum_sq_solid = SolidDefinition.single_output_transform( name='sum_sq', inputs=[sum_sq_input], transform_fn=sum_sq_transform, diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index 8816935342e44..2eeb487c2b684 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -14,6 +14,7 @@ OutputDefinition, PipelineContextDefinition, PipelineDefinition, + Result, SolidDefinition, SourceDefinition, ) diff --git a/python_modules/dagster/dagster/cli/pipeline.py b/python_modules/dagster/dagster/cli/pipeline.py index a03245f2e03aa..29a747a914c06 100644 --- a/python_modules/dagster/dagster/cli/pipeline.py +++ b/python_modules/dagster/dagster/cli/pipeline.py @@ -140,18 +140,19 @@ def print_solid(printer, solid): with printer.with_indent(): print_inputs(printer, solid) - printer.line('Output:') - - if solid.output.materializations: - printer.line('Materializations:') - for materialization_def in solid.output.materializations: - arg_list = format_argument_dict(materialization_def.argument_def_dict) - with printer.with_indent(): - printer.line( - '{name}({arg_list})'.format( - name=materialization_def.name, arg_list=arg_list + printer.line('Outputs:') + + for output in solid.outputs: + if output.materializations: + printer.line('Materializations:') + for materialization_def in output.materializations: + arg_list = format_argument_dict(materialization_def.argument_def_dict) + with printer.with_indent(): + printer.line( + '{name}({arg_list})'.format( + name=materialization_def.name, arg_list=arg_list + ) ) - ) def print_inputs(printer, solid): diff --git a/python_modules/dagster/dagster/config.py b/python_modules/dagster/dagster/config.py index 645b344b922ce..3c412a4593728 100644 --- a/python_modules/dagster/dagster/config.py +++ b/python_modules/dagster/dagster/config.py @@ -41,9 +41,15 @@ class Environment( namedtuple('EnvironmentData', 'context sources materializations expectations, execution') ): def __new__( - cls, sources, *, context=None, materializations=None, expectations=None, execution=None + cls, + sources=None, + *, + context=None, + materializations=None, + expectations=None, + execution=None ): - check.dict_param(sources, 'sources', key_type=str, value_type=dict) + sources = check.opt_dict_param(sources, 'sources', key_type=str, value_type=dict) for _solid_name, source_dict in sources.items(): check.dict_param(source_dict, 'source_dict', key_type=str, value_type=Source) diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index 439f596f9090a..aee052a03a9f0 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -26,6 +26,7 @@ MaterializationDefinition, OutputDefinition, PipelineDefinition, + Result, SolidDefinition, SolidInputHandle, SolidOutputHandle, @@ -46,13 +47,10 @@ DagsterType, ) -class ComputeResult(namedtuple('_ComputeResult', 'output_name value')): - def __new__(cls, output_name, value): - return super(ComputeResult, cls).__new__( - cls, - check.str_param(output_name, 'output_name'), - value, - ) +def get_single_solid_output(solid): + check.inst_param(solid, 'solid', SolidDefinition) + check.invariant(len(solid.outputs) == 1) + return solid.outputs[0] class ComputeNodeOutputHandle(namedtuple('_ComputeNodeOutputHandle', 'compute_node output_name')): @@ -129,6 +127,7 @@ def failure_result(compute_node, tag, failure_data): LOG_LEVEL = ERROR logger = define_colored_console_logger('dagster-compute-nodes', LOG_LEVEL) + @contextmanager def _user_code_error_boundary(context, msg, **kwargs): ''' @@ -167,6 +166,7 @@ class ComputeNodeTag(Enum): MATERIALIZATION_INPUT = 'mat_input' EXPECTATION_INPUT = 'expectation_input' + def _execute_core_transform(context, solid_transform_fn, values_dict): ''' Execute the user-specified transform for the solid. Wrap in an error boundary and do @@ -179,13 +179,11 @@ def _execute_core_transform(context, solid_transform_fn, values_dict): error_str = 'Error occured during core transform' with _user_code_error_boundary(context, error_str): with time_execution_scope() as timer_result: - transformed_value = solid_transform_fn(context, values_dict) + for result in solid_transform_fn(context, values_dict): + yield result context.metric('core_transform_time_ms', timer_result.millis) - return transformed_value - - class ComputeNodeInput: def __init__(self, name, dagster_type, prev_output_handle): @@ -224,18 +222,22 @@ def __init__(self, friendly_name, node_inputs, node_outputs, arg_dict, compute_f self.tag = check.inst_param(tag, 'tag', ComputeNodeTag) self.solid = check.inst_param(solid, 'solid', SolidDefinition) - def _create_compute_node_result(self, compute_result): - check.inst_param(compute_result, 'compute_result', ComputeResult) + def node_named(self, name): + check.str_param(name, 'name') + for node_output in self.node_outputs: + if node_output.name == name: + return node_output - value = compute_result.value + check.failed('not found') - check.invariant(len(self.node_outputs) == 1) - node_output = self.node_outputs[0] + def _create_compute_node_result(self, result): + check.inst_param(result, 'result', Result) + node_output = self.node_named(result.output_name) - if not node_output.dagster_type.is_python_valid_value(value): + if not node_output.dagster_type.is_python_valid_value(result.value): raise DagsterInvariantViolationError( - f'''Solid {self.solid.name} output {value} + f'''Solid {self.solid.name} output {result.value} which does not match the type for Dagster Type {node_output.dagster_type.name}''' ) @@ -244,8 +246,8 @@ def _create_compute_node_result(self, compute_result): compute_node=self, tag=self.tag, success_data=ComputeNodeSuccessData( - output_name=compute_result.output_name, - value=compute_result.value, + output_name=result.output_name, + value=result.value, ), ) @@ -253,8 +255,6 @@ def execute(self, context, inputs): check.inst_param(context, 'context', ExecutionContext) check.dict_param(inputs, 'inputs', key_type=str) - check.invariant(len(self.solid.outputs) == 1) - logger.debug(f'Entering execution for {self.friendly_name}') # do runtime type checks of inputs versus node inputs @@ -271,13 +271,13 @@ def execute(self, context, inputs): try: with _user_code_error_boundary(context, error_str): - compute_results = list(self.compute_fn(context, inputs)) + results = list(self.compute_fn(context, inputs)) if not self.node_outputs: return - for compute_result in compute_results: - yield self._create_compute_node_result(compute_result) + for result in results: + yield self._create_compute_node_result(result) except DagsterUserCodeExecutionError as dagster_user_exception: yield ComputeNodeResult.failure_result( @@ -289,8 +289,6 @@ def execute(self, context, inputs): ) return - - def output_named(self, name): check.str_param(name, 'name') @@ -301,8 +299,6 @@ def output_named(self, name): check.failed(f'output {name} not found') - - def execute_compute_nodes(context, compute_nodes): check.inst_param(context, 'context', ExecutionContext) check.list_param(compute_nodes, 'compute_nodes', of_type=ComputeNode) @@ -330,14 +326,15 @@ def execute_compute_nodes(context, compute_nodes): def _yieldify(sync_compute_fn): def _wrap(context, inputs): yield sync_compute_fn(context, inputs) + return _wrap + class SingleSyncOutputComputeNode(ComputeNode): def __init__(self, *, sync_compute_fn, **kwargs): super().__init__(compute_fn=_yieldify(sync_compute_fn), **kwargs) - def create_compute_node_from_source_config(solid, input_name, source_config): check.inst_param(solid, 'solid', SolidDefinition) check.str_param(input_name, 'input_name') @@ -364,13 +361,15 @@ def create_compute_node_from_source_config(solid, input_name, source_config): ), ], arg_dict=arg_dict, - sync_compute_fn=lambda context, _inputs: ComputeResult( + sync_compute_fn=lambda context, _inputs: Result( output_name=SOURCE_OUTPUT, value=source_def.source_fn(context, arg_dict) ), tag=ComputeNodeTag.SOURCE, solid=solid, ) + + def create_source_compute_node_dict_from_environment(pipeline, environment): check.inst_param(pipeline, 'pipeline', PipelineDefinition) check.inst_param(environment, 'environment', config.Environment) @@ -385,10 +384,12 @@ def create_source_compute_node_dict_from_environment(pipeline, environment): def get_lambda(output_name, value): - return lambda _context, _args: ComputeResult(output_name, value) + return lambda _context, _args: Result(output_name, value) + SOURCE_OUTPUT = 'source_output' + def create_source_compute_node_dict_from_input_values(pipeline, input_values): check.inst_param(pipeline, 'pipeline', PipelineDefinition) check.dict_param(input_values, 'input_values', key_type=str) @@ -489,26 +490,24 @@ def create_compute_node_graph_from_environment(pipeline, environment): ) - def create_expectation_cn(solid, expectation_def, friendly_name, tag, prev_node_output_handle): check.inst_param(solid, 'solid', SolidDefinition) check.inst_param(expectation_def, 'input_expct_def', ExpectationDefinition) check.inst_param(prev_node_output_handle, 'prev_node_output_handle', ComputeNodeOutputHandle) + output = get_single_solid_output(solid) + return SingleSyncOutputComputeNode( friendly_name=friendly_name, node_inputs=[ ComputeNodeInput( name=EXPECTATION_INPUT, - dagster_type=solid.output.dagster_type, + dagster_type=output.dagster_type, prev_output_handle=prev_node_output_handle, ) ], node_outputs=[ - ComputeNodeOutput( - name=EXPECTATION_VALUE_OUTPUT, - dagster_type=solid.output.dagster_type - ), + ComputeNodeOutput(name=EXPECTATION_VALUE_OUTPUT, dagster_type=output.dagster_type), ], arg_dict={}, sync_compute_fn=_create_expectation_lambda( @@ -554,6 +553,7 @@ def create_expectations_cn_graph(solid, inout_def, prev_node_output_handle, tag) create_cn_output_handle(join_cn, output_name), ) + def _prev_node_handle(dep_structure, solid, input_def, source_cn_dict, logical_output_mapper): check.inst_param(dep_structure, 'dep_structure', DependencyStructure) check.inst_param(solid, 'solid', SolidDefinition) @@ -580,6 +580,7 @@ def _prev_node_handle(dep_structure, solid, input_def, source_cn_dict, logical_o solid_output_handle = dep_structure.get_dep(input_handle) return logical_output_mapper.get_cn_output_handle(solid_output_handle) + def create_cn_output_handle(compute_node, cn_output_name): check.inst_param(compute_node, 'compute_node', ComputeNode) check.str_param(cn_output_name, 'cn_output_name') @@ -670,15 +671,13 @@ def create_compute_node_graph_from_source_dict( cn_output_handle = prev_cn_output_handle cn_inputs.append( - ComputeNodeInput( - input_def.name, input_def.dagster_type, cn_output_handle - ) + ComputeNodeInput(input_def.name, input_def.dagster_type, cn_output_handle) ) solid_transform_cn = create_compute_node_from_solid_transform(topo_solid, cn_inputs) for output_def in topo_solid.outputs: - if evaluate_expectations and topo_solid.output.expectations: + if evaluate_expectations and output_def.expectations: expectations_graph = create_expectations_cn_graph( topo_solid, output_def, @@ -696,7 +695,6 @@ def create_compute_node_graph_from_source_dict( create_cn_output_handle(solid_transform_cn, output_def.name), ) - compute_nodes.append(solid_transform_cn) for materialization in materializations: @@ -759,8 +757,10 @@ def _create_join_node(solid, prev_nodes, prev_output_name): ExpectationExecutionInfo = namedtuple('ExpectationExecutionInfo', 'solid expectation_def') + def _create_join_lambda(_context, inputs): - return ComputeResult(output_name=JOIN_OUTPUT, value=list(inputs.values())[0]) + return Result(output_name=JOIN_OUTPUT, value=list(inputs.values())[0]) + def _create_expectation_lambda(solid, expectation_def, output_name): check.inst_param(solid, 'solid', SolidDefinition) @@ -774,7 +774,7 @@ def _do_expectation(context, inputs): inputs[EXPECTATION_INPUT], ) if expt_result.success: - return ComputeResult(output_name=output_name, value=inputs[EXPECTATION_INPUT]) + return Result(output_name=output_name, value=inputs[EXPECTATION_INPUT]) raise DagsterExpectationFailedError(None) # for now @@ -787,7 +787,8 @@ def _construct_materialization_cn(pipeline, materialization, prev_output_handle) check.inst_param(prev_output_handle, 'prev_output_handle', ComputeNodeOutputHandle) solid = pipeline.solid_named(materialization.solid) - mat_def = solid.output.materialization_of_type(materialization.name) + output = get_single_solid_output(solid) + mat_def = output.materialization_of_type(materialization.name) error_context_str = 'source type {mat}'.format(mat=mat_def.name) @@ -802,7 +803,7 @@ def _construct_materialization_cn(pipeline, materialization, prev_output_handle) node_inputs=[ ComputeNodeInput( name=MATERIALIZATION_INPUT, - dagster_type=solid.output.dagster_type, + dagster_type=output.dagster_type, prev_output_handle=prev_output_handle, ) ], @@ -819,7 +820,7 @@ def _create_materialization_lambda(mat_def, materialization, output_name): check.inst_param(materialization, 'materialization', config.Materialization) check.str_param(output_name, 'output_name') - return lambda context, inputs: ComputeResult( + return lambda context, inputs: Result( output_name=output_name, value=mat_def.materialization_fn( context, @@ -828,14 +829,12 @@ def _create_materialization_lambda(mat_def, materialization, output_name): ), ) + def create_compute_node_from_solid_transform(solid, node_inputs): check.inst_param(solid, 'solid', SolidDefinition) check.list_param(node_inputs, 'node_inputs', of_type=ComputeNodeInput) - check.invariant(len(solid.outputs) == 1) - - output_name = solid.outputs[0].name - return SingleSyncOutputComputeNode( + return ComputeNode( friendly_name=f'{solid.name}.transform', node_inputs=node_inputs, node_outputs=[ @@ -843,13 +842,10 @@ def create_compute_node_from_solid_transform(solid, node_inputs): for output in solid.outputs ], arg_dict={}, - sync_compute_fn=lambda context, inputs: ComputeResult( - output_name=output_name, - value=_execute_core_transform( - context, - solid.transform_fn, - inputs, - ), + compute_fn=lambda context, inputs: _execute_core_transform( + context, + solid.transform_fn, + inputs, ), tag=ComputeNodeTag.TRANSFORM, solid=solid, diff --git a/python_modules/dagster/dagster/core/core_tests/test_execute_solid.py b/python_modules/dagster/dagster/core/core_tests/test_execute_solid.py index 839d8bb848544..d6f971bf1d1cf 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_execute_solid.py +++ b/python_modules/dagster/dagster/core/core_tests/test_execute_solid.py @@ -13,7 +13,7 @@ from dagster.core.execution import ( output_single_solid, - ExecutionResultBase, + ExecutionStepResult, DagsterExecutionFailureReason, ExecutionContext, execute_single_solid, @@ -61,7 +61,7 @@ def materialization_fn_inst(context, arg_dict, data): argument_def_dict={}, ) - single_solid = SolidDefinition( + single_solid = SolidDefinition.single_output_transform( name='some_node', inputs=[some_input], transform_fn=tranform_fn_inst, @@ -118,7 +118,7 @@ def materialization_fn(context, arg_dict, data): assert arg_dict == {} output_events['called'] = True - hello_world = SolidDefinition( + hello_world = SolidDefinition.single_output_transform( name='hello_world', inputs=[ create_custom_source_input( @@ -161,7 +161,7 @@ def materialization_fn(context, arg_dict, data): def test_execute_solid_with_args(): test_output = {} - single_solid = SolidDefinition( + single_solid = SolidDefinition.single_output_transform( name='some_node', inputs=[create_single_dict_input()], transform_fn=lambda context, args: args['some_input'], @@ -193,7 +193,7 @@ def test_execute_solid_with_failed_input_expectation_non_throwing(): throw_on_error=False, ) - assert isinstance(solid_execution_result, ExecutionResultBase) + assert isinstance(solid_execution_result, ExecutionStepResult) assert solid_execution_result.success is False # assert solid_execution_result.reason == DagsterExecutionFailureReason.EXPECTATION_FAILURE @@ -228,7 +228,7 @@ def failing_expectation_fn(_context, _info, _some_input): failing_expect = ExpectationDefinition(name='failing', expectation_fn=failing_expectation_fn) - return SolidDefinition( + return SolidDefinition.single_output_transform( name='some_node', inputs=[create_single_dict_input(expectations=[failing_expect])], transform_fn=lambda context, args: args['some_input'], @@ -248,7 +248,7 @@ def test_execute_solid_with_failed_output_expectation_non_throwing(): throw_on_error=False ) - assert isinstance(solid_execution_result, ExecutionResultBase) + assert isinstance(solid_execution_result, ExecutionStepResult) assert solid_execution_result.success is False # assert solid_execution_result.reason == DagsterExecutionFailureReason.EXPECTATION_FAILURE @@ -281,7 +281,7 @@ def _set_key_value(ddict, key, value): def test_execute_solid_with_no_inputs(): did_run_dict = {} - no_args_solid = SolidDefinition( + no_args_solid = SolidDefinition.single_output_transform( name='no_args_solid', inputs=[], transform_fn=lambda context, args: _set_key_value(did_run_dict, 'did_run', True), @@ -306,7 +306,7 @@ def failing_expectation_fn(_context, _info, _output): name='output_failure', expectation_fn=failing_expectation_fn ) - return SolidDefinition( + return SolidDefinition.single_output_transform( name='some_node', inputs=[create_single_dict_input()], transform_fn=lambda context, args: args['some_input'], diff --git a/python_modules/dagster/dagster/core/core_tests/test_iterator_execution.py b/python_modules/dagster/dagster/core/core_tests/test_iterator_execution.py index 7f574a14889f3..4cda4b1078d12 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_iterator_execution.py +++ b/python_modules/dagster/dagster/core/core_tests/test_iterator_execution.py @@ -36,7 +36,7 @@ def materialization_fn(_context, _arg_dict, data_iter): argument_def_dict={}, ) - iterable_solid = SolidDefinition( + iterable_solid = SolidDefinition.single_output_transform( name='some_node', inputs=[some_input], transform_fn=transform_fn, diff --git a/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py b/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py new file mode 100644 index 0000000000000..9c73e1e4cccdb --- /dev/null +++ b/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py @@ -0,0 +1,36 @@ +from dagster import ( + OutputDefinition, + PipelineDefinition, + Result, + SolidDefinition, + config, + execute_pipeline, +) + + +def test_multiple_outputs(): + def _t_fn(_context, _inputs): + yield Result('output_one', 'foo') + yield Result('output_two', 'bar') + + solid = SolidDefinition( + name='multiple_outputs', + inputs=[], + outputs=[ + OutputDefinition(name='output_one'), + OutputDefinition(name='output_two'), + ], + transform_fn=_t_fn, + ) + + pipeline = PipelineDefinition(solids=[solid]) + + result = execute_pipeline(pipeline, config.Environment(sources={})) + + assert result.result_list[0].name == 'multiple_outputs' + assert result.result_list[0].output_name == 'output_one' + assert result.result_list[0].transformed_value == 'foo' + + assert result.result_list[1].name == 'multiple_outputs' + assert result.result_list[1].output_name == 'output_two' + assert result.result_list[1].transformed_value == 'bar' diff --git a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py index fbed4f3f2be63..e66f723982429 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py +++ b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py @@ -15,7 +15,7 @@ def test_execute_solid_with_input_same_name(): - solid = SolidDefinition( + solid = SolidDefinition.single_output_transform( 'a_thing', inputs=[ InputDefinition( @@ -61,14 +61,14 @@ def test_execute_two_solids_with_same_input_name(): ], ) - solid_one = SolidDefinition( + solid_one = SolidDefinition.single_output_transform( 'solid_one', inputs=[input_def], transform_fn=lambda context, args: args['a_thing'] + args['a_thing'], output=dagster.OutputDefinition(), ) - solid_two = SolidDefinition( + solid_two = SolidDefinition.single_output_transform( 'solid_two', inputs=[input_def], transform_fn=lambda context, args: args['a_thing'] + args['a_thing'], @@ -98,7 +98,7 @@ def test_execute_two_solids_with_same_input_name(): def test_execute_dep_solid_different_input_name(): - first_solid = SolidDefinition( + first_solid = SolidDefinition.single_output_transform( 'first_solid', inputs=[ InputDefinition( @@ -116,7 +116,7 @@ def test_execute_dep_solid_different_input_name(): output=dagster.OutputDefinition(), ) - second_solid = SolidDefinition( + second_solid = SolidDefinition.single_output_transform( 'second_solid', inputs=[ InputDefinition( @@ -166,7 +166,7 @@ def s_fn(arg_dict, executed, key): 's2_t2_source': False, } - table_one = SolidDefinition( + table_one = SolidDefinition.single_output_transform( 'table_one', inputs=[ InputDefinition( @@ -185,7 +185,7 @@ def s_fn(arg_dict, executed, key): output=dagster.OutputDefinition(), ) - table_two = SolidDefinition( + table_two = SolidDefinition.single_output_transform( 'table_two', inputs=[ InputDefinition( diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py index dedbce41e5e25..0733c6c13ce4d 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py @@ -64,7 +64,7 @@ def root_transform(_context, args): passed_rows.append({name: 'transform_called'}) return passed_rows - return SolidDefinition( + return SolidDefinition.single_output_transform( name=name, inputs=[create_input_set_input_def(input_name)], transform_fn=root_transform, @@ -83,7 +83,7 @@ def create_root_transform_failure_solid(name): def failed_transform(**_kwargs): raise Exception('Transform failed') - return SolidDefinition( + return SolidDefinition.single_output_transform( name=name, inputs=[inp], transform_fn=failed_transform, @@ -102,7 +102,7 @@ def failed_input_fn(_context, _args): argument_def_dict={}, ) - return SolidDefinition( + return SolidDefinition.single_output_transform( name=name, inputs=[inp], transform_fn=lambda **_kwargs: {}, @@ -118,7 +118,7 @@ def root_transform(**kwargs): passed_rows.append({name: 'transform_called'}) return passed_rows - return SolidDefinition( + return SolidDefinition.single_output_transform( name=name, inputs=[create_input_set_input_def(input_name)], transform_fn=root_transform, @@ -194,7 +194,7 @@ def transform_fn(_context, args): check.failed('user error') return [args['A'], args['B'], {'C': 'transform_called'}] - solid_c = SolidDefinition( + solid_c = SolidDefinition.single_output_transform( name='C', inputs=[InputDefinition(name='A'), InputDefinition(name='B')], transform_fn=transform_fn, diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py index 3eaef2b3e0b72..ba1e59c091805 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py @@ -16,7 +16,7 @@ execute_pipeline_iterator, execute_pipeline_iterator_in_memory, ExecutionContext, - ExecutionResultBase, + ExecutionStepResult, ) from dagster.utils.compatability import create_custom_source_input @@ -56,7 +56,7 @@ def dep_transform(_context, args): #return copy.deepcopy(passed_rows) return passed_rows - return SolidDefinition( + return SolidDefinition.single_output_transform( name=name, inputs=inputs, transform_fn=dep_transform, @@ -78,7 +78,7 @@ def root_transform(_context, args): #return copy.deepcopy(passed_rows) return passed_rows - return SolidDefinition( + return SolidDefinition.single_output_transform( name=name, inputs=[inp], transform_fn=root_transform, @@ -285,8 +285,8 @@ def transform_called(name): def assert_equivalent_results(left, right): - check.inst_param(left, 'left', ExecutionResultBase) - check.inst_param(right, 'right', ExecutionResultBase) + check.inst_param(left, 'left', ExecutionStepResult) + check.inst_param(right, 'right', ExecutionStepResult) assert left.success == right.success assert left.name == right.name @@ -295,8 +295,8 @@ def assert_equivalent_results(left, right): def assert_all_results_equivalent(expected_results, result_results): - check.list_param(expected_results, 'expected_results', of_type=ExecutionResultBase) - check.list_param(result_results, 'result_results', of_type=ExecutionResultBase) + check.list_param(expected_results, 'expected_results', of_type=ExecutionStepResult) + check.list_param(result_results, 'result_results', of_type=ExecutionStepResult) assert len(expected_results) == len(result_results) for expected, result in zip(expected_results, result_results): assert_equivalent_results(expected, result) diff --git a/python_modules/dagster/dagster/core/core_tests/test_transform_only_pipeline.py b/python_modules/dagster/dagster/core/core_tests/test_transform_only_pipeline.py index 2935296518861..81a8bc46c6bd0 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_transform_only_pipeline.py +++ b/python_modules/dagster/dagster/core/core_tests/test_transform_only_pipeline.py @@ -19,7 +19,7 @@ def _set_key_value(ddict, key, value): def test_execute_solid_with_dep_only_inputs_no_api(): did_run_dict = {} - step_one_solid = SolidDefinition( + step_one_solid = SolidDefinition.single_output_transform( name='step_one_solid', inputs=[], transform_fn=lambda context, args: _set_key_value(did_run_dict, 'step_one', True), @@ -32,7 +32,7 @@ def test_execute_solid_with_dep_only_inputs_no_api(): argument_def_dict={}, ) - step_two_solid = SolidDefinition( + step_two_solid = SolidDefinition.single_output_transform( name='step_two_solid', inputs=[only_dep_input], transform_fn=lambda context, args: _set_key_value(did_run_dict, 'step_two', True), @@ -64,14 +64,14 @@ def test_execute_solid_with_dep_only_inputs_no_api(): def test_execute_solid_with_dep_only_inputs_with_api(): did_run_dict = {} - step_one_solid = SolidDefinition( + step_one_solid = SolidDefinition.single_output_transform( name='step_one_solid', inputs=[], transform_fn=lambda context, args: _set_key_value(did_run_dict, 'step_one', True), output=OutputDefinition(), ) - step_two_solid = SolidDefinition( + step_two_solid = SolidDefinition.single_output_transform( name='step_two_solid', transform_fn=lambda context, args: _set_key_value(did_run_dict, 'step_two', True), inputs=[InputDefinition(step_one_solid.name)], diff --git a/python_modules/dagster/dagster/core/decorators.py b/python_modules/dagster/dagster/core/decorators.py index bafad4f49f995..4dafc3213cd0c 100644 --- a/python_modules/dagster/dagster/core/decorators.py +++ b/python_modules/dagster/dagster/core/decorators.py @@ -50,7 +50,7 @@ def __call__(self, fn): _validate_transform_fn(self.name, fn, self.inputs, expect_context) transform_fn = _create_transform_wrapper(fn, self.inputs, expect_context) - return SolidDefinition( + return SolidDefinition.single_output_transform( name=self.name, inputs=self.inputs, output=self.output, diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index aaafa2d50435a..a179fea925d5f 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -337,14 +337,14 @@ class OutputDefinition: # runtime type info def __init__( self, - # name=None, + name=None, dagster_type=None, materializations=None, expectations=None, output_callback=None, description=None ): - self.name = DEFAULT_OUTPUT + self.name = check.opt_str_param(name, 'name', DEFAULT_OUTPUT) self.dagster_type = check.opt_inst_param( dagster_type, 'dagster_type', types.DagsterType, types.Any @@ -407,11 +407,20 @@ def __eq__(self, other): return self.solid.name == other.solid.name and self.output.name == other.output.name +class Result(namedtuple('_Result', 'output_name value')): + def __new__(cls, output_name, value): + return super(Result, cls).__new__( + cls, + check.str_param(output_name, 'output_name'), + value, + ) + + # One or more inputs # The core computation in the native kernel abstraction # The output class SolidDefinition: - def __init__(self, name, inputs, transform_fn, output, description=None): + def __init__(self, name, inputs, transform_fn, outputs, description=None): # if output: # check.invariant(outputs is None) # self.outputs = [output] @@ -422,18 +431,28 @@ def __init__(self, name, inputs, transform_fn, output, description=None): self.name = check_valid_name(name) self.inputs = check.list_param(inputs, 'inputs', InputDefinition) self.transform_fn = check.callable_param(transform_fn, 'transform') - self.output = check.inst_param(output, 'output', OutputDefinition) + self.outputs = check.list_param(outputs, 'outputs', OutputDefinition) self.description = check.opt_str_param(description, 'description') - self.outputs = [output] - input_handles = {} for inp in self.inputs: input_handles[inp.name] = SolidInputHandle(self, inp) self.input_handles = input_handles - self.output_handles = {output.name: SolidOutputHandle(self, output)} + output_handles = {} + for output in outputs: + output_handles[output.name] = SolidOutputHandle(self, output) + + self.output_handles = output_handles + + @staticmethod + def single_output_transform(name, inputs, transform_fn, output, description=None): + def _new_transform_fn(context, inputs): + value = transform_fn(context, inputs) + yield Result(DEFAULT_OUTPUT, value) + + return SolidDefinition(name, inputs, _new_transform_fn, [output], description) # Notes to self diff --git a/python_modules/dagster/dagster/core/errors.py b/python_modules/dagster/dagster/core/errors.py index 88037768a733a..0c9fc63a13d51 100644 --- a/python_modules/dagster/dagster/core/errors.py +++ b/python_modules/dagster/dagster/core/errors.py @@ -60,5 +60,5 @@ def __init__(self, execution_result, *args, **kwargs): self.execution_result = check.opt_inst_param( execution_result, 'execution_result', - dagster.core.execution.ExecutionResultBase, + dagster.core.execution.ExecutionStepResult, ) diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index 6f9ccce1920d9..c82abb5214eb6 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -60,7 +60,7 @@ def __init__( ): self.context = check.inst_param(context, 'context', ExecutionContext) self.result_list = check.list_param( - result_list, 'result_list', of_type=ExecutionResultBase + result_list, 'result_list', of_type=ExecutionStepResult ) @property @@ -75,26 +75,21 @@ def result_named(self, name): check.failed('Did not find result {name} in pipeline execution result'.format(name=name)) -class ExecutionResultBase: - def __init__(self, success, context, transformed_value): +class ExecutionStepResult: + def __init__(self, *, success, context, transformed_value, name, dagster_user_exception, solid, tag, output_name): self.success = check.bool_param(success, 'success') self.context = context self.transformed_value = transformed_value - - - -class BridgeSolidExecutionResult(ExecutionResultBase): - def __init__(self, success, context, transformed_value, name, dagster_user_exception, solid, tag): - super().__init__(success, context, transformed_value) self.name = name self.dagster_user_exception = dagster_user_exception self.solid = solid self.tag = tag + self.output_name = output_name def copy(self): ''' This must be used instead of copy.deepcopy() because exceptions cannot be deepcopied''' - return BridgeSolidExecutionResult( + return ExecutionStepResult( name=self.name, solid=self.solid, success=self.success, @@ -102,6 +97,7 @@ def copy(self): context=self.context, dagster_user_exception=self.dagster_user_exception, tag=self.tag, + output_name=self.output_name ) def reraise_user_error(self): @@ -109,64 +105,6 @@ def reraise_user_error(self): six.reraise(*self.dagster_user_exception.original_exc_info) -class LegacySolidExecutionResult(ExecutionResultBase): - ''' - A class to represent the result of the execution of a single solid. Pipeline - commands return iterators or lists of these results. - - (TODO: explain the various error states) - ''' - - def __init__( - self, - success, - solid, - transformed_value, - reason=None, - exception=None, - _input_expectation_results=None, - _output_expectation_results=None, - context=None, - ): - super().__init__(success, context, transformed_value) - if not success: - check.param_invariant( - isinstance(reason, DagsterExecutionFailureReason), 'reason', - 'Must provide a reason is result is a failure' - ) - # self.transformed_value = transformed_value - self.solid = check.inst_param(solid, 'solid', SolidDefinition) - self.reason = reason - self.exception = check.opt_inst_param(exception, 'exception', Exception) - - if reason == DagsterExecutionFailureReason.USER_CODE_ERROR: - check.inst(exception, DagsterUserCodeExecutionError) - self.user_exception = exception.user_exception - else: - self.user_exception = None - - - def reraise_user_error(self): - check.invariant(self.reason == DagsterExecutionFailureReason.USER_CODE_ERROR) - check.inst(self.exception, DagsterUserCodeExecutionError) - six.reraise(*self.exception.original_exc_info) - - @property - def name(self): - return self.solid.name - - def copy(self): - ''' This must be used instead of copy.deepcopy() because exceptions cannot - be deepcopied''' - return LegacySolidExecutionResult( - success=self.success, - solid=self.solid, - transformed_value=copy.deepcopy(self.transformed_value), - context=self.context, - reason=self.reason, - exception=self.exception, - ) - def copy_result_list(result_list): if result_list is None: return result_list @@ -223,26 +161,14 @@ def execute_single_solid(context, solid, environment, throw_on_error=True): def _do_throw_on_error(execution_result): - check.inst_param(execution_result, 'execution_result', ExecutionResultBase) + check.inst_param(execution_result, 'execution_result', ExecutionStepResult) if execution_result.success: return - if isinstance(execution_result, LegacySolidExecutionResult): - if execution_result.reason == DagsterExecutionFailureReason.EXPECTATION_FAILURE: - raise DagsterExpectationFailedError(execution_result) - elif execution_result.reason == DagsterExecutionFailureReason.USER_CODE_ERROR: - execution_result.reraise_user_error() - - check.invariant(execution_result.exception) - raise execution_result.exception - - if isinstance(execution_result, BridgeSolidExecutionResult): - if isinstance(execution_result.dagster_user_exception, DagsterUserCodeExecutionError): - execution_result.reraise_user_error() - - raise execution_result.dagster_user_exception - + if isinstance(execution_result.dagster_user_exception, DagsterUserCodeExecutionError): + execution_result.reraise_user_error() + raise execution_result.dagster_user_exception def output_single_solid( context, @@ -471,7 +397,7 @@ def _execute_pipeline_iterator(context, pipeline, env): for cn_result in execute_compute_nodes(context, cn_nodes): cn_node = cn_result.compute_node if not cn_result.success: - yield BridgeSolidExecutionResult( + yield ExecutionStepResult( success=False, context=context, transformed_value=None, @@ -479,11 +405,12 @@ def _execute_pipeline_iterator(context, pipeline, env): dagster_user_exception=cn_result.failure_data.dagster_user_exception, solid=cn_node.solid, tag=cn_result.tag, + output_name=None ) return if cn_node.tag == ComputeNodeTag.TRANSFORM: - yield BridgeSolidExecutionResult( + yield ExecutionStepResult( success=True, context=context, transformed_value=cn_result.success_data.value, @@ -491,6 +418,7 @@ def _execute_pipeline_iterator(context, pipeline, env): dagster_user_exception=None, solid=cn_node.solid, tag=cn_node.tag, + output_name=cn_result.success_data.output_name, ) def execute_pipeline( diff --git a/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py b/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py index 99ba3c8f533b0..b1ca78275a5cf 100644 --- a/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py +++ b/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py @@ -11,7 +11,7 @@ @solid( inputs=[InputDefinition('num', dagster_pd.DataFrame)], - output=OutputDefinition(dagster_pd.DataFrame) + output=OutputDefinition(dagster_type=dagster_pd.DataFrame) ) def sum_solid(num): sum_df = num.copy() @@ -21,7 +21,7 @@ def sum_solid(num): @solid( inputs=[InputDefinition('sum_df', dagster_pd.DataFrame)], - output=OutputDefinition(dagster_pd.DataFrame) + output=OutputDefinition(dagster_type=dagster_pd.DataFrame) ) def sum_sq_solid(sum_df): sum_sq_df = sum_df.copy() @@ -31,7 +31,7 @@ def sum_sq_solid(sum_df): @solid( inputs=[InputDefinition('sum_sq_solid', dagster_pd.DataFrame)], - output=OutputDefinition(dagster_pd.DataFrame) + output=OutputDefinition(dagster_type=dagster_pd.DataFrame) ) def always_fails_solid(**_kwargs): raise Exception('I am a programmer and I make error') diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py index 14f4e58a578a3..fb821b4e1fcf1 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py @@ -66,11 +66,11 @@ def transform_fn(_context, args): return num_csv # supports CSV and PARQUET by default - hello_world = dagster.SolidDefinition( + hello_world = dagster.SolidDefinition.single_output_transform( name='hello_world', inputs=[table_input], transform_fn=transform_fn, - output=dagster.OutputDefinition(dagster_pd.DataFrame) + output=dagster.OutputDefinition(dagster_type=dagster_pd.DataFrame) ) return hello_world @@ -78,7 +78,7 @@ def transform_fn(_context, args): def create_decorator_based_solid(): @solid( inputs=[dagster.InputDefinition('num_csv', dagster_pd.DataFrame)], - output=dagster.OutputDefinition(dagster_pd.DataFrame), + output=dagster.OutputDefinition(dagster_type=dagster_pd.DataFrame), ) def hello_world(num_csv): num_csv['sum'] = num_csv['num1'] + num_csv['num2'] diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py index d10e650d9df72..9eeeae4886546 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py @@ -43,7 +43,7 @@ def hello_world_transform_fn(_context, args): argument_def_dict={'path': ArgumentDefinition(types.Path)} ) - hello_world = SolidDefinition( + hello_world = SolidDefinition.single_output_transform( name='hello_world', inputs=[csv_input], transform_fn=hello_world_transform_fn, @@ -110,7 +110,7 @@ def transform_fn(_context, args): num_df['sum'] = num_df['num1'] + num_df['num2'] return num_df - hello_world = SolidDefinition( + hello_world = SolidDefinition.single_output_transform( name='hello_world', inputs=[create_dataframe_input(name='num_df')], transform_fn=transform_fn, @@ -149,7 +149,7 @@ def solid_one_transform(_context, args): num_df['sum'] = num_df['num1'] + num_df['num2'] return num_df - solid_one = SolidDefinition( + solid_one = SolidDefinition.single_output_transform( name='solid_one', inputs=[create_dataframe_input(name='num_df')], transform_fn=solid_one_transform, @@ -161,7 +161,7 @@ def solid_two_transform(_context, args): sum_df['sum_sq'] = sum_df['sum'] * sum_df['sum'] return sum_df - solid_two = SolidDefinition( + solid_two = SolidDefinition.single_output_transform( name='solid_two', inputs=[create_dataframe_input(name='sum_df')], transform_fn=solid_two_transform, diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py index 7e28bda25a920..baca65ac83fbb 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py @@ -30,11 +30,11 @@ def _dataframe_solid(name, inputs, transform_fn): - return SolidDefinition( + return SolidDefinition.single_output_transform( name=name, inputs=inputs, transform_fn=transform_fn, - output=OutputDefinition(dagster_pd.DataFrame), + output=OutputDefinition(dagster_type=dagster_pd.DataFrame), ) @@ -86,7 +86,7 @@ def materialization_fn_inst(context, arg_dict, df): argument_def_dict={}, ) - single_solid = SolidDefinition( + single_solid = SolidDefinition.single_output_transform( name='sum_table', inputs=[csv_input], transform_fn=transform, @@ -125,7 +125,7 @@ def materialization_fn_inst(context, arg_dict, df): argument_def_dict={'path': ArgumentDefinition(types.Path)} ) - solid_def = SolidDefinition( + solid_def = SolidDefinition.single_output_transform( name='sum_table', inputs=[csv_input], transform_fn=transform, @@ -169,7 +169,7 @@ def transform(_context, args): @solid( inputs=[InputDefinition('num_csv', dagster_pd.DataFrame)], - output=OutputDefinition(dagster_pd.DataFrame), + output=OutputDefinition(dagster_type=dagster_pd.DataFrame), ) def sum_table(num_csv): check.inst_param(num_csv, 'num_csv', pd.DataFrame) @@ -179,7 +179,7 @@ def sum_table(num_csv): @solid( inputs=[InputDefinition('sum_df', dagster_pd.DataFrame)], - output=OutputDefinition(dagster_pd.DataFrame), + output=OutputDefinition(dagster_type=dagster_pd.DataFrame), ) def sum_sq_table(sum_df): sum_df['sum_squared'] = sum_df['sum'] * sum_df['sum'] @@ -190,7 +190,7 @@ def sum_sq_table(sum_df): inputs=[ InputDefinition('sum_table_renamed', dagster_pd.DataFrame) ], - output=OutputDefinition(dagster_pd.DataFrame), + output=OutputDefinition(dagster_type=dagster_pd.DataFrame), ) def sum_sq_table_renamed_input(sum_table_renamed): sum_table_renamed['sum_squared'] = sum_table_renamed['sum'] * sum_table_renamed['sum'] diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_user_error.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_user_error.py index 75b3ed1f59a8a..1a48c1e30e757 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_user_error.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_user_error.py @@ -12,11 +12,11 @@ def _dataframe_solid(name, inputs, transform_fn): - return SolidDefinition( + return SolidDefinition.single_output_transform( name=name, inputs=inputs, transform_fn=transform_fn, - output=OutputDefinition(dagster_pd.DataFrame), + output=OutputDefinition(dagster_type=dagster_pd.DataFrame), ) @@ -24,7 +24,9 @@ def test_wrong_output_value(): csv_input = InputDefinition('num_csv', dagster_pd.DataFrame) @solid( - name="test_wrong_output", inputs=[csv_input], output=OutputDefinition(dagster_pd.DataFrame) + name="test_wrong_output", + inputs=[csv_input], + output=OutputDefinition(dagster_type=dagster_pd.DataFrame) ) def df_solid(num_csv): return 'not a dataframe' diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py index a9ce4958c16e8..8435fa69f5edb 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py +++ b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py @@ -64,7 +64,7 @@ def test_single_templated_sql_solid_single_table_raw_api(): ] ) - sum_table_transform_solid = SolidDefinition( + sum_table_transform_solid = SolidDefinition.single_output_transform( name='sum_table_transform', inputs=[sum_table_input], transform_fn=_create_templated_sql_transform_with_output(sql, 'sum_table'), @@ -155,7 +155,7 @@ def test_single_templated_sql_solid_double_table_raw_api(): ] ) - sum_solid = SolidDefinition( + sum_solid = SolidDefinition.single_output_transform( name='sum_solid', inputs=[sum_table_input, num_table_input], transform_fn=_create_templated_sql_transform_with_output( diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/subquery_builder_experimental.py b/python_modules/dagster/dagster/sqlalchemy_kernel/subquery_builder_experimental.py index d3dcf490810e2..416e23391e303 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/subquery_builder_experimental.py +++ b/python_modules/dagster/dagster/sqlalchemy_kernel/subquery_builder_experimental.py @@ -106,7 +106,7 @@ def create_sql_solid(name, inputs, sql_text): check.list_param(inputs, 'inputs', of_type=InputDefinition) check.str_param(sql_text, 'sql_text') - return SolidDefinition( + return SolidDefinition.single_output_transform( name, inputs=inputs, transform_fn=create_sql_transform(sql_text), @@ -131,7 +131,7 @@ def create_sql_statement_solid(name, sql_text, inputs=None): if inputs is None: inputs = [] - return SolidDefinition( + return SolidDefinition.single_output_transform( name=name, transform_fn=_create_sql_alchemy_transform_fn(sql_text), inputs=inputs, diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/templated.py b/python_modules/dagster/dagster/sqlalchemy_kernel/templated.py index cba8eb1fb974d..02168194833c7 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/templated.py +++ b/python_modules/dagster/dagster/sqlalchemy_kernel/templated.py @@ -109,7 +109,7 @@ def create_templated_sql_transform_solid( dep_inputs = [_create_table_input(dep.name) for dep in table_deps] table_inputs = [_create_table_input(table) for table in table_arguments] - return SolidDefinition( + return SolidDefinition.single_output_transform( name=name, inputs=table_inputs + dep_inputs + extra_inputs, transform_fn=_create_templated_sql_transform_with_output(sql, output), From fbde1684eb70419ed7bb69eca676b8ad817d7b4d Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Sat, 18 Aug 2018 17:35:59 -0700 Subject: [PATCH 004/103] Eliminate Mapper and replace with strongly typed dict subclasses --- .../dagster/dagster/core/compute_nodes.py | 75 ++++++++----------- 1 file changed, 33 insertions(+), 42 deletions(-) diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index aee052a03a9f0..7fe7bc75941be 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -369,12 +369,22 @@ def create_compute_node_from_source_config(solid, input_name, source_config): solid=solid, ) +class SourceComputeNodeMap(dict): + def __getitem__(self, key): + check.inst_param(key, 'key', SolidInputHandle) + return dict.__getitem__(self, key) + + def __setitem__(self, key, val): + check.inst_param(key, 'key', SolidInputHandle) + check.inst_param(val, 'val', ComputeNode) + return dict.__setitem__(self, key, val) + def create_source_compute_node_dict_from_environment(pipeline, environment): check.inst_param(pipeline, 'pipeline', PipelineDefinition) check.inst_param(environment, 'environment', config.Environment) - source_cn_dict = {} + source_cn_dict = SourceComputeNodeMap() for solid_name, sources_by_input in environment.sources.items(): solid = pipeline.solid_named(solid_name) for input_name, source_config in sources_by_input.items(): @@ -394,7 +404,7 @@ def create_source_compute_node_dict_from_input_values(pipeline, input_values): check.inst_param(pipeline, 'pipeline', PipelineDefinition) check.dict_param(input_values, 'input_values', key_type=str) - source_cn_dict = {} + source_cn_dict = SourceComputeNodeMap() for solid_name, sources_by_input in input_values.items(): solid = pipeline.solid_named(solid_name) for input_name, input_value in sources_by_input.items(): @@ -554,18 +564,12 @@ def create_expectations_cn_graph(solid, inout_def, prev_node_output_handle, tag) ) -def _prev_node_handle(dep_structure, solid, input_def, source_cn_dict, logical_output_mapper): +def _prev_node_handle(dep_structure, solid, input_def, source_cn_dict, compute_node_output_map): check.inst_param(dep_structure, 'dep_structure', DependencyStructure) check.inst_param(solid, 'solid', SolidDefinition) check.inst_param(input_def, 'input_def', InputDefinition) - check.dict_param( - source_cn_dict, - 'source_cn_dict', - key_type=SolidInputHandle, - value_type=ComputeNode, - ) - - check.inst_param(logical_output_mapper, 'mapper', LogicalSolidOutputMapper) + check.inst_param(source_cn_dict, 'source_cn_dict', SourceComputeNodeMap) + check.inst_param(compute_node_output_map, 'compute_node_output_map', ComputeNodeOutputMap) input_handle = solid.input_handle(input_def.name) @@ -578,7 +582,7 @@ def _prev_node_handle(dep_structure, solid, input_def, source_cn_dict, logical_o ) solid_output_handle = dep_structure.get_dep(input_handle) - return logical_output_mapper.get_cn_output_handle(solid_output_handle) + return compute_node_output_map[solid_output_handle] def create_cn_output_handle(compute_node, cn_output_name): @@ -587,18 +591,15 @@ def create_cn_output_handle(compute_node, cn_output_name): return ComputeNodeOutputHandle(compute_node, cn_output_name) -class LogicalSolidOutputMapper: - def __init__(self): - self._output_handles = {} - - def set_mapping(self, solid_output_handle, cn_output_handle): - check.inst_param(solid_output_handle, 'solid_output_handle', SolidOutputHandle) - check.inst_param(cn_output_handle, 'cn_output_handle', ComputeNodeOutputHandle) - self._output_handles[solid_output_handle] = cn_output_handle +class ComputeNodeOutputMap(dict): + def __getitem__(self, key): + check.inst_param(key, 'key', SolidOutputHandle) + return dict.__getitem__(self, key) - def get_cn_output_handle(self, solid_output_handle): - check.inst_param(solid_output_handle, 'solid_output_handle', SolidOutputHandle) - return self._output_handles[solid_output_handle] + def __setitem__(self, key, val): + check.inst_param(key, 'key', SolidOutputHandle) + check.inst_param(val, 'val', ComputeNodeOutputHandle) + return dict.__setitem__(self, key, val) def create_compute_node_graph_from_source_dict( @@ -611,13 +612,7 @@ def create_compute_node_graph_from_source_dict( ): check.inst_param(pipeline, 'pipeline', PipelineDefinition) - - check.dict_param( - source_cn_dict, - 'source_cn_dict', - key_type=SolidInputHandle, - value_type=ComputeNode, - ) + check.inst_param(source_cn_dict, 'source_cn_dict', SourceComputeNodeMap) materializations = check.opt_list_param( materializations, @@ -631,7 +626,7 @@ def create_compute_node_graph_from_source_dict( compute_nodes = list(source_cn_dict.values()) - logical_output_mapper = LogicalSolidOutputMapper() + cn_output_node_map = ComputeNodeOutputMap() subgraph = create_subgraph( pipeline, @@ -648,7 +643,7 @@ def create_compute_node_graph_from_source_dict( topo_solid, input_def, source_cn_dict, - logical_output_mapper, + cn_output_node_map, ) check.inst(prev_cn_output_handle, ComputeNodeOutputHandle) @@ -677,6 +672,7 @@ def create_compute_node_graph_from_source_dict( solid_transform_cn = create_compute_node_from_solid_transform(topo_solid, cn_inputs) for output_def in topo_solid.outputs: + output_handle = topo_solid.output_handle(output_def.name) if evaluate_expectations and output_def.expectations: expectations_graph = create_expectations_cn_graph( topo_solid, @@ -685,14 +681,11 @@ def create_compute_node_graph_from_source_dict( tag=ComputeNodeTag.OUTPUT_EXPECTATION ) compute_nodes = compute_nodes + expectations_graph.nodes - logical_output_mapper.set_mapping( - topo_solid.output_handle(output_def.name), - expectations_graph.terminal_cn_output_handle, - ) + cn_output_node_map[output_handle] = expectations_graph.terminal_cn_output_handle else: - logical_output_mapper.set_mapping( - topo_solid.output_handle(output_def.name), - create_cn_output_handle(solid_transform_cn, output_def.name), + cn_output_node_map[output_handle] = create_cn_output_handle( + solid_transform_cn, + output_def.name, ) compute_nodes.append(solid_transform_cn) @@ -702,9 +695,7 @@ def create_compute_node_graph_from_source_dict( mat_cn = _construct_materialization_cn( pipeline, materialization, - logical_output_mapper.get_cn_output_handle( - mat_solid.output_handle(materialization.output_name) - ), + cn_output_node_map[mat_solid.output_handle(materialization.output_name)], ) compute_nodes.append(mat_cn) From ba71a460a3038a06750c54c59557a08f566de523 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Sat, 18 Aug 2018 18:21:42 -0700 Subject: [PATCH 005/103] Add config_dict to SolidDefinition With this in place Sources and Materializations can go --- python_modules/dagster/dagster/config.py | 12 +- .../dagster/dagster/core/compute_nodes.py | 116 ++++++++---------- .../core/core_tests/test_compute_nodes.py | 9 +- .../core/core_tests/test_multiple_outputs.py | 7 +- .../core/core_tests/test_naming_collisions.py | 69 ++++++----- .../core/core_tests/test_solid_with_config.py | 62 ++++++++++ .../dagster/dagster/core/definitions.py | 55 +++------ .../dagster/dagster/core/execution.py | 12 ++ python_modules/dagster/dagster/core/types.py | 5 +- 9 files changed, 201 insertions(+), 146 deletions(-) create mode 100644 python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py diff --git a/python_modules/dagster/dagster/config.py b/python_modules/dagster/dagster/config.py index 3c412a4593728..c341530c317cf 100644 --- a/python_modules/dagster/dagster/config.py +++ b/python_modules/dagster/dagster/config.py @@ -23,6 +23,12 @@ def __new__(cls, name, args): ) +class Solid(namedtuple('Solid', 'config_dict')): + def __new__(cls, config_dict): + return super(Solid, + cls).__new__(cls, check.dict_param(config_dict, 'config_dict', key_type=str)) + + class Execution(namedtuple('ExecutionData', 'from_solids through_solids')): def __new__(cls, from_solids=None, through_solids=None): return super(Execution, cls).__new__( @@ -38,12 +44,15 @@ def single_solid(solid_name): class Environment( - namedtuple('EnvironmentData', 'context sources materializations expectations, execution') + namedtuple( + 'EnvironmentData', 'context solids sources materializations expectations, execution' + ) ): def __new__( cls, sources=None, *, + solids=None, context=None, materializations=None, expectations=None, @@ -68,6 +77,7 @@ def __new__( return super(Environment, cls).__new__( cls, context=context, + solids=check.opt_dict_param(solids, 'solids', key_type=str, value_type=Solid), sources=sources, materializations=check.opt_list_param( materializations, 'materializations', of_type=Materialization diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index 7fe7bc75941be..320fbb08d4a12 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -167,7 +167,7 @@ class ComputeNodeTag(Enum): EXPECTATION_INPUT = 'expectation_input' -def _execute_core_transform(context, solid_transform_fn, values_dict): +def _execute_core_transform(context, solid_transform_fn, values_dict, config_dict): ''' Execute the user-specified transform for the solid. Wrap in an error boundary and do all relevant logging and metrics tracking @@ -175,12 +175,15 @@ def _execute_core_transform(context, solid_transform_fn, values_dict): check.inst_param(context, 'context', ExecutionContext) check.callable_param(solid_transform_fn, 'solid_transform_fn') check.dict_param(values_dict, 'values_dict', key_type=str) + check.dict_param(config_dict, 'config_dict', key_type=str) error_str = 'Error occured during core transform' with _user_code_error_boundary(context, error_str): with time_execution_scope() as timer_result: - for result in solid_transform_fn(context, values_dict): - yield result + gen = solid_transform_fn(context, values_dict, config_dict) + if gen is not None: + for result in gen: + yield result context.metric('core_transform_time_ms', timer_result.millis) @@ -212,11 +215,9 @@ def __init__(self, friendly_name, node_inputs, node_outputs, arg_dict, compute_f for node_input in node_inputs: node_input_dict[node_input.name] = node_input self.node_input_dict = node_input_dict - self.node_outputs = check.list_param( node_outputs, 'node_outputs', of_type=ComputeNodeOutput ) - self.arg_dict = check.dict_param(arg_dict, 'arg_dict', key_type=str) self.compute_fn = check.callable_param(compute_fn, 'compute_fn') self.tag = check.inst_param(tag, 'tag', ComputeNodeTag) @@ -228,7 +229,7 @@ def node_named(self, name): if node_output.name == name: return node_output - check.failed('not found') + check.failed(f'{name} not found') def _create_compute_node_result(self, result): check.inst_param(result, 'result', Result) @@ -271,11 +272,14 @@ def execute(self, context, inputs): try: with _user_code_error_boundary(context, error_str): - results = list(self.compute_fn(context, inputs)) + gen = self.compute_fn(context, inputs) - if not self.node_outputs: + if gen is None: + check.invariant(not self.node_outputs) return + results = list(gen) + for result in results: yield self._create_compute_node_result(result) @@ -394,7 +398,7 @@ def create_source_compute_node_dict_from_environment(pipeline, environment): def get_lambda(output_name, value): - return lambda _context, _args: Result(output_name, value) + return lambda _context, _args: Result(output_name=output_name, value=value) SOURCE_OUTPUT = 'source_output' @@ -454,52 +458,21 @@ def topological_nodes(self): def create_compute_node_graph_from_env(pipeline, env): import dagster.core.execution if isinstance(env, dagster.core.execution.ConfigEnv): - return create_compute_node_graph_from_environment( - pipeline, - env.environment, - ) + source_cn_dict = create_source_compute_node_dict_from_environment(pipeline, env.environment) elif isinstance(env, dagster.core.execution.InMemoryEnv): - return create_compute_node_graph_from_input_values( + source_cn_dict = create_source_compute_node_dict_from_input_values( pipeline, env.input_values, - from_solids=env.from_solids, - through_solids=env.through_solids, - evaluate_expectations=env.evaluate_expectations, ) else: check.not_implemented('unsupported') - -def create_compute_node_graph_from_input_values( - pipeline, - input_values, - from_solids=None, - through_solids=None, - evaluate_expectations=True, -): - source_cn_dict = create_source_compute_node_dict_from_input_values(pipeline, input_values) - return create_compute_node_graph_from_source_dict( - pipeline, - source_cn_dict, - from_solids=from_solids, - through_solids=through_solids, - evaluate_expectations=evaluate_expectations, - ) - - -def create_compute_node_graph_from_environment(pipeline, environment): - source_cn_dict = create_source_compute_node_dict_from_environment(pipeline, environment) - return create_compute_node_graph_from_source_dict( pipeline, + env, source_cn_dict, - materializations=environment.materializations, - from_solids=environment.execution.from_solids, - through_solids=environment.execution.through_solids, - evaluate_expectations=environment.expectations.evaluate, ) - def create_expectation_cn(solid, expectation_def, friendly_name, tag, prev_node_output_handle): check.inst_param(solid, 'solid', SolidDefinition) check.inst_param(expectation_def, 'input_expct_def', ExpectationDefinition) @@ -602,36 +575,23 @@ def __setitem__(self, key, val): return dict.__setitem__(self, key, val) -def create_compute_node_graph_from_source_dict( - pipeline, - source_cn_dict, - materializations=None, - from_solids=None, - through_solids=None, - evaluate_expectations=True, -): +def create_compute_node_graph_from_source_dict(pipeline, env, source_cn_dict): + import dagster.core.execution check.inst_param(pipeline, 'pipeline', PipelineDefinition) + check.inst_param(env, 'env', dagster.core.execution.DagsterEnv) check.inst_param(source_cn_dict, 'source_cn_dict', SourceComputeNodeMap) - materializations = check.opt_list_param( - materializations, - 'materializations', - of_type=config.Materialization, - ) - dep_structure = pipeline.dependency_structure - check.bool_param(evaluate_expectations, 'evaluate_expectations') - compute_nodes = list(source_cn_dict.values()) cn_output_node_map = ComputeNodeOutputMap() subgraph = create_subgraph( pipeline, - check.opt_list_param(from_solids, 'from_solid', of_type=str), - check.opt_list_param(through_solids, 'through_solid', of_type=str), + env.from_solids, + env.through_solids, ) for topo_solid in subgraph.topological_solids: @@ -650,7 +610,7 @@ def create_compute_node_graph_from_source_dict( # jam in input expectations here - if evaluate_expectations and input_def.expectations: + if env.evaluate_expectations and input_def.expectations: expectations_graph = create_expectations_cn_graph( topo_solid, input_def, @@ -669,11 +629,21 @@ def create_compute_node_graph_from_source_dict( ComputeNodeInput(input_def.name, input_def.dagster_type, cn_output_handle) ) - solid_transform_cn = create_compute_node_from_solid_transform(topo_solid, cn_inputs) + validated_config_args = validate_args( + topo_solid.config_dict_def, + env.config_dict_for_solid(topo_solid.name), + 'TODO for solid config', + ) + + solid_transform_cn = create_compute_node_from_solid_transform( + topo_solid, + cn_inputs, + validated_config_args, + ) for output_def in topo_solid.outputs: output_handle = topo_solid.output_handle(output_def.name) - if evaluate_expectations and output_def.expectations: + if env.evaluate_expectations and output_def.expectations: expectations_graph = create_expectations_cn_graph( topo_solid, output_def, @@ -690,7 +660,7 @@ def create_compute_node_graph_from_source_dict( compute_nodes.append(solid_transform_cn) - for materialization in materializations: + for materialization in env.materializations: mat_solid = pipeline.solid_named(materialization.solid) mat_cn = _construct_materialization_cn( pipeline, @@ -789,7 +759,15 @@ def _construct_materialization_cn(pipeline, materialization, prev_output_handle) error_context_str, ) - return SingleSyncOutputComputeNode( + def _compute_fn(context, inputs): + mat_def.materialization_fn( + context, + materialization.args, + inputs[MATERIALIZATION_INPUT], + ) + + + return ComputeNode( friendly_name=f'{solid.name}.materialization.{mat_def.name}', node_inputs=[ ComputeNodeInput( @@ -800,7 +778,7 @@ def _construct_materialization_cn(pipeline, materialization, prev_output_handle) ], node_outputs=[], arg_dict=arg_dict, - sync_compute_fn=_create_materialization_lambda(mat_def, materialization, 'TODO_REMOVE'), + compute_fn=_compute_fn, tag=ComputeNodeTag.MATERIALIZATION, solid=solid, ) @@ -821,9 +799,10 @@ def _create_materialization_lambda(mat_def, materialization, output_name): ) -def create_compute_node_from_solid_transform(solid, node_inputs): +def create_compute_node_from_solid_transform(solid, node_inputs, config_args): check.inst_param(solid, 'solid', SolidDefinition) check.list_param(node_inputs, 'node_inputs', of_type=ComputeNodeInput) + check.dict_param(config_args, 'config_args', key_type=str) return ComputeNode( friendly_name=f'{solid.name}.transform', @@ -837,6 +816,7 @@ def create_compute_node_from_solid_transform(solid, node_inputs): context, solid.transform_fn, inputs, + config_args, ), tag=ComputeNodeTag.TRANSFORM, solid=solid, diff --git a/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py b/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py index a61da23de13eb..d543e7257335b 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py +++ b/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py @@ -9,7 +9,8 @@ OutputDefinition, ) -from dagster.core.compute_nodes import create_compute_node_graph_from_environment +from dagster.core.execution import ConfigEnv +from dagster.core.compute_nodes import create_compute_node_graph_from_env @solid(name='noop', inputs=[], output=OutputDefinition()) @@ -51,7 +52,8 @@ def test_compute_noop_node(): environment = config.Environment(sources={}) - compute_node_graph = create_compute_node_graph_from_environment(pipeline, environment) + env = ConfigEnv(pipeline, environment) + compute_node_graph = create_compute_node_graph_from_env(pipeline, env) assert len(compute_node_graph.nodes) == 1 @@ -72,7 +74,8 @@ def test_compute_node_with_source(): } ) - compute_node_graph = create_compute_node_graph_from_environment(pipeline, environment) + env = ConfigEnv(pipeline, environment) + compute_node_graph = create_compute_node_graph_from_env(pipeline, env) assert len(compute_node_graph.nodes) == 2 context = ExecutionContext() diff --git a/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py b/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py index 9c73e1e4cccdb..11ae0414e792d 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py +++ b/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py @@ -9,9 +9,9 @@ def test_multiple_outputs(): - def _t_fn(_context, _inputs): - yield Result('output_one', 'foo') - yield Result('output_two', 'bar') + def _t_fn(_context, _inputs, _config_dict): + yield Result(output_name='output_one', value='foo') + yield Result(output_name='output_two', value='bar') solid = SolidDefinition( name='multiple_outputs', @@ -20,6 +20,7 @@ def _t_fn(_context, _inputs): OutputDefinition(name='output_one'), OutputDefinition(name='output_two'), ], + config_def={}, transform_fn=_t_fn, ) diff --git a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py index e66f723982429..ca80fd7491d22 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py +++ b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py @@ -1,52 +1,57 @@ import dagster -from dagster import config -from dagster.core.definitions import ( - SolidDefinition, - InputDefinition, - SourceDefinition, +from dagster import ( ArgumentDefinition, DependencyDefinition, + InputDefinition, + OutputDefinition, + SolidDefinition, + SourceDefinition, + config, + types, + execute_pipeline, + ExecutionContext, + Result, + PipelineDefinition, ) -from dagster.core.execution import (ExecutionContext, execute_single_solid, execute_pipeline) - -from dagster.core import types +def define_pass_value_solid(name): + def _value_t_fn(_context, _inputs, config_dict): + yield Result(config_dict['value']) + return SolidDefinition( + name=name, + inputs=[], + outputs=[OutputDefinition(dagster_type=types.String)], + config_def={'value': ArgumentDefinition(types.String)}, + transform_fn=_value_t_fn, + ) def test_execute_solid_with_input_same_name(): - solid = SolidDefinition.single_output_transform( + a_thing_solid = SolidDefinition.single_output_transform( 'a_thing', - inputs=[ - InputDefinition( - name='a_thing', - sources=[ - SourceDefinition( - source_type='a_source_type', - source_fn=lambda context, arg_dict: arg_dict['an_arg'], - argument_def_dict={'an_arg': ArgumentDefinition(types.String)}, - ), - ], - ), - ], + inputs=[InputDefinition(name='a_thing')], transform_fn=lambda context, args: args['a_thing'] + args['a_thing'], output=dagster.OutputDefinition(), ) - result = execute_single_solid( - ExecutionContext(), - solid, - environment=config.Environment( - sources={ - 'a_thing': { - 'a_thing': config.Source(name='a_source_type', args={'an_arg': 'foo'}) - } + pipeline = PipelineDefinition( + solids=[define_pass_value_solid('pass_value'), a_thing_solid], + dependencies={ + 'a_thing' : { + 'a_thing' : DependencyDefinition('pass_value') } - ) + }, ) - assert result.success - assert result.transformed_value == 'foofoo' + result = execute_pipeline( + pipeline, + config.Environment( + solids={'pass_value': config.Solid(config_dict={'value': 'foo'})} + ), + ) + + assert result.result_named('a_thing').transformed_value == 'foofoo' def test_execute_two_solids_with_same_input_name(): diff --git a/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py b/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py new file mode 100644 index 0000000000000..25fb00fb0f4d3 --- /dev/null +++ b/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py @@ -0,0 +1,62 @@ +import pytest + +from dagster import ( + ArgumentDefinition, + PipelineDefinition, + SolidDefinition, + config, + execute_pipeline, + types, +) + +from dagster.core.errors import DagsterTypeError + + +def test_basic_solid_with_config(): + did_get = {} + + def _t_fn(_context, _inputs, config_dict): + did_get['yep'] = config_dict + + solid = SolidDefinition( + name='with_context', + inputs=[], + outputs=[], + config_def={'some_config': ArgumentDefinition(types.String)}, + transform_fn=_t_fn, + ) + + pipeline = PipelineDefinition(solids=[solid]) + + execute_pipeline( + pipeline, + config.Environment( + solids={'with_context': config.Solid(config_dict={'some_config': 'foo'})} + ), + ) + + assert 'yep' in did_get + assert 'some_config' in did_get['yep'] + + +def test_config_arg_mismatch(): + def _t_fn(_context, _inputs, _config_dict): + raise Exception('should not reach') + + solid = SolidDefinition( + name='with_context', + inputs=[], + outputs=[], + config_def={'some_config': ArgumentDefinition(types.String)}, + transform_fn=_t_fn, + ) + + pipeline = PipelineDefinition(solids=[solid]) + + with pytest.raises(DagsterTypeError): + execute_pipeline( + pipeline, + config.Environment( + solids={'with_context': config.Solid(config_dict={'some_config': 1})} + ), + ) diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index a179fea925d5f..9cebab6b389c9 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -407,32 +407,23 @@ def __eq__(self, other): return self.solid.name == other.solid.name and self.output.name == other.output.name -class Result(namedtuple('_Result', 'output_name value')): - def __new__(cls, output_name, value): +class Result(namedtuple('_Result', 'value output_name')): + def __new__(cls, value, output_name=DEFAULT_OUTPUT): return super(Result, cls).__new__( cls, - check.str_param(output_name, 'output_name'), value, + check.str_param(output_name, 'output_name'), ) -# One or more inputs -# The core computation in the native kernel abstraction -# The output class SolidDefinition: - def __init__(self, name, inputs, transform_fn, outputs, description=None): - # if output: - # check.invariant(outputs is None) - # self.outputs = [output] - # else: - # check.invariant(outputs is not None) - # self.outputs = check.list_param(outputs, 'outputs', of_type=OutputDefinition) - + def __init__(self, *, name, inputs, transform_fn, outputs, config_def, description=None): self.name = check_valid_name(name) self.inputs = check.list_param(inputs, 'inputs', InputDefinition) - self.transform_fn = check.callable_param(transform_fn, 'transform') + self.transform_fn = check.callable_param(transform_fn, 'transform_fn') self.outputs = check.list_param(outputs, 'outputs', OutputDefinition) self.description = check.opt_str_param(description, 'description') + self.config_dict_def = check_argument_def_dict(config_def) input_handles = {} for inp in self.inputs: @@ -448,30 +439,18 @@ def __init__(self, name, inputs, transform_fn, outputs, description=None): @staticmethod def single_output_transform(name, inputs, transform_fn, output, description=None): - def _new_transform_fn(context, inputs): + def _new_transform_fn(context, inputs, _config_dict): value = transform_fn(context, inputs) - yield Result(DEFAULT_OUTPUT, value) - - return SolidDefinition(name, inputs, _new_transform_fn, [output], description) - - # Notes to self - - # Input Definitions - # - DematerializationDefinitions - # - Arguments - # - Compute (args) => Value - # - Expectations - # - Dependency - - # Transform Definition - # - Function (inputs) => Value - # - Runtime Types (Inputs and Outputs) - - # Output Definition - # - MaterializationDefinitions - # - Arguments - # - Compute (value, args) => Result - # - Expectations + yield Result(output_name=DEFAULT_OUTPUT, value=value) + + return SolidDefinition( + name=name, + inputs=inputs, + transform_fn=_new_transform_fn, + outputs=[output], + config_def={}, + description=description, + ) def input_handle(self, name): check.str_param(name, 'name') diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index c82abb5214eb6..9470cf4582705 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -274,6 +274,10 @@ def from_solids(self): def through_solids(self): return self._through_solids + def config_dict_for_solid(self, name): + check.str_param(name, 'name') + return {} + @contextmanager def yield_context(self): return _wrap_in_yield(self.context) @@ -353,6 +357,14 @@ def materializations(self): def evaluate_expectations(self): return self.environment.expectations.evaluate + def config_dict_for_solid(self, name): + check.str_param(name, 'name') + if name not in self.environment.solids: + return {} + else: + return self.environment.solids[name].config_dict + + def execute_pipeline_iterator(pipeline, environment): check.inst_param(pipeline, 'pipeline', PipelineDefinition) check.inst_param(environment, 'enviroment', config.Environment) diff --git a/python_modules/dagster/dagster/core/types.py b/python_modules/dagster/dagster/core/types.py index fcd709d58875e..017c4ca38a998 100644 --- a/python_modules/dagster/dagster/core/types.py +++ b/python_modules/dagster/dagster/core/types.py @@ -15,7 +15,10 @@ def is_python_valid_value(self, _value): class DagsterScalarType(DagsterType): - pass + def __init__(self, *args, **kwargs): + super().__init__(*args, **kwargs) + self.default_sources = [] + self.default_materializations = [] class _DagsterAnyType(DagsterType): From 2b25c0ca890f33e474fad118c4a37351c32ce796 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Sun, 19 Aug 2018 04:45:38 -0700 Subject: [PATCH 006/103] Eliminate SourceDefinition Now that solids are more generic, they can totally replace SourceDefinitions. This diff does that. --- .../dagster_ge_tests/test_pandas_ge.py | 79 ++-- python_modules/dagster/dagster/__init__.py | 2 - .../dagster/dagster/cli/pipeline.py | 17 - python_modules/dagster/dagster/config.py | 46 +-- .../dagster/dagster/core/compute_nodes.py | 138 +------ .../core/core_tests/test_compute_nodes.py | 49 +-- .../core/core_tests/test_custom_context.py | 31 +- .../core/core_tests/test_decorators.py | 229 ++--------- .../core/core_tests/test_execute_solid.py | 314 --------------- .../core_tests/test_iterator_execution.py | 58 --- .../core/core_tests/test_multiple_outputs.py | 2 +- .../core/core_tests/test_naming_collisions.py | 278 +++---------- .../core/core_tests/test_pipeline_errors.py | 165 +------- .../core_tests/test_pipeline_execution.py | 155 ++++---- .../test_transform_only_pipeline.py | 9 +- .../dagster/dagster/core/decorators.py | 47 --- .../dagster/dagster/core/definitions.py | 54 +-- .../dagster/dagster/core/execution.py | 60 +-- python_modules/dagster/dagster/core/types.py | 5 - .../dagster/dagster/core/utility_solids.py | 38 ++ .../test_pandas_hello_world.py | 22 +- .../test_sql_project_pipeline.py | 5 +- .../pandas_hello_world/env.yml | 18 +- .../pandas_hello_world/pipeline.py | 17 +- .../dagster/dagster/pandas_kernel/__init__.py | 99 ++--- .../test_pandas_hello_world_library_slide.py | 82 ++-- ...est_pandas_hello_world_no_library_slide.py | 153 +++---- .../pandas_kernel_tests/test_pandas_solids.py | 376 ++++++++---------- .../test_pandas_user_error.py | 82 ++-- .../test_basic_solid.py | 91 +++-- .../test_isolated_templated_sql_tests.py | 323 +++++++-------- .../subquery_builder_experimental.py | 26 +- .../dagster/sqlalchemy_kernel/templated.py | 134 ++----- .../dagster/dagster/utils/compatability.py | 28 -- .../dagster/dagster_tests/test_config.py | 37 -- 35 files changed, 926 insertions(+), 2343 deletions(-) delete mode 100644 python_modules/dagster/dagster/core/core_tests/test_execute_solid.py delete mode 100644 python_modules/dagster/dagster/core/core_tests/test_iterator_execution.py create mode 100644 python_modules/dagster/dagster/core/utility_solids.py diff --git a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py index ef229ace8fc9e..ac928a6a87a97 100644 --- a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py +++ b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py @@ -69,56 +69,59 @@ def test_single_node_passing_expectation(): in_df = pd.DataFrame.from_dict({'num1': [1, 3], 'num2': [2, 4]}) pipeline = dagster.PipelineDefinition(solids=[sum_solid]) - result = execute_pipeline_in_memory( - dagster.ExecutionContext(), - pipeline, - input_values={sum_solid.name: { - 'num_df': in_df - }}, - ) - assert result.success - assert result.result_list[0].success - assert result.result_list[0].transformed_value.to_dict('list') == { - 'num1': [1, 3], - 'num2': [2, 4], - 'sum': [3, 7], - } + return + # result = execute_pipeline_in_memory( + # dagster.ExecutionContext(), + # pipeline, + # input_values={sum_solid.name: { + # 'num_df': in_df + # }}, + # ) + # assert result.success + # assert result.result_list[0].success + # assert result.result_list[0].transformed_value.to_dict('list') == { + # 'num1': [1, 3], + # 'num2': [2, 4], + # 'sum': [3, 7], + # } def test_single_node_passing_json_config_expectations(): in_df = pd.DataFrame.from_dict({'num1': [1, 3], 'num2': [2, 4]}) pipeline = dagster.PipelineDefinition(solids=[sum_solid_expectations_config]) - result = execute_pipeline_in_memory( - dagster.ExecutionContext(), - pipeline, - input_values={sum_solid_expectations_config.name: { - 'num_df': in_df - }}, - ) - assert result.success - assert result.result_list[0].success - assert result.result_list[0].transformed_value.to_dict('list') == { - 'num1': [1, 3], - 'num2': [2, 4], - 'sum': [3, 7], - } + return + # result = execute_pipeline_in_memory( + # dagster.ExecutionContext(), + # pipeline, + # input_values={sum_solid_expectations_config.name: { + # 'num_df': in_df + # }}, + # ) + # assert result.success + # assert result.result_list[0].success + # assert result.result_list[0].transformed_value.to_dict('list') == { + # 'num1': [1, 3], + # 'num2': [2, 4], + # 'sum': [3, 7], + # } def test_single_node_failing_expectation(): in_df = pd.DataFrame.from_dict({'num1': [1, 3], 'num2': [2, 4]}) pipeline = dagster.PipelineDefinition(solids=[sum_solid_fails_input_expectation]) - result = execute_pipeline_in_memory( - dagster.ExecutionContext(), - pipeline, - input_values={sum_solid_fails_input_expectation.name: { - 'num_df': in_df - }}, - throw_on_error=False - ) - assert not result.success - return + # result = execute_pipeline_in_memory( + # dagster.ExecutionContext(), + # pipeline, + # input_values={sum_solid_fails_input_expectation.name: { + # 'num_df': in_df + # }}, + # throw_on_error=False + # ) + # assert not result.success + + # return # TODO redo expectation result API # assert len(result.result_list) == 1 # first_solid_result = result.result_list[0] diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index 2eeb487c2b684..730acd55de51b 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -16,13 +16,11 @@ PipelineDefinition, Result, SolidDefinition, - SourceDefinition, ) from dagster.core.decorators import ( materialization, solid, - source, with_context, ) diff --git a/python_modules/dagster/dagster/cli/pipeline.py b/python_modules/dagster/dagster/cli/pipeline.py index 29a747a914c06..2e91f9e7d34f7 100644 --- a/python_modules/dagster/dagster/cli/pipeline.py +++ b/python_modules/dagster/dagster/cli/pipeline.py @@ -161,23 +161,6 @@ def print_inputs(printer, solid): with printer.with_indent(): printer.line('Input: {name}'.format(name=input_def.name)) - if input_def.sources: - print_sources(printer, input_def.sources) - - -def print_sources(printer, sources): - with printer.with_indent(): - printer.line('Sources:') - with printer.with_indent(): - for source_def in sources: - arg_list = format_argument_dict(source_def.argument_def_dict) - printer.line( - '{input_name}({arg_list})'.format( - input_name=source_def.source_type, - arg_list=arg_list, - ) - ) - def format_argument_dict(arg_def_dict): return ', '.join( diff --git a/python_modules/dagster/dagster/config.py b/python_modules/dagster/dagster/config.py index c341530c317cf..eda5528e2bd08 100644 --- a/python_modules/dagster/dagster/config.py +++ b/python_modules/dagster/dagster/config.py @@ -44,24 +44,11 @@ def single_solid(solid_name): class Environment( - namedtuple( - 'EnvironmentData', 'context solids sources materializations expectations, execution' - ) + namedtuple('EnvironmentData', 'context solids materializations expectations, execution') ): def __new__( - cls, - sources=None, - *, - solids=None, - context=None, - materializations=None, - expectations=None, - execution=None + cls, *, solids=None, context=None, materializations=None, expectations=None, execution=None ): - sources = check.opt_dict_param(sources, 'sources', key_type=str, value_type=dict) - for _solid_name, source_dict in sources.items(): - check.dict_param(source_dict, 'source_dict', key_type=str, value_type=Source) - check.opt_inst_param(context, 'context', Context) check.opt_inst_param(execution, 'execution', Execution) @@ -78,7 +65,6 @@ def __new__( cls, context=context, solids=check.opt_dict_param(solids, 'solids', key_type=str, value_type=Solid), - sources=sources, materializations=check.opt_list_param( materializations, 'materializations', of_type=Materialization ), @@ -88,7 +74,7 @@ def __new__( @staticmethod def empty(): - return Environment(sources={}) + return Environment() class Source(namedtuple('SourceData', 'name args')): @@ -115,17 +101,6 @@ def _construct_context(yml_config_object): return None -def _construct_sources(yml_config_object): - sources = defaultdict(dict) - sources_obj = check.dict_elem(yml_config_object, 'sources') - for solid_name, args_yml in sources_obj.items(): - for input_name, source_yml in args_yml.items(): - sources[solid_name][input_name] = Source( - name=source_yml['name'], args=source_yml['args'] - ) - return sources - - def _construct_materializations(yml_config_object): materializations = [ Materialization(solid=m['solid'], name=m['name'], args=m['args']) @@ -156,11 +131,24 @@ def _construct_execution(yml_config_object): ) +def _construct_solids(yml_config_object): + solid_dict = check.opt_dict_elem(yml_config_object, 'solids') + if solid_dict is None: + return None + + solid_configs = {} + for solid_name, solid_yml_object in solid_dict.items(): + config_dict = check.dict_elem(solid_yml_object, 'config') + solid_configs[solid_name] = Solid(config_dict) + + return solid_configs + + def construct_environment(yml_config_object): check.dict_param(yml_config_object, 'yml_config_object') return Environment( - sources=_construct_sources(yml_config_object), + solids=_construct_solids(yml_config_object), materializations=_construct_materializations(yml_config_object), context=_construct_context(yml_config_object), execution=_construct_execution(yml_config_object), diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index 320fbb08d4a12..85e124063f638 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -156,9 +156,7 @@ class ComputeNodeTag(Enum): INPUT_EXPECTATION = 'INPUT_EXPECTATION' OUTPUT_EXPECTATION = 'OUTPUT_EXPECTATION' JOIN = 'JOIN' - SOURCE = 'SOURCE' MATERIALIZATION = 'MATERIALIZATION' - INPUTSTUB = 'INPUTSTUB' EXPECTATION_VALUE_OUTPUT = 'expectation_value' @@ -339,95 +337,6 @@ def __init__(self, *, sync_compute_fn, **kwargs): super().__init__(compute_fn=_yieldify(sync_compute_fn), **kwargs) -def create_compute_node_from_source_config(solid, input_name, source_config): - check.inst_param(solid, 'solid', SolidDefinition) - check.str_param(input_name, 'input_name') - check.inst_param(source_config, 'source_config', config.Source) - - input_def = solid.input_def_named(input_name) - source_def = input_def.source_of_type(source_config.name) - - error_context_str = 'source type {source}'.format(source=source_def.source_type) - - arg_dict = validate_args( - source_def.argument_def_dict, - source_config.args, - error_context_str, - ) - - return SingleSyncOutputComputeNode( - friendly_name=f'{solid.name}.{input_name}.source.{source_config.name}', - node_inputs=[], - node_outputs=[ - ComputeNodeOutput( - name=SOURCE_OUTPUT, - dagster_type=input_def.dagster_type, - ), - ], - arg_dict=arg_dict, - sync_compute_fn=lambda context, _inputs: Result( - output_name=SOURCE_OUTPUT, - value=source_def.source_fn(context, arg_dict) - ), - tag=ComputeNodeTag.SOURCE, - solid=solid, - ) - -class SourceComputeNodeMap(dict): - def __getitem__(self, key): - check.inst_param(key, 'key', SolidInputHandle) - return dict.__getitem__(self, key) - - def __setitem__(self, key, val): - check.inst_param(key, 'key', SolidInputHandle) - check.inst_param(val, 'val', ComputeNode) - return dict.__setitem__(self, key, val) - - -def create_source_compute_node_dict_from_environment(pipeline, environment): - check.inst_param(pipeline, 'pipeline', PipelineDefinition) - check.inst_param(environment, 'environment', config.Environment) - - source_cn_dict = SourceComputeNodeMap() - for solid_name, sources_by_input in environment.sources.items(): - solid = pipeline.solid_named(solid_name) - for input_name, source_config in sources_by_input.items(): - compute_node = create_compute_node_from_source_config(solid, input_name, source_config) - source_cn_dict[solid.input_handle(input_name)] = compute_node - return source_cn_dict - - -def get_lambda(output_name, value): - return lambda _context, _args: Result(output_name=output_name, value=value) - - -SOURCE_OUTPUT = 'source_output' - - -def create_source_compute_node_dict_from_input_values(pipeline, input_values): - check.inst_param(pipeline, 'pipeline', PipelineDefinition) - check.dict_param(input_values, 'input_values', key_type=str) - - source_cn_dict = SourceComputeNodeMap() - for solid_name, sources_by_input in input_values.items(): - solid = pipeline.solid_named(solid_name) - for input_name, input_value in sources_by_input.items(): - input_handle = solid.input_handle(input_name) - source_cn_dict[input_handle] = SingleSyncOutputComputeNode( - friendly_name=f'{solid_name}.{input_name}.stub', - node_inputs=[], - # This is just a stub of a pre-existing value, so we are not - # going to make any type guarantees - node_outputs=[ComputeNodeOutput(SOURCE_OUTPUT, Any)], - arg_dict={}, - sync_compute_fn=get_lambda(SOURCE_OUTPUT, input_value), - tag=ComputeNodeTag.INPUTSTUB, - solid=pipeline.solid_named(solid_name), - ) - - return source_cn_dict - - def print_graph(graph, printer=print): check.inst_param(graph, 'graph', ComputeNodeGraph) printer = IndentingPrinter(printer=printer) @@ -455,24 +364,6 @@ def topological_nodes(self): yield self.cn_dict[cn_guid] -def create_compute_node_graph_from_env(pipeline, env): - import dagster.core.execution - if isinstance(env, dagster.core.execution.ConfigEnv): - source_cn_dict = create_source_compute_node_dict_from_environment(pipeline, env.environment) - elif isinstance(env, dagster.core.execution.InMemoryEnv): - source_cn_dict = create_source_compute_node_dict_from_input_values( - pipeline, - env.input_values, - ) - else: - check.not_implemented('unsupported') - - return create_compute_node_graph_from_source_dict( - pipeline, - env, - source_cn_dict, - ) - def create_expectation_cn(solid, expectation_def, friendly_name, tag, prev_node_output_handle): check.inst_param(solid, 'solid', SolidDefinition) check.inst_param(expectation_def, 'input_expct_def', ExpectationDefinition) @@ -537,25 +428,21 @@ def create_expectations_cn_graph(solid, inout_def, prev_node_output_handle, tag) ) -def _prev_node_handle(dep_structure, solid, input_def, source_cn_dict, compute_node_output_map): +def _prev_node_handle(dep_structure, solid, input_def, compute_node_output_map): check.inst_param(dep_structure, 'dep_structure', DependencyStructure) check.inst_param(solid, 'solid', SolidDefinition) check.inst_param(input_def, 'input_def', InputDefinition) - check.inst_param(source_cn_dict, 'source_cn_dict', SourceComputeNodeMap) check.inst_param(compute_node_output_map, 'compute_node_output_map', ComputeNodeOutputMap) input_handle = solid.input_handle(input_def.name) - if input_handle in source_cn_dict: - return create_cn_output_handle(source_cn_dict[input_handle], SOURCE_OUTPUT) - else: - check.invariant( - dep_structure.has_dep(input_handle), - f'{input_handle} not found in dependency structure', - ) + check.invariant( + dep_structure.has_dep(input_handle), + f'{input_handle} not found in dependency structure', + ) - solid_output_handle = dep_structure.get_dep(input_handle) - return compute_node_output_map[solid_output_handle] + solid_output_handle = dep_structure.get_dep(input_handle) + return compute_node_output_map[solid_output_handle] def create_cn_output_handle(compute_node, cn_output_name): @@ -575,16 +462,14 @@ def __setitem__(self, key, val): return dict.__setitem__(self, key, val) -def create_compute_node_graph_from_source_dict(pipeline, env, source_cn_dict): +def create_compute_node_graph_from_env(pipeline, env): import dagster.core.execution - check.inst_param(pipeline, 'pipeline', PipelineDefinition) check.inst_param(env, 'env', dagster.core.execution.DagsterEnv) - check.inst_param(source_cn_dict, 'source_cn_dict', SourceComputeNodeMap) dep_structure = pipeline.dependency_structure - compute_nodes = list(source_cn_dict.values()) + compute_nodes = [] cn_output_node_map = ComputeNodeOutputMap() @@ -602,7 +487,6 @@ def create_compute_node_graph_from_source_dict(pipeline, env, source_cn_dict): dep_structure, topo_solid, input_def, - source_cn_dict, cn_output_node_map, ) @@ -632,7 +516,7 @@ def create_compute_node_graph_from_source_dict(pipeline, env, source_cn_dict): validated_config_args = validate_args( topo_solid.config_dict_def, env.config_dict_for_solid(topo_solid.name), - 'TODO for solid config', + 'config for solid {solid_name}'.format(solid_name=topo_solid.name), ) solid_transform_cn = create_compute_node_from_solid_transform( @@ -751,7 +635,7 @@ def _construct_materialization_cn(pipeline, materialization, prev_output_handle) output = get_single_solid_output(solid) mat_def = output.materialization_of_type(materialization.name) - error_context_str = 'source type {mat}'.format(mat=mat_def.name) + error_context_str = 'materialization type {mat}'.format(mat=mat_def.name) arg_dict = validate_args( mat_def.argument_def_dict, diff --git a/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py b/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py index d543e7257335b..cc6a104ece958 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py +++ b/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py @@ -1,23 +1,16 @@ from dagster import ( ExecutionContext, - InputDefinition, PipelineContextDefinition, PipelineDefinition, config, solid, - source, OutputDefinition, ) -from dagster.core.execution import ConfigEnv +from dagster.core.execution import DagsterEnv from dagster.core.compute_nodes import create_compute_node_graph_from_env -@solid(name='noop', inputs=[], output=OutputDefinition()) -def noop_solid(): - return 'foo' - - def silencing_default_context(): return { 'default': @@ -28,17 +21,9 @@ def silencing_default_context(): } -@source() -def load_value_source(): - return 'value' - - -@solid( - inputs=[InputDefinition(name='some_input', sources=[load_value_source])], - output=OutputDefinition() -) -def solid_with_source(some_input): - return some_input +@solid(name='noop', inputs=[], output=OutputDefinition()) +def noop_solid(): + return 'foo' def silencing_pipeline(solids): @@ -50,9 +35,9 @@ def test_compute_noop_node(): noop_solid, ]) - environment = config.Environment(sources={}) + environment = config.Environment() - env = ConfigEnv(pipeline, environment) + env = DagsterEnv(pipeline, environment) compute_node_graph = create_compute_node_graph_from_env(pipeline, env) assert len(compute_node_graph.nodes) == 1 @@ -62,25 +47,3 @@ def test_compute_noop_node(): outputs = list(compute_node_graph.nodes[0].execute(context, {})) assert outputs[0].success_data.value == 'foo' - - -def test_compute_node_with_source(): - pipeline = silencing_pipeline(solids=[solid_with_source]) - environment = config.Environment( - sources={ - 'solid_with_source': { - 'some_input': config.Source(name='load_value_source', args={}) - } - } - ) - - env = ConfigEnv(pipeline, environment) - compute_node_graph = create_compute_node_graph_from_env(pipeline, env) - assert len(compute_node_graph.nodes) == 2 - - context = ExecutionContext() - - node_list = list(compute_node_graph.topological_nodes()) - - assert list(node_list[0].execute(context, {}))[0].success_data.value == 'value' - assert list(node_list[1].execute(context, {'some_input': 'bar'}))[0].success_data.value == 'bar' diff --git a/python_modules/dagster/dagster/core/core_tests/test_custom_context.py b/python_modules/dagster/dagster/core/core_tests/test_custom_context.py index 45da88cc17380..1fe54908f3a3d 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_custom_context.py +++ b/python_modules/dagster/dagster/core/core_tests/test_custom_context.py @@ -31,7 +31,7 @@ def default_context_transform(context): pipeline = PipelineDefinition(solids=[default_context_transform]) execute_pipeline( - pipeline, environment=config.Environment(sources={}, context=config.Context('default', {})) + pipeline, environment=config.Environment(context=config.Context('default', {})) ) @@ -48,17 +48,13 @@ def default_context_transform(context): pipeline = PipelineDefinition(solids=[default_context_transform]) execute_pipeline( pipeline, - environment=config.Environment( - sources={}, context=config.Context('default', {'log_level': 'INFO'}) - ) + environment=config.Environment(context=config.Context('default', {'log_level': 'INFO'})) ) with pytest.raises(DagsterTypeError, message='Argument mismatch in context default'): execute_pipeline( pipeline, - environment=config.Environment( - sources={}, context=config.Context('default', {'log_level': 2}) - ) + environment=config.Environment(context=config.Context('default', {'log_level': 2})) ) @@ -93,8 +89,7 @@ def args_test(context): ) execute_pipeline( - pipeline, - environment=config.Environment(sources={}, context=config.Context('custom_one', {})) + pipeline, environment=config.Environment(context=config.Context('custom_one', {})) ) @@ -124,13 +119,13 @@ def custom_context_transform(context): ) environment_one = config.Environment( - sources={}, context=config.Context('custom_one', {'arg_one': 'value_two'}) + context=config.Context('custom_one', {'arg_one': 'value_two'}) ) execute_pipeline(pipeline, environment=environment_one) environment_two = config.Environment( - sources={}, context=config.Context('custom_two', {'arg_one': 'value_two'}) + context=config.Context('custom_two', {'arg_one': 'value_two'}) ) execute_pipeline(pipeline, environment=environment_two) @@ -168,7 +163,7 @@ def _yield_context(_pipeline, args): ) environment_one = config.Environment( - sources={}, context=config.Context('custom_one', {'arg_one': 'value_two'}) + context=config.Context('custom_one', {'arg_one': 'value_two'}) ) execute_pipeline(pipeline, environment=environment_one) @@ -188,9 +183,7 @@ def never_transform(): default_context_pipeline = PipelineDefinition(solids=[never_transform]) - environment_context_not_found = config.Environment( - sources={}, context=config.Context('not_found', {}) - ) + environment_context_not_found = config.Environment(context=config.Context('not_found', {})) with pytest.raises(DagsterInvariantViolationError, message='Context not_found not found'): execute_pipeline( @@ -200,7 +193,7 @@ def never_transform(): ) environment_arg_name_mismatch = config.Environment( - sources={}, context=config.Context('default', {'unexpected': 'value'}) + context=config.Context('default', {'unexpected': 'value'}) ) with pytest.raises(DagsterTypeError, message='Argument mismatch in context default'): @@ -221,9 +214,7 @@ def never_transform(): } ) - environment_no_args_error = config.Environment( - sources={}, context=config.Context('default', {}) - ) + environment_no_args_error = config.Environment(context=config.Context('default', {})) with pytest.raises(DagsterTypeError, message='Argument mismatch in context default'): execute_pipeline( @@ -233,7 +224,7 @@ def never_transform(): ) environment_type_mismatch_error = config.Environment( - sources={}, context=config.Context('default', {'string_arg': 1}) + context=config.Context('default', {'string_arg': 1}) ) with pytest.raises(DagsterTypeError, message='Argument mismatch in context default'): diff --git a/python_modules/dagster/dagster/core/core_tests/test_decorators.py b/python_modules/dagster/dagster/core/core_tests/test_decorators.py index c115fe561109d..6cf31fce7f78a 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_decorators.py +++ b/python_modules/dagster/dagster/core/core_tests/test_decorators.py @@ -1,19 +1,27 @@ import pytest import dagster -from dagster import config -from dagster.core import types -from dagster.core.definitions import ( - OutputDefinition, - InputDefinition, +from dagster import ( ArgumentDefinition, + DependencyDefinition, + InputDefinition, + OutputDefinition, + PipelineDefinition, + config, + execute_pipeline, + types, ) from dagster.core.errors import DagsterInvalidDefinitionError -from dagster.core.decorators import solid, source, materialization, with_context +from dagster.core.decorators import ( + materialization, + solid, + with_context, +) from dagster.core.execution import ( output_single_solid, execute_single_solid, ExecutionContext, ) +from dagster.core.utility_solids import define_pass_mem_value # This file tests a lot of parameter name stuff # So these warnings are spurious @@ -26,7 +34,7 @@ def create_test_context(): def create_empty_test_env(): - return config.Environment(sources={}) + return config.Environment() def test_solid(): @@ -79,68 +87,23 @@ def hello_world(_context): def test_solid_with_input(): - @source(name="TEST", argument_def_dict={'foo': ArgumentDefinition(types.String)}) - def test_source(foo): - return {'foo': foo} - - @solid(inputs=[InputDefinition(name="foo_to_foo", sources=[test_source])]) + @solid(inputs=[InputDefinition(name="foo_to_foo")]) def hello_world(foo_to_foo): return foo_to_foo - result = execute_single_solid( - create_test_context(), - hello_world, - environment=config.Environment( - sources={ - 'hello_world': { - 'foo_to_foo': config.Source(name='TEST', args={'foo': 'bar'}) - } - } - ), + pipeline = PipelineDefinition( + solids=[define_pass_mem_value('test_value', {'foo': 'bar'}), hello_world], + dependencies={'hello_world': { + 'foo_to_foo': DependencyDefinition('test_value'), + }} ) - assert result.success - - assert result.transformed_value['foo'] == 'bar' - - -def test_sources(): - @source(name="WITH_CONTEXT", argument_def_dict={'foo': ArgumentDefinition(types.String)}) - @with_context - def context_source(_context, foo): - return {'foo': foo} - - @source(name="NO_CONTEXT", argument_def_dict={'foo': ArgumentDefinition(types.String)}) - def no_context_source(foo): - return {'foo': foo} - - @solid(inputs=[InputDefinition(name="i", sources=[context_source, no_context_source])]) - def hello_world(i): - return i - - result = execute_single_solid( - create_test_context(), - hello_world, - environment=config.Environment( - sources={'hello_world': { - 'i': config.Source(name='NO_CONTEXT', args={'foo': 'bar'}) - }} - ), + pipeline_result = execute_pipeline( + pipeline, + environment=config.Environment(), ) - assert result.success - - assert result.transformed_value['foo'] == 'bar' - - result = execute_single_solid( - create_test_context(), - hello_world, - environment=config.Environment( - sources={'hello_world': { - 'i': config.Source(name='NO_CONTEXT', args={'foo': 'bar'}) - }} - ), - ) + result = pipeline_result.result_named('hello_world') assert result.success @@ -170,7 +133,7 @@ def hello(): output_single_solid( create_test_context(), hello, - environment=config.Environment(sources={}), + environment=config.Environment(), name='CONTEXT', arg_dict={'foo': 'bar'} ) @@ -182,7 +145,7 @@ def hello(): output_single_solid( create_test_context(), hello, - environment=config.Environment(sources={}), + environment=config.Environment(), name='NO_CONTEXT', arg_dict={'foo': 'bar'} ) @@ -191,34 +154,24 @@ def hello(): def test_solid_definition_errors(): - @source(name="test_source") - def test_source(): - pass - with pytest.raises(DagsterInvalidDefinitionError): - @solid( - inputs=[InputDefinition(name="foo", sources=[test_source])], output=OutputDefinition() - ) + @solid(inputs=[InputDefinition(name="foo")], output=OutputDefinition()) @with_context def vargs(_context, foo, *args): pass with pytest.raises(DagsterInvalidDefinitionError): - @solid( - inputs=[InputDefinition(name="foo", sources=[test_source])], output=OutputDefinition() - ) + @solid(inputs=[InputDefinition(name="foo")], output=OutputDefinition()) def wrong_name(bar): pass with pytest.raises(DagsterInvalidDefinitionError): @solid( - inputs=[ - InputDefinition(name="foo", sources=[test_source]), - InputDefinition(name="bar", sources=[test_source]) - ], + inputs=[InputDefinition(name="foo"), + InputDefinition(name="bar")], output=OutputDefinition() ) def wrong_name_2(foo): @@ -226,54 +179,42 @@ def wrong_name_2(foo): with pytest.raises(DagsterInvalidDefinitionError): - @solid( - inputs=[InputDefinition(name="foo", sources=[test_source])], output=OutputDefinition() - ) + @solid(inputs=[InputDefinition(name="foo")], output=OutputDefinition()) @with_context def no_context(foo): pass with pytest.raises(DagsterInvalidDefinitionError): - @solid( - inputs=[InputDefinition(name="foo", sources=[test_source])], output=OutputDefinition() - ) + @solid(inputs=[InputDefinition(name="foo")], output=OutputDefinition()) def yes_context(_context, foo): pass with pytest.raises(DagsterInvalidDefinitionError): - @solid( - inputs=[InputDefinition(name="foo", sources=[test_source])], output=OutputDefinition() - ) + @solid(inputs=[InputDefinition(name="foo")], output=OutputDefinition()) def extras(foo, bar): pass @solid( - inputs=[ - InputDefinition(name="foo", sources=[test_source]), - InputDefinition(name="bar", sources=[test_source]) - ], + inputs=[InputDefinition(name="foo"), + InputDefinition(name="bar")], output=OutputDefinition() ) def valid_kwargs(**kwargs): pass @solid( - inputs=[ - InputDefinition(name="foo", sources=[test_source]), - InputDefinition(name="bar", sources=[test_source]) - ], + inputs=[InputDefinition(name="foo"), + InputDefinition(name="bar")], output=OutputDefinition() ) def valid(foo, bar): pass @solid( - inputs=[ - InputDefinition(name="foo", sources=[test_source]), - InputDefinition(name="bar", sources=[test_source]) - ], + inputs=[InputDefinition(name="foo"), + InputDefinition(name="bar")], output=OutputDefinition() ) @with_context @@ -281,10 +222,8 @@ def valid_rontext(context, foo, bar): pass @solid( - inputs=[ - InputDefinition(name="foo", sources=[test_source]), - InputDefinition(name="bar", sources=[test_source]) - ], + inputs=[InputDefinition(name="foo"), + InputDefinition(name="bar")], output=OutputDefinition() ) @with_context @@ -292,84 +231,6 @@ def valid_context_2(_context, foo, bar): pass -def test_source_definition_errors(): - with pytest.raises(DagsterInvalidDefinitionError): - - @source(argument_def_dict={'foo': ArgumentDefinition(types.String)}) - @with_context - def vargs(context, foo, *args): - pass - - with pytest.raises(DagsterInvalidDefinitionError): - - @source(argument_def_dict={'foo': ArgumentDefinition(types.String)}) - def wrong_name(bar): - pass - - with pytest.raises(DagsterInvalidDefinitionError): - - @source( - argument_def_dict={ - 'foo': ArgumentDefinition(types.String), - 'bar': ArgumentDefinition(types.String) - } - ) - def wrong_name_2(foo): - pass - - with pytest.raises(DagsterInvalidDefinitionError): - - @source(argument_def_dict={'foo': ArgumentDefinition(types.String)}) - @with_context - def no_context(foo): - pass - - with pytest.raises(DagsterInvalidDefinitionError): - - @source(argument_def_dict={'foo': ArgumentDefinition(types.String)}) - def yes_context(context, foo): - pass - - with pytest.raises(DagsterInvalidDefinitionError): - - @source(argument_def_dict={'foo': ArgumentDefinition(types.String)}) - def extras(foo, bar): - pass - - @source(argument_def_dict={'foo': ArgumentDefinition(types.String)}) - def valid_kwargs(**kwargs): - pass - - @source( - argument_def_dict={ - 'foo': ArgumentDefinition(types.String), - 'bar': ArgumentDefinition(types.String) - } - ) - def valid(foo, bar): - pass - - @source( - argument_def_dict={ - 'foo': ArgumentDefinition(types.String), - 'bar': ArgumentDefinition(types.String) - } - ) - @with_context - def valid_rontext(context, foo, bar): - pass - - @source( - argument_def_dict={ - 'foo': ArgumentDefinition(types.String), - 'bar': ArgumentDefinition(types.String) - } - ) - @with_context - def valid_context_2(_context, foo, bar): - pass - - def test_materialization_definition_errors(): with pytest.raises(DagsterInvalidDefinitionError): @@ -477,12 +338,6 @@ def solid_desc(): assert solid_desc.description == 'foo' - @source(description='bar') - def source_desc(): - pass - - assert source_desc.description == 'bar' - @materialization(description='baaz') def materialization_desc(): pass diff --git a/python_modules/dagster/dagster/core/core_tests/test_execute_solid.py b/python_modules/dagster/dagster/core/core_tests/test_execute_solid.py deleted file mode 100644 index d6f971bf1d1cf..0000000000000 --- a/python_modules/dagster/dagster/core/core_tests/test_execute_solid.py +++ /dev/null @@ -1,314 +0,0 @@ -import pytest - -from dagster import check, config -from dagster.core import types - -from dagster.core.definitions import ( - SolidDefinition, - ExpectationDefinition, - ExpectationResult, - ArgumentDefinition, - OutputDefinition, -) - -from dagster.core.execution import ( - output_single_solid, - ExecutionStepResult, - DagsterExecutionFailureReason, - ExecutionContext, - execute_single_solid, -) - -from dagster.core.errors import DagsterExpectationFailedError - -from dagster.utils.compatability import ( - create_custom_source_input, create_single_materialization_output -) - - -def create_test_context(): - return ExecutionContext() - - -def single_input_env(solid_name, input_name, args=None): - check.str_param(solid_name, 'solid_name') - check.str_param(input_name, 'input_name') - args = check.opt_dict_param(args, 'args') - return config.Environment(sources={solid_name: {input_name: config.Source('CUSTOM', args)}}) - - -def test_execute_solid_no_args(): - some_input = create_custom_source_input( - name='some_input', - source_fn=lambda context, arg_dict: [{'data_key': 'data_value'}], - argument_def_dict={} - ) - - def tranform_fn_inst(_context, args): - args['some_input'][0]['data_key'] = 'new_value' - return args['some_input'] - - test_output = {} - - def materialization_fn_inst(context, arg_dict, data): - assert isinstance(context, ExecutionContext) - assert isinstance(arg_dict, dict) - test_output['thedata'] = data - - custom_output = create_single_materialization_output( - name='CUSTOM', - materialization_fn=materialization_fn_inst, - argument_def_dict={}, - ) - - single_solid = SolidDefinition.single_output_transform( - name='some_node', - inputs=[some_input], - transform_fn=tranform_fn_inst, - output=custom_output, - ) - - output_single_solid( - create_test_context(), - single_solid, - environment=single_input_env('some_node', 'some_input'), - name='CUSTOM', - arg_dict={} - ) - - assert test_output['thedata'] == [{'data_key': 'new_value'}] - - -def create_single_dict_input(expectations=None): - return create_custom_source_input( - name='some_input', - source_fn=lambda context, arg_dict: [{'key': arg_dict['str_arg']}], - argument_def_dict={'str_arg' : ArgumentDefinition(types.String)}, - expectations=expectations or [], - ) - - -def create_noop_output(test_output, expectations=None): - def set_test_output(context, arg_dict, output): - assert isinstance(context, ExecutionContext) - assert arg_dict == {} - test_output['thedata'] = output - - return create_single_materialization_output( - name='CUSTOM', - materialization_fn=set_test_output, - argument_def_dict={}, - expectations=expectations - ) - - -def test_hello_world(): - def transform_fn(context, args): - assert isinstance(context, ExecutionContext) - hello_world_input = args['hello_world_input'] - assert isinstance(hello_world_input, dict) - hello_world_input['hello'] = 'world' - return hello_world_input - - output_events = {} - - def materialization_fn(context, arg_dict, data): - assert data['hello'] == 'world' - assert isinstance(context, ExecutionContext) - assert arg_dict == {} - output_events['called'] = True - - hello_world = SolidDefinition.single_output_transform( - name='hello_world', - inputs=[ - create_custom_source_input( - name='hello_world_input', - source_fn=lambda context, arg_dict: {}, - argument_def_dict={}, - ) - ], - transform_fn=transform_fn, - output=create_single_materialization_output( - name='CUSTOM', materialization_fn=materialization_fn, argument_def_dict={} - ), - ) - - result = execute_single_solid( - create_test_context(), - hello_world, - environment=single_input_env('hello_world', 'hello_world_input'), - ) - - assert result.success - - assert result.transformed_value['hello'] == 'world' - - assert 'called' not in output_events - - output_result = output_single_solid( - create_test_context(), - hello_world, - environment=single_input_env('hello_world', 'hello_world_input'), - name='CUSTOM', - arg_dict={} - ) - - assert output_result.success - - assert 'called' in output_events - - -def test_execute_solid_with_args(): - test_output = {} - - single_solid = SolidDefinition.single_output_transform( - name='some_node', - inputs=[create_single_dict_input()], - transform_fn=lambda context, args: args['some_input'], - output=create_noop_output(test_output), - ) - - result = output_single_solid( - create_test_context(), - single_solid, - environment=single_input_env('some_node', 'some_input', {'str_arg': 'an_input_arg'}), - name='CUSTOM', - arg_dict={}, - ) - - assert result.success - - assert test_output['thedata'][0]['key'] == 'an_input_arg' - - -def test_execute_solid_with_failed_input_expectation_non_throwing(): - single_solid = create_input_failing_solid() - - solid_execution_result = output_single_solid( - create_test_context(), - single_solid, - environment=single_input_env('some_node', 'some_input', {'str_arg': 'an_input_arg'}), - name='CUSTOM', - arg_dict={}, - throw_on_error=False, - ) - - assert isinstance(solid_execution_result, ExecutionStepResult) - assert solid_execution_result.success is False - # assert solid_execution_result.reason == DagsterExecutionFailureReason.EXPECTATION_FAILURE - - -def test_execute_solid_with_failed_input_expectation_throwing(): - single_solid = create_input_failing_solid() - - with pytest.raises(DagsterExpectationFailedError): - output_single_solid( - create_test_context(), - single_solid, - environment=single_input_env('some_node', 'some_input', {'str_arg': 'an_input_arg'}), - name='CUSTOM', - arg_dict={}, - ) - - with pytest.raises(DagsterExpectationFailedError): - output_single_solid( - create_test_context(), - single_solid, - environment=single_input_env('some_node', 'some_input', {'str_arg': 'an_input_arg'}), - name='CUSTOM', - arg_dict={}, - ) - - -def create_input_failing_solid(): - test_output = {} - - def failing_expectation_fn(_context, _info, _some_input): - return ExpectationResult(success=False) - - failing_expect = ExpectationDefinition(name='failing', expectation_fn=failing_expectation_fn) - - return SolidDefinition.single_output_transform( - name='some_node', - inputs=[create_single_dict_input(expectations=[failing_expect])], - transform_fn=lambda context, args: args['some_input'], - output=create_noop_output(test_output), - ) - - -def test_execute_solid_with_failed_output_expectation_non_throwing(): - failing_solid = create_output_failing_solid() - - solid_execution_result = output_single_solid( - create_test_context(), - failing_solid, - environment=single_input_env('some_node', 'some_input', {'str_arg': 'an_input_arg'}), - name='CUSTOM', - arg_dict={}, - throw_on_error=False - ) - - assert isinstance(solid_execution_result, ExecutionStepResult) - assert solid_execution_result.success is False - # assert solid_execution_result.reason == DagsterExecutionFailureReason.EXPECTATION_FAILURE - - -def test_execute_solid_with_failed_output_expectation_throwing(): - failing_solid = create_output_failing_solid() - - with pytest.raises(DagsterExpectationFailedError): - output_single_solid( - create_test_context(), - failing_solid, - environment=single_input_env('some_node', 'some_input', {'str_arg': 'an_input_arg'}), - name='CUSTOM', - arg_dict={}, - ) - - with pytest.raises(DagsterExpectationFailedError): - output_single_solid( - create_test_context(), - failing_solid, - environment=single_input_env('some_node', 'some_input', {'str_arg': 'an_input_arg'}), - name='CUSTOM', - arg_dict={}, - ) - - -def _set_key_value(ddict, key, value): - ddict[key] = value - - -def test_execute_solid_with_no_inputs(): - did_run_dict = {} - no_args_solid = SolidDefinition.single_output_transform( - name='no_args_solid', - inputs=[], - transform_fn=lambda context, args: _set_key_value(did_run_dict, 'did_run', True), - output=OutputDefinition(), - ) - - result = execute_single_solid( - ExecutionContext(), no_args_solid, environment=config.Environment.empty() - ) - - assert result.success - assert did_run_dict['did_run'] is True - - -def create_output_failing_solid(): - test_output = {} - - def failing_expectation_fn(_context, _info, _output): - return ExpectationResult(success=False) - - output_expectation = ExpectationDefinition( - name='output_failure', expectation_fn=failing_expectation_fn - ) - - return SolidDefinition.single_output_transform( - name='some_node', - inputs=[create_single_dict_input()], - transform_fn=lambda context, args: args['some_input'], - output=create_noop_output(test_output, expectations=[output_expectation]), - ) diff --git a/python_modules/dagster/dagster/core/core_tests/test_iterator_execution.py b/python_modules/dagster/dagster/core/core_tests/test_iterator_execution.py deleted file mode 100644 index 4cda4b1078d12..0000000000000 --- a/python_modules/dagster/dagster/core/core_tests/test_iterator_execution.py +++ /dev/null @@ -1,58 +0,0 @@ -import dagster -from dagster import config -from dagster.core.definitions import SolidDefinition -from dagster.utils.compatability import ( - create_custom_source_input, create_single_materialization_output -) -from dagster.core.execution import output_single_solid - - -def test_iterator_solid(): - def input_fn(_context, _arg_dict): - yield 1 - yield 2 - - some_input = create_custom_source_input( - name='iter_numbers', - source_fn=input_fn, - argument_def_dict={}, - ) - - def transform_fn(_context, args): - for value in args['iter_numbers']: - yield value + 1 - - output_spot = {} - - def materialization_fn(_context, _arg_dict, data_iter): - output_spot['list'] = list(data_iter) - - # in a real case we would iterate over - # and stream to disk - - custom_output = create_single_materialization_output( - name='CUSTOM', - materialization_fn=materialization_fn, - argument_def_dict={}, - ) - - iterable_solid = SolidDefinition.single_output_transform( - name='some_node', - inputs=[some_input], - transform_fn=transform_fn, - output=custom_output, - ) - - output_single_solid( - dagster.ExecutionContext(), - iterable_solid, - environment=config.Environment( - sources={'some_node': { - 'iter_numbers': config.Source('CUSTOM', {}) - }} - ), - name='CUSTOM', - arg_dict={} - ) - - assert output_spot['list'] == [2, 3] diff --git a/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py b/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py index 11ae0414e792d..0d37fc66cccdd 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py +++ b/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py @@ -26,7 +26,7 @@ def _t_fn(_context, _inputs, _config_dict): pipeline = PipelineDefinition(solids=[solid]) - result = execute_pipeline(pipeline, config.Environment(sources={})) + result = execute_pipeline(pipeline, config.Environment()) assert result.result_list[0].name == 'multiple_outputs' assert result.result_list[0].output_name == 'output_one' diff --git a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py index ca80fd7491d22..42bf9aa0b4ee8 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py +++ b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py @@ -5,27 +5,16 @@ DependencyDefinition, InputDefinition, OutputDefinition, + PipelineDefinition, + Result, SolidDefinition, - SourceDefinition, config, - types, execute_pipeline, - ExecutionContext, - Result, - PipelineDefinition, + types, ) -def define_pass_value_solid(name): - def _value_t_fn(_context, _inputs, config_dict): - yield Result(config_dict['value']) +from dagster.core.utility_solids import define_pass_value_solid - return SolidDefinition( - name=name, - inputs=[], - outputs=[OutputDefinition(dagster_type=types.String)], - config_def={'value': ArgumentDefinition(types.String)}, - transform_fn=_value_t_fn, - ) def test_execute_solid_with_input_same_name(): a_thing_solid = SolidDefinition.single_output_transform( @@ -37,34 +26,21 @@ def test_execute_solid_with_input_same_name(): pipeline = PipelineDefinition( solids=[define_pass_value_solid('pass_value'), a_thing_solid], - dependencies={ - 'a_thing' : { - 'a_thing' : DependencyDefinition('pass_value') - } - }, + dependencies={'a_thing': { + 'a_thing': DependencyDefinition('pass_value') + }}, ) result = execute_pipeline( pipeline, - config.Environment( - solids={'pass_value': config.Solid(config_dict={'value': 'foo'})} - ), + config.Environment(solids={'pass_value': config.Solid(config_dict={'value': 'foo'})}), ) assert result.result_named('a_thing').transformed_value == 'foofoo' def test_execute_two_solids_with_same_input_name(): - input_def = InputDefinition( - name='a_thing', - sources=[ - SourceDefinition( - source_type='a_source_type', - source_fn=lambda context, arg_dict: arg_dict['an_arg'], - argument_def_dict={'an_arg': ArgumentDefinition(types.String)}, - ), - ], - ) + input_def = InputDefinition(name='a_thing') solid_one = SolidDefinition.single_output_transform( 'solid_one', @@ -80,18 +56,33 @@ def test_execute_two_solids_with_same_input_name(): output=dagster.OutputDefinition(), ) - pipeline = dagster.PipelineDefinition(solids=[solid_one, solid_two]) + pipeline = dagster.PipelineDefinition( + solids=[ + define_pass_value_solid('pass_to_one'), + define_pass_value_solid('pass_to_two'), + solid_one, + solid_two, + ], + dependencies={ + 'solid_one': { + 'a_thing': DependencyDefinition('pass_to_one') + }, + 'solid_two': { + 'a_thing': DependencyDefinition('pass_to_two') + } + } + ) result = execute_pipeline( pipeline, environment=config.Environment( - sources={ - 'solid_one': { - 'a_thing': config.Source(name='a_source_type', args={'an_arg': 'foo'}) - }, - 'solid_two': { - 'a_thing': config.Source(name='a_source_type', args={'an_arg': 'bar'}) - }, + solids={ + 'pass_to_one': config.Solid({ + 'value': 'foo' + }), + 'pass_to_two': config.Solid({ + 'value': 'bar' + }), } ) ) @@ -103,20 +94,11 @@ def test_execute_two_solids_with_same_input_name(): def test_execute_dep_solid_different_input_name(): + pass_to_first = define_pass_value_solid('pass_to_first') + first_solid = SolidDefinition.single_output_transform( 'first_solid', - inputs=[ - InputDefinition( - name='a_thing', - sources=[ - SourceDefinition( - source_type='a_source_type', - source_fn=lambda context, arg_dict: arg_dict['an_arg'], - argument_def_dict={'an_arg': ArgumentDefinition(types.String)}, - ), - ], - ), - ], + inputs=[InputDefinition(name='a_thing')], transform_fn=lambda context, args: args['a_thing'] + args['a_thing'], output=dagster.OutputDefinition(), ) @@ -124,195 +106,33 @@ def test_execute_dep_solid_different_input_name(): second_solid = SolidDefinition.single_output_transform( 'second_solid', inputs=[ - InputDefinition( - name='an_input', - - ), + InputDefinition(name='an_input'), ], transform_fn=lambda context, args: args['an_input'] + args['an_input'], output=dagster.OutputDefinition(), ) - pipeline = dagster.PipelineDefinition( - solids=[first_solid, second_solid], + solids=[pass_to_first, first_solid, second_solid], dependencies={ + 'first_solid': { + 'a_thing': DependencyDefinition('pass_to_first'), + }, 'second_solid': { - 'an_input': DependencyDefinition('first_solid') - } + 'an_input': DependencyDefinition('first_solid'), + }, } ) result = dagster.execute_pipeline( pipeline, - environment=config.Environment( - sources={ - 'first_solid': { - 'a_thing': config.Source(name='a_source_type', args={'an_arg': 'bar'}) - } - } - ) + environment=config.Environment(solids={'pass_to_first': config.Solid({ + 'value': 'bar' + })}) ) assert result.success - assert len(result.result_list) == 2 - assert result.result_list[0].transformed_value == 'barbar' - assert result.result_list[1].transformed_value == 'barbarbarbar' - - -def test_execute_dep_solid_same_input_name(): - def s_fn(arg_dict, executed, key): - executed[key] = True - return arg_dict - - executed = { - 's1_t1_source': False, - 's2_t1_source': False, - 's2_t2_source': False, - } - - table_one = SolidDefinition.single_output_transform( - 'table_one', - inputs=[ - InputDefinition( - name='table_one', - sources=[ - SourceDefinition( - source_type='TABLE', - source_fn= - lambda context, arg_dict: s_fn(arg_dict, executed, 's1_t1_source'), - argument_def_dict={'name': ArgumentDefinition(types.String)}, - ), - ], - ), - ], - transform_fn=lambda context, args: args['table_one'], - output=dagster.OutputDefinition(), - ) - - table_two = SolidDefinition.single_output_transform( - 'table_two', - inputs=[ - InputDefinition( - name='table_one', - sources=[ - SourceDefinition( - source_type='TABLE', - source_fn= - lambda context, arg_dict: s_fn(arg_dict, executed, 's2_t1_source'), - argument_def_dict={'name': ArgumentDefinition(types.String)}, - ), - ], - ), - InputDefinition( - name='table_two', - sources=[ - SourceDefinition( - source_type='TABLE', - source_fn= - lambda context, arg_dict: s_fn(arg_dict, executed, 's2_t2_source'), - argument_def_dict={'name': ArgumentDefinition(types.String)}, - ), - ], - ), - ], - transform_fn=lambda context, args: args['table_two'], - output=dagster.OutputDefinition(), - ) - - pipeline = dagster.PipelineDefinition( - solids=[table_one, table_two], - dependencies={ - 'table_two' : { - 'table_one' : DependencyDefinition('table_one') - } - } - ) - - sources = { - 'table_one': { - 'table_one': config.Source(name='TABLE', args={'name': 'table_one_instance'}), - }, - 'table_two': { - 'table_two': config.Source(name='TABLE', args={'name': 'table_two_instance'}), - }, - } - - complete_environment = config.Environment(sources=sources) - - both_solids_result = dagster.execute_pipeline(pipeline, environment=complete_environment) - - assert executed == { - 's1_t1_source': True, - 's2_t1_source': False, - 's2_t2_source': True, - } - - assert both_solids_result.success - - assert len(both_solids_result.result_list) == 2 - assert both_solids_result.result_list[0].transformed_value == {'name': 'table_one_instance'} - assert both_solids_result.result_list[1].transformed_value == {'name': 'table_two_instance'} - - # reset execution marks - executed['s1_t1_source'] = False - executed['s2_t1_source'] = False - executed['s2_t2_source'] = False - - second_only_env = config.Environment( - sources={ - 'table_two': { - 'table_one': config.Source(name='TABLE', args={'name': 'table_one_instance'}), - 'table_two': config.Source(name='TABLE', args={'name': 'table_two_instance'}), - }, - }, - execution=config.Execution(from_solids=['table_two']), - ) - - second_solid_only_result = dagster.execute_pipeline(pipeline, environment=second_only_env) - - assert second_solid_only_result.success - assert len(second_solid_only_result.result_list) == 1 - assert second_solid_only_result.result_list[0].name == 'table_two' - assert second_solid_only_result.result_list[0].transformed_value == { - 'name': 'table_two_instance' - } - - assert executed == { - 's1_t1_source': False, - 's2_t1_source': True, - 's2_t2_source': True, - } - - # reset execution marks - executed['s1_t1_source'] = False - executed['s2_t1_source'] = False - executed['s2_t2_source'] = False - - first_only_env = config.Environment( - # sources=sources, - sources={ - 'table_one': { - 'table_one': config.Source(name='TABLE', args={'name': 'table_one_instance'}), - }, - }, - execution=config.Execution(through_solids=['table_one']), - ) - - first_solid_only_result = dagster.execute_pipeline( - pipeline, - environment=first_only_env, - ) - - assert first_solid_only_result.success - assert len(first_solid_only_result.result_list) == 1 - assert first_solid_only_result.result_list[0].name == 'table_one' - assert first_solid_only_result.result_list[0].transformed_value == { - 'name': 'table_one_instance' - } - - assert executed == { - 's1_t1_source': True, - 's2_t1_source': False, - 's2_t2_source': False, - } + assert len(result.result_list) == 3 + assert result.result_list[0].transformed_value == 'bar' + assert result.result_list[1].transformed_value == 'barbar' + assert result.result_list[2].transformed_value == 'barbarbarbar' diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py index 0733c6c13ce4d..522c700e367ff 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py @@ -1,28 +1,23 @@ -from dagster import check -from dagster import config - -import dagster -import dagster.core -from dagster.core.definitions import ( - SolidDefinition, - InputDefinition, +from dagster import ( DependencyDefinition, -) -from dagster.core.execution import ( - DagsterExecutionFailureReason, - execute_pipeline, ExecutionContext, + InputDefinition, + OutputDefinition, + PipelineContextDefinition, + PipelineDefinition, + SolidDefinition, + check, + config, + execute_pipeline, ) + from dagster.core.errors import DagsterUserCodeExecutionError -from dagster.utils.compatability import ( - create_custom_source_input, create_single_materialization_output -) def silencing_default_context(): return { 'default': - dagster.PipelineContextDefinition( + PipelineContextDefinition( argument_def_dict={}, context_fn=lambda _pipeline, _args: ExecutionContext(), ) @@ -30,151 +25,43 @@ def silencing_default_context(): def silencing_pipeline(solids, dependencies=None): - return dagster.PipelineDefinition( + return PipelineDefinition( solids=solids, dependencies=dependencies, context_definitions=silencing_default_context(), ) -def create_failing_output_def(): - def failing_materialization_fn(*_args, **_kwargs): - raise Exception('something bad happened') - - return create_single_materialization_output( - name='CUSTOM', - materialization_fn=failing_materialization_fn, - argument_def_dict={}, - ) - - -def create_input_set_input_def(input_name): - return create_custom_source_input( - input_name, - source_fn=lambda context, arg_dict: [{input_name: 'input_set'}], - argument_def_dict={}, - ) - - def create_root_success_solid(name): - input_name = name + '_input' - - def root_transform(_context, args): - passed_rows = list(args.values())[0] + def root_transform(_context, _args): + passed_rows = [] passed_rows.append({name: 'transform_called'}) return passed_rows return SolidDefinition.single_output_transform( name=name, - inputs=[create_input_set_input_def(input_name)], + inputs=[], transform_fn=root_transform, - output=dagster.OutputDefinition(), + output=OutputDefinition(), ) def create_root_transform_failure_solid(name): - input_name = name + '_input' - inp = create_custom_source_input( - input_name, - source_fn=lambda context, arg_dict: [{input_name: 'input_set'}], - argument_def_dict={}, - ) - def failed_transform(**_kwargs): raise Exception('Transform failed') return SolidDefinition.single_output_transform( name=name, - inputs=[inp], + inputs=[], transform_fn=failed_transform, - output=dagster.OutputDefinition(), - ) - - -def create_root_input_failure_solid(name): - def failed_input_fn(_context, _args): - raise Exception('something bad happened') - - input_name = name + '_input' - inp = create_custom_source_input( - input_name, - source_fn=failed_input_fn, - argument_def_dict={}, - ) - - return SolidDefinition.single_output_transform( - name=name, - inputs=[inp], - transform_fn=lambda **_kwargs: {}, - output=dagster.OutputDefinition(), - ) - - -def create_root_output_failure_solid(name): - input_name = name + '_input' - - def root_transform(**kwargs): - passed_rows = list(kwargs.values())[0] - passed_rows.append({name: 'transform_called'}) - return passed_rows - - return SolidDefinition.single_output_transform( - name=name, - inputs=[create_input_set_input_def(input_name)], - transform_fn=root_transform, - output=create_failing_output_def(), - ) - - -def no_args_env(solid_name, input_name, materializations=None): - return config.Environment( - sources={solid_name: { - input_name: config.Source(name='CUSTOM', args={}) - }}, - materializations=materializations, + output=OutputDefinition(), ) def test_transform_failure_pipeline(): pipeline = silencing_pipeline(solids=[create_root_transform_failure_solid('failing')]) pipeline_result = execute_pipeline( - pipeline, environment=no_args_env('failing', 'failing_input'), throw_on_error=False - ) - - assert not pipeline_result.success - - result_list = pipeline_result.result_list - - assert len(result_list) == 1 - assert not result_list[0].success - assert result_list[0].dagster_user_exception - - -def test_input_failure_pipeline(): - pipeline = silencing_pipeline(solids=[create_root_input_failure_solid('failing_input')]) - pipeline_result = execute_pipeline( - pipeline, - environment=no_args_env('failing_input', 'failing_input_input'), - throw_on_error=False - ) - - result_list = pipeline_result.result_list - - assert len(result_list) == 1 - assert not result_list[0].success - assert result_list[0].dagster_user_exception - - -def test_output_failure_pipeline(): - pipeline = silencing_pipeline(solids=[create_root_output_failure_solid('failing_output')]) - - pipeline_result = execute_pipeline( - pipeline, - environment=no_args_env( - 'failing_output', 'failing_output_input', - [config.Materialization(solid='failing_output', name='CUSTOM', args={})] - ), - throw_on_error=False, + pipeline, environment=config.Environment(), throw_on_error=False ) assert not pipeline_result.success @@ -198,19 +85,9 @@ def transform_fn(_context, args): name='C', inputs=[InputDefinition(name='A'), InputDefinition(name='B')], transform_fn=transform_fn, - output=dagster.OutputDefinition(), + output=OutputDefinition(), ) - environment = config.Environment( - sources={ - 'A': { - 'A_input': config.Source('CUSTOM', {}), - }, - 'B': { - 'B_input': config.Source('CUSTOM', {}) - } - } - ) pipeline = silencing_pipeline( solids=[solid_a, solid_b, solid_c], dependencies={ @@ -222,7 +99,7 @@ def transform_fn(_context, args): ) pipeline_result = execute_pipeline( pipeline, - environment=environment, + environment=config.Environment(), throw_on_error=False, ) diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py index ba1e59c091805..5ccc4f9aee2fe 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py @@ -2,6 +2,7 @@ from dagster import ( DependencyDefinition, + InputDefinition, OutputDefinition, PipelineDefinition, SolidDefinition, @@ -19,8 +20,6 @@ ExecutionStepResult, ) -from dagster.utils.compatability import create_custom_source_input - # protected members # pylint: disable=W0212 @@ -39,21 +38,11 @@ def create_dep_input_fn(name): def create_solid_with_deps(name, *solid_deps): - # def throw_input_fn(arg_dict): - # return [{name: 'input_set'}] - - inputs = [ - create_custom_source_input( - solid_dep.name, - source_fn=create_dep_input_fn(solid_dep.name), - argument_def_dict={}, - ) for solid_dep in solid_deps - ] + inputs = [InputDefinition(solid_dep.name) for solid_dep in solid_deps] def dep_transform(_context, args): passed_rows = list(args.values())[0] passed_rows.append({name: 'transform_called'}) - #return copy.deepcopy(passed_rows) return passed_rows return SolidDefinition.single_output_transform( @@ -66,16 +55,12 @@ def dep_transform(_context, args): def create_root_solid(name): input_name = name + '_input' - inp = create_custom_source_input( - input_name, - source_fn=lambda context, arg_dict: [{input_name: 'input_set'}], - argument_def_dict={}, - ) + inp = InputDefinition(input_name) def root_transform(_context, args): passed_rows = list(args.values())[0] + passed_rows.append({name: 'transform_called'}) - #return copy.deepcopy(passed_rows) return passed_rows return SolidDefinition.single_output_transform( @@ -121,24 +106,26 @@ def graph_from_solids_only(solids, dependencies): def test_diamond_deps_adjaceny_lists(): - # A <-- (B, C) <-- D - - node_a = create_root_solid('A') - node_b = create_solid_with_deps('B', node_a) - node_c = create_solid_with_deps('C', node_a) - node_d = create_solid_with_deps('D', node_b, node_c) - forward_edges, backwards_edges = _do_construct( - [node_a, node_b, node_c, node_d], + create_diamond_solids(), diamond_deps(), ) - assert forward_edges == {'A': {'B', 'C'}, 'B': {'D'}, 'C': {'D'}, 'D': set()} - assert backwards_edges == {'D': {'B', 'C'}, 'B': {'A'}, 'C': {'A'}, 'A': set()} + assert forward_edges == {'A_source': {'A'}, 'A': {'B', 'C'}, 'B': {'D'}, 'C': {'D'}, 'D': set()} + assert backwards_edges == { + 'D': {'B', 'C'}, + 'B': {'A'}, + 'C': {'A'}, + 'A': {'A_source'}, + 'A_source': set() + } def diamond_deps(): return { + 'A': { + 'A_input': DependencyDefinition('A_source'), + }, 'B': { 'A': DependencyDefinition('A') }, @@ -175,12 +162,16 @@ def test_disconnected_graphs_adjaceny_lists(): assert backwards_edges == {'B': {'A'}, 'A': set(), 'D': {'C'}, 'C': set()} +from dagster.core.utility_solids import define_pass_mem_value + + def create_diamond_solids(): + a_source = define_pass_mem_value('A_source', [input_set('A_input')]) node_a = create_root_solid('A') node_b = create_solid_with_deps('B', node_a) node_c = create_solid_with_deps('C', node_a) node_d = create_solid_with_deps('D', node_b, node_c) - return [node_d, node_c, node_b, node_a] + return [node_d, node_c, node_b, node_a, a_source] def create_diamond_graph(): @@ -189,7 +180,7 @@ def create_diamond_graph(): def test_diamond_toposort(): graph = create_diamond_graph() - assert graph.topological_order == ['A', 'B', 'C', 'D'] + assert graph.topological_order == ['A_source', 'A', 'B', 'C', 'D'] def test_single_node_unprovided_inputs(): @@ -199,45 +190,35 @@ def test_single_node_unprovided_inputs(): assert solid_graph.compute_unprovided_inputs('A', []) == set(['A_input']) -def test_diamond_toposort_unprovided_inputs(): - solid_graph = create_diamond_graph() +# need to replicate these tests with new api +# def test_diamond_toposort_unprovided_inputs(): +# solid_graph = create_diamond_graph() - # no inputs - assert solid_graph.compute_unprovided_inputs('A', []) == set(['A_input']) - assert solid_graph.compute_unprovided_inputs('B', []) == set(['A_input']) - assert solid_graph.compute_unprovided_inputs('C', []) == set(['A_input']) - assert solid_graph.compute_unprovided_inputs('D', []) == set(['A_input']) +# # no inputs +# assert solid_graph.compute_unprovided_inputs('A', []) == set(['A_input']) +# assert solid_graph.compute_unprovided_inputs('B', []) == set(['A_input']) +# assert solid_graph.compute_unprovided_inputs('C', []) == set(['A_input']) +# assert solid_graph.compute_unprovided_inputs('D', []) == set(['A_input']) - # root input - assert solid_graph.compute_unprovided_inputs('A', ['A_input']) == set() - assert solid_graph.compute_unprovided_inputs('B', ['A_input']) == set() - assert solid_graph.compute_unprovided_inputs('C', ['A_input']) == set() - assert solid_graph.compute_unprovided_inputs('D', ['A_input']) == set() - - # immediate input - assert solid_graph.compute_unprovided_inputs('A', ['A_input']) == set() - assert solid_graph.compute_unprovided_inputs('B', ['A']) == set() - assert solid_graph.compute_unprovided_inputs('C', ['A']) == set() - assert solid_graph.compute_unprovided_inputs('D', ['B', 'C']) == set() - - # mixed satisified inputs - assert solid_graph.compute_unprovided_inputs('D', ['A_input', 'C']) == set() - assert solid_graph.compute_unprovided_inputs('D', ['B', 'A_input']) == set() - - # mixed unsatisifed inputs - assert solid_graph.compute_unprovided_inputs('D', ['C']) == set(['A_input']) - assert solid_graph.compute_unprovided_inputs('D', ['B']) == set(['A_input']) +# # root input +# assert solid_graph.compute_unprovided_inputs('A', ['A_input']) == set() +# assert solid_graph.compute_unprovided_inputs('B', ['A_input']) == set() +# assert solid_graph.compute_unprovided_inputs('C', ['A_input']) == set() +# assert solid_graph.compute_unprovided_inputs('D', ['A_input']) == set() +# # immediate input +# assert solid_graph.compute_unprovided_inputs('A', ['A_input']) == set() +# assert solid_graph.compute_unprovided_inputs('B', ['A']) == set() +# assert solid_graph.compute_unprovided_inputs('C', ['A']) == set() +# assert solid_graph.compute_unprovided_inputs('D', ['B', 'C']) == set() -def test_unprovided_input_param_invariants(): - node_a = create_root_solid('A') - solid_graph = graph_from_solids_only([node_a], {}) +# # mixed satisified inputs +# assert solid_graph.compute_unprovided_inputs('D', ['A_input', 'C']) == set() +# assert solid_graph.compute_unprovided_inputs('D', ['B', 'A_input']) == set() - with pytest.raises(check.ParameterCheckError): - solid_graph.compute_unprovided_inputs('B', []) - - with pytest.raises(check.ParameterCheckError): - solid_graph.compute_unprovided_inputs('A', ['no_input_here']) +# # mixed unsatisifed inputs +# assert solid_graph.compute_unprovided_inputs('D', ['C']) == set(['A_input']) +# assert solid_graph.compute_unprovided_inputs('D', ['B']) == set(['A_input']) def test_execution_subgraph_one_node(): @@ -304,67 +285,65 @@ def assert_all_results_equivalent(expected_results, result_results): def test_pipeline_execution_graph_diamond(): pipeline = PipelineDefinition(solids=create_diamond_solids(), dependencies=diamond_deps()) - environment = config.Environment(sources={'A': {'A_input': config.Source('CUSTOM', {})}}) - return _do_test(pipeline, lambda: execute_pipeline_iterator( + environment = config.Environment() + return _do_test(lambda: execute_pipeline_iterator( pipeline, environment=environment, )) -def test_pipeline_execution_graph_diamond_in_memory(): - pipeline = PipelineDefinition(solids=create_diamond_solids(), dependencies=diamond_deps()) - input_values = {'A_input': [{'A_input': 'input_set'}]} - return _do_test(pipeline, lambda: execute_pipeline_iterator_in_memory( - ExecutionContext(), - pipeline, - input_values={'A': input_values}, - )) - +# def test_pipeline_execution_graph_diamond_in_memory(): +# pipeline = PipelineDefinition(solids=create_diamond_solids(), dependencies=diamond_deps()) +# # input_values = {'A_input': [{'A_input': 'input_set'}]} +# return _do_test(lambda: execute_pipeline_iterator_in_memory( +# ExecutionContext(), +# pipeline, +# {} +# # input_values={'A': input_values}, +# )) -def _do_test(_pipeline, do_execute_pipeline_iter): - solid_graph = create_diamond_graph() - pipeline = PipelineDefinition(solids=solid_graph.solids) +def _do_test(do_execute_pipeline_iter): results = list() for result in do_execute_pipeline_iter(): results.append(result.copy()) - assert results[0].transformed_value[0] == input_set('A_input') - assert results[0].transformed_value[1] == transform_called('A') + assert results[1].transformed_value[0] == input_set('A_input') + assert results[1].transformed_value[1] == transform_called('A') - assert results[0].transformed_value == [input_set('A_input'), transform_called('A')] + assert results[1].transformed_value == [input_set('A_input'), transform_called('A')] - assert results[1].transformed_value == [ + assert results[2].transformed_value == [ input_set('A_input'), transform_called('A'), transform_called('C'), - ] or results[1].transformed_value == [ + ] or results[2].transformed_value == [ input_set('A_input'), transform_called('A'), transform_called('B'), ] - assert results[2].transformed_value == [ + assert results[3].transformed_value == [ input_set('A_input'), transform_called('A'), transform_called('C'), transform_called('B'), - ] or [ + ] or results[3].transformed_value == [ input_set('A_input'), transform_called('A'), transform_called('B'), transform_called('C'), ] - assert results[3].transformed_value == [ + assert results[4].transformed_value == [ input_set('A_input'), transform_called('A'), transform_called('C'), transform_called('B'), transform_called('D'), - ] or [ + ] or results[4].transformed_value == [ input_set('A_input'), transform_called('A'), transform_called('B'), diff --git a/python_modules/dagster/dagster/core/core_tests/test_transform_only_pipeline.py b/python_modules/dagster/dagster/core/core_tests/test_transform_only_pipeline.py index 81a8bc46c6bd0..fcd7277006698 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_transform_only_pipeline.py +++ b/python_modules/dagster/dagster/core/core_tests/test_transform_only_pipeline.py @@ -8,7 +8,6 @@ config, execute_pipeline, ) -from dagster.utils.compatability import create_custom_source_input def _set_key_value(ddict, key, value): @@ -26,15 +25,9 @@ def test_execute_solid_with_dep_only_inputs_no_api(): output=OutputDefinition(), ) - only_dep_input = create_custom_source_input( - name='step_one_solid', - source_fn=lambda arg_dict: check.not_implemented('should not get here'), - argument_def_dict={}, - ) - step_two_solid = SolidDefinition.single_output_transform( name='step_two_solid', - inputs=[only_dep_input], + inputs=[InputDefinition('step_one_solid')], transform_fn=lambda context, args: _set_key_value(did_run_dict, 'step_two', True), output=OutputDefinition(), ) diff --git a/python_modules/dagster/dagster/core/decorators.py b/python_modules/dagster/dagster/core/decorators.py index 4dafc3213cd0c..491dc71f4a226 100644 --- a/python_modules/dagster/dagster/core/decorators.py +++ b/python_modules/dagster/dagster/core/decorators.py @@ -3,7 +3,6 @@ from dagster import check from .definitions import ( SolidDefinition, - SourceDefinition, InputDefinition, OutputDefinition, MaterializationDefinition, @@ -80,52 +79,6 @@ def transform(context, args): return transform -class _Source: - def __init__(self, name=None, argument_def_dict=None, description=None): - self.source_type = check.opt_str_param(name, 'name') - self.argument_def_dict = check.opt_dict_param(argument_def_dict, 'argument_def_dict') - self.description = check.opt_str_param(description, 'description') - - def __call__(self, fn): - include_context = getattr(fn, 'has_context', False) - if include_context: - fn = fn.fn - - if not self.source_type: - self.source_type = fn.__name__ - - _validate_source_fn(fn, self.source_type, self.argument_def_dict, include_context) - source_fn = _create_source_wrapper(fn, self.argument_def_dict, include_context) - - return SourceDefinition( - source_type=self.source_type, - source_fn=source_fn, - argument_def_dict=self.argument_def_dict, - description=self.description, - ) - - -def source(*, name=None, argument_def_dict=None, description=None): - return _Source(name=name, argument_def_dict=argument_def_dict, description=description) - - -def _create_source_wrapper(fn, arg_def_dict, include_context=False): - arg_names = arg_def_dict.keys() - - @wraps(fn) - def source_fn(context, args): - kwargs = {} - for arg in arg_names: - kwargs[arg] = args[arg] - - if include_context: - return fn(context, **kwargs) - else: - return fn(**kwargs) - - return source_fn - - class _Materialization: def __init__(self, name=None, argument_def_dict=None, description=None): self.name = check.opt_str_param(name, 'name') diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index 9cebab6b389c9..75f8542b3bb21 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -216,34 +216,6 @@ def __init__(self, name, expectation_fn, description=None): self.description = check.opt_str_param(description, 'description') -class SourceDefinition: - ''' - name: name of the source - - source_fn: callable - The input function defines exactly what happens when the source is invoked. - - def simplified_read_csv_example(context, arg_dict): - context.info('I am in an input.') # use context for logging - return pd.read_csv(arg_dict['path']) - - argument_def_dict: { str: DagsterType } - Define the arguments expected by this source . A dictionary that maps a string - (argument name) to an argument type (defined in dagster.core.types) Continuing - the above example, the csv signature would be: - - argument_def_dict = {'path' : dagster.core.types.Path } - - ''' - - def __init__(self, source_type, source_fn, argument_def_dict, description=None): - check.callable_param(source_fn, 'source_fn') - self.source_type = check_valid_name(source_type) - self.source_fn = check.callable_param(source_fn, 'source_fn') - self.argument_def_dict = check_argument_def_dict(argument_def_dict) - self.description = check.opt_str_param(description, 'description') - - class InputDefinition: ''' An InputDefinition instances represents an argument to a transform defined within a solid. @@ -263,21 +235,10 @@ class InputDefinition: ''' def __init__( - self, - name, - dagster_type=None, - sources=None, - expectations=None, - input_callback=None, - description=None + self, name, dagster_type=None, expectations=None, input_callback=None, description=None ): self.name = check_valid_name(name) - if sources is None and dagster_type is not None: - sources = dagster_type.default_sources - - self.sources = check.opt_list_param(sources, 'sources', of_type=SourceDefinition) - self.dagster_type = check.opt_inst_param( dagster_type, 'dagster_type', types.DagsterType, types.Any ) @@ -288,19 +249,6 @@ def __init__( self.input_callback = check.opt_callable_param(input_callback, 'input_callback') self.description = check.opt_str_param(description, 'description') - def source_of_type(self, source_type): - check.str_param(source_type, 'source_type') - for source in self.sources: - if source.source_type == source_type: - return source - - check.failed( - 'Source {source_type} not found in input {input_name}.'.format( - source_type=source_type, - input_name=self.name, - ) - ) - class MaterializationDefinition: ''' diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index 9470cf4582705..753ea9237860e 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -140,7 +140,6 @@ def execute_single_solid(context, solid, environment, throw_on_error=True): check.invariant(environment.execution.through_solids == []) single_solid_environment = config.Environment( - sources=environment.sources, materializations=environment.materializations, expectations=environment.expectations, context=environment.context, @@ -194,7 +193,6 @@ def output_single_solid( ), environment=config.Environment( context=environment.context, - sources=environment.sources, materializations=[ config.Materialization( solid=solid.name, name=name, args=arg_dict @@ -257,56 +255,7 @@ def _wrap(): return thing -class InMemoryEnv(DagsterEnv): - def __init__(self, context, pipeline, input_values, from_solids=None, through_solids=None): - super().__init__() - self.context = check.inst_param(context, 'context', ExecutionContext) - self.pipeline = check.inst_param(pipeline, 'pipeline', PipelineDefinition) - self.input_values = check.dict_param(input_values, 'input_values', key_type=str) - self._from_solids = check.opt_list_param(from_solids, from_solids, of_type=str) - self._through_solids = check.opt_list_param(through_solids, through_solids, of_type=str) - - @property - def from_solids(self): - return self._from_solids - - @property - def through_solids(self): - return self._through_solids - - def config_dict_for_solid(self, name): - check.str_param(name, 'name') - return {} - - @contextmanager - def yield_context(self): - return _wrap_in_yield(self.context) - - @property - def materializations(self): - return [] - - @property - def evaluate_expectations(self): - return True - - def _validate_environment(environment, pipeline): - for solid_name, input_configs in environment.sources.items(): - if not pipeline.has_solid(solid_name): - raise DagsterInvariantViolationError( - f'Solid "{solid_name} not found' - ) - - solid_inst = pipeline.solid_named(solid_name) - - for input_name, _source_configs in input_configs.items(): - if not solid_inst.has_input(input_name): - raise DagsterInvariantViolationError( - f'Input "{input_name}" not found in the pipeline on solid "{solid_name}".' + \ - f'Input must be one of {repr([inp.name for inp in solid_inst.inputs])}' - ) - context_name = environment.context.name if context_name not in pipeline.context_definitions: @@ -316,9 +265,8 @@ def _validate_environment(environment, pipeline): ) -class ConfigEnv(DagsterEnv): +class DagsterEnv: def __init__(self, pipeline, environment): - super().__init__() # This is not necessarily the best spot for these calls _validate_environment(environment, pipeline) self.pipeline = check.inst_param(pipeline, 'pipeline', PipelineDefinition) @@ -369,12 +317,12 @@ def execute_pipeline_iterator(pipeline, environment): check.inst_param(pipeline, 'pipeline', PipelineDefinition) check.inst_param(environment, 'enviroment', config.Environment) - env = ConfigEnv(pipeline, environment) + env = DagsterEnv(pipeline, environment) with env.yield_context() as context: return _execute_pipeline_iterator( context, pipeline, - ConfigEnv(pipeline, environment) + DagsterEnv(pipeline, environment) ) def execute_pipeline_iterator_in_memory( @@ -442,7 +390,7 @@ def execute_pipeline( check.inst_param(environment, 'environment', config.Environment) return _execute_pipeline( pipeline, - ConfigEnv(pipeline, environment), + DagsterEnv(pipeline, environment), throw_on_error, ) diff --git a/python_modules/dagster/dagster/core/types.py b/python_modules/dagster/dagster/core/types.py index 017c4ca38a998..967b32cdb731a 100644 --- a/python_modules/dagster/dagster/core/types.py +++ b/python_modules/dagster/dagster/core/types.py @@ -17,7 +17,6 @@ def is_python_valid_value(self, _value): class DagsterScalarType(DagsterType): def __init__(self, *args, **kwargs): super().__init__(*args, **kwargs) - self.default_sources = [] self.default_materializations = [] @@ -40,15 +39,11 @@ def __init__( self, name, python_type, - default_sources=None, default_materializations=None, description=None, ): super().__init__(name, description) self.python_type = check.type_param(python_type, 'python_type') - self.default_sources = check.opt_list_param( - default_sources, 'default_sources', of_type=dagster.SourceDefinition - ) self.default_materializations = check.opt_list_param( default_materializations, 'default_materializations', diff --git a/python_modules/dagster/dagster/core/utility_solids.py b/python_modules/dagster/dagster/core/utility_solids.py new file mode 100644 index 0000000000000..d76188ff6fff8 --- /dev/null +++ b/python_modules/dagster/dagster/core/utility_solids.py @@ -0,0 +1,38 @@ +from dagster import ( + ArgumentDefinition, + OutputDefinition, + Result, + SolidDefinition, + check, + types, +) + + +def define_pass_value_solid(name): + check.str_param(name, 'name') + + def _value_t_fn(_context, _inputs, config_dict): + yield Result(config_dict['value']) + + return SolidDefinition( + name=name, + inputs=[], + outputs=[OutputDefinition(dagster_type=types.String)], + config_def={'value': ArgumentDefinition(types.String)}, + transform_fn=_value_t_fn, + ) + + +def define_pass_mem_value(name, value): + check.str_param(name, 'name') + + def _value_t_fn(_context, _inputs, _config_dict): + yield Result(value) + + return SolidDefinition( + name=name, + inputs=[], + outputs=[OutputDefinition()], + config_def={}, + transform_fn=_value_t_fn, + ) diff --git a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/pandas_hello_world/test_pandas_hello_world.py b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/pandas_hello_world/test_pandas_hello_world.py index fe4d54c821fe3..c32f09c97bca9 100644 --- a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/pandas_hello_world/test_pandas_hello_world.py +++ b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/pandas_hello_world/test_pandas_hello_world.py @@ -9,22 +9,19 @@ print_solids, ) -from dagster.dagster_examples.pandas_hello_world.pipeline import define_pipeline +from dagster.dagster_examples.pandas_hello_world.pipeline import define_success_pipeline def test_pipeline_include(): - assert define_pipeline() + assert define_success_pipeline() def test_execute_pipeline(): - pipeline = define_pipeline() + pipeline = define_success_pipeline() environment = config.Environment( - sources={ - 'sum_solid': { - 'num': config.Source(name='CSV', args={'path': script_relative_path('num.csv')}) - } - }, - execution=config.Execution(from_solids=['sum_solid'], through_solids=['sum_sq_solid']), + solids={'load_num_csv': config.Solid({ + 'path': script_relative_path('num.csv') + })}, ) result = execute_pipeline(pipeline, environment=environment) @@ -55,7 +52,7 @@ def test_cli_execute(): os.chdir(script_relative_path('../..')) do_execute_command( - define_pipeline(), + define_success_pipeline(), script_relative_path('../../pandas_hello_world/env.yml'), lambda *_args, **_kwargs: None, ) @@ -65,6 +62,5 @@ def test_cli_execute(): def test_cli_print(): - print_pipeline(define_pipeline(), full=False, print_fn=lambda *_args, **_kwargs: None) - print_pipeline(define_pipeline(), full=True, print_fn=lambda *_args, **_kwargs: None) - print_solids(define_pipeline(), print_fn=lambda *_args, **_kwargs: None) + print_pipeline(define_success_pipeline(), full=False, print_fn=lambda *_args, **_kwargs: None) + print_pipeline(define_success_pipeline(), full=True, print_fn=lambda *_args, **_kwargs: None) diff --git a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py index c996453446e72..43413bba1b7a9 100644 --- a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py +++ b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py @@ -145,10 +145,7 @@ def test_full_in_memory_pipeline(): def test_full_persisted_pipeline(): pipeline = create_full_pipeline() pipeline_result = dagster.execute_pipeline( - pipeline, - environment=config.Environment( - sources={}, context=config.Context(name='persisted', args={}) - ) + pipeline, environment=config.Environment(context=config.Context(name='persisted', args={})) ) assert pipeline_result.success diff --git a/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/env.yml b/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/env.yml index 688df483defbb..d6c14ff14d915 100644 --- a/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/env.yml +++ b/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/env.yml @@ -1,25 +1,19 @@ -execution: - from: sum_solid - through: sum_sq_solid - context: name: default args: log_level: ERROR -sources: - sum_solid: - num: - name: CSV - args: - path: 'pandas_hello_world/num.csv' +solids: + load_num_csv: + config: + path: "pandas_hello_world/num.csv" materializations: - solid: sum_solid name: CSV args: - path: '/tmp/sum.csv' + path: "/tmp/sum.csv" - solid: sum_sq_solid name: CSV args: - path: '/tmp/sum_sq.csv' + path: "/tmp/sum_sq.csv" diff --git a/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py b/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py index b1ca78275a5cf..7a92fc1b71e56 100644 --- a/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py +++ b/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py @@ -41,11 +41,15 @@ def define_pipeline(): return dagster.PipelineDefinition( name='pandas_hello_world_fails', solids=[ + dagster_pd.load_csv_solid('load_num_csv'), sum_solid, sum_sq_solid, always_fails_solid, ], dependencies={ + 'sum_solid': { + 'num': DependencyDefinition('load_num_csv') + }, 'sum_sq_solid': { 'sum_df': DependencyDefinition(sum_solid.name), }, @@ -57,4 +61,15 @@ def define_pipeline(): def define_success_pipeline(): - return PipelineDefinition(name='pandas_hello_world', solids=[sum_solid, sum_sq_solid]) + return PipelineDefinition( + name='pandas_hello_world', + solids=[dagster_pd.load_csv_solid('load_num_csv'), sum_solid, sum_sq_solid], + dependencies={ + 'sum_solid': { + 'num': DependencyDefinition('load_num_csv') + }, + 'sum_sq_solid': { + 'sum_df': DependencyDefinition(sum_solid.name), + }, + }, + ) diff --git a/python_modules/dagster/dagster/pandas_kernel/__init__.py b/python_modules/dagster/dagster/pandas_kernel/__init__.py index 088a1ac1c0ca2..d9ab61aee50e3 100644 --- a/python_modules/dagster/dagster/pandas_kernel/__init__.py +++ b/python_modules/dagster/dagster/pandas_kernel/__init__.py @@ -3,106 +3,58 @@ import pandas as pd -import dagster -from dagster import (check, types) - - -def parquet_dataframe_source(**read_parquet_kwargs): - def callback(context, arg_dict): - check.inst_param(context, 'context', dagster.ExecutionContext) - check.str_param(arg_dict['path'], 'path') - df = pd.read_parquet(arg_dict['path'], **read_parquet_kwargs) - context.metric('rows', df.shape[0]) - return df - - return dagster.SourceDefinition( - source_type='PARQUET', - source_fn=callback, - argument_def_dict={ - 'path': dagster.ArgumentDefinition(dagster.types.Path), - }, - ) - - -def csv_dataframe_source(name=None, **read_csv_kwargs): - def callback(context, arg_dict): - check.inst_param(context, 'context', dagster.ExecutionContext) - check.str_param(arg_dict['path'], 'path') - df = pd.read_csv(arg_dict['path'], **read_csv_kwargs) - context.metric('rows', df.shape[0]) - return df - - return dagster.SourceDefinition( - source_type=check.opt_str_param(name, 'name', 'CSV'), - source_fn=callback, - argument_def_dict={ - 'path': dagster.ArgumentDefinition(dagster.types.Path), - }, - ) - - -def table_dataframe_source(**read_table_kwargs): - def callback(context, arg_dict): - check.inst_param(context, 'context', dagster.ExecutionContext) - path = check.str_elem(arg_dict, 'path') - df = pd.read_table(path, **read_table_kwargs) - context.metric('rows', df.shape[0]) - return df - - return dagster.SourceDefinition( - source_type='TABLE', - source_fn=callback, - argument_def_dict={ - 'path': dagster.ArgumentDefinition(dagster.types.Path), - }, - ) +from dagster import ( + ArgumentDefinition, + ExecutionContext, + MaterializationDefinition, + OutputDefinition, + Result, + SolidDefinition, + check, + types, +) def dataframe_csv_materialization(): def to_csv_fn(context, arg_dict, df): check.inst_param(df, 'df', pd.DataFrame) - check.inst_param(context, 'context', dagster.ExecutionContext) + check.inst_param(context, 'context', ExecutionContext) check.dict_param(arg_dict, 'arg_dict') path = check.str_elem(arg_dict, 'path') df.to_csv(path, index=False) - return dagster.MaterializationDefinition( + return MaterializationDefinition( name='CSV', materialization_fn=to_csv_fn, - argument_def_dict={'path': dagster.ArgumentDefinition(types.Path)}, + argument_def_dict={'path': ArgumentDefinition(types.Path)}, ) def dataframe_parquet_materialization(): def to_parquet_fn(context, arg_dict, df): check.inst_param(df, 'df', pd.DataFrame) - check.inst_param(context, 'context', dagster.ExecutionContext) + check.inst_param(context, 'context', ExecutionContext) check.dict_param(arg_dict, 'arg_dict') path = check.str_elem(arg_dict, 'path') df.to_parquet(path) - return dagster.MaterializationDefinition( + return MaterializationDefinition( name='PARQUET', materialization_fn=to_parquet_fn, - argument_def_dict={'path': dagster.ArgumentDefinition(types.Path)}, + argument_def_dict={'path': ArgumentDefinition(types.Path)}, ) def _create_dataframe_type(): - return dagster.types.PythonObjectType( + return types.PythonObjectType( name='PandasDataFrame', python_type=pd.DataFrame, description= '''Two-dimensional size-mutable, potentially heterogeneous tabular data structure with labeled axes (rows and columns). See http://pandas.pydata.org/ ''', - default_sources=[ - parquet_dataframe_source(), - csv_dataframe_source(), - table_dataframe_source(), - ], default_materializations=[ dataframe_csv_materialization(), dataframe_parquet_materialization(), @@ -111,3 +63,20 @@ def _create_dataframe_type(): DataFrame = _create_dataframe_type() + + +def load_csv_solid(name): + check.str_param(name, 'name') + + def _t_fn(_context, _inputs, config_dict): + yield Result(pd.read_csv(config_dict['path'])) + + return SolidDefinition( + name=name, + inputs=[], + outputs=[OutputDefinition(dagster_type=DataFrame)], + transform_fn=_t_fn, + config_def={ + 'path': ArgumentDefinition(types.Path), + } + ) diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py index fb821b4e1fcf1..d281e0f584262 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py @@ -1,21 +1,27 @@ import pandas as pd -import dagster -from dagster import config -from dagster.core.execution import (execute_single_solid, output_single_solid) +from dagster import ( + DependencyDefinition, + InputDefinition, + OutputDefinition, + PipelineDefinition, + SolidDefinition, + config, + execute_pipeline, +) + from dagster.core.decorators import solid from dagster.utils.test import (script_relative_path, get_temp_file_name) import dagster.pandas_kernel as dagster_pd -def create_num_csv_environment(): +def create_num_csv_environment(materializations=None): return config.Environment( - sources={ - 'hello_world': { - 'num_csv': config.Source('CSV', {'path': script_relative_path('num.csv')}) - } - } + solids={'load_csv': config.Solid({ + 'path': script_relative_path('num.csv') + })}, + materializations=materializations, ) @@ -25,12 +31,22 @@ def test_hello_world_with_dataframe_fns(): def run_hello_world(hello_world): - result = execute_single_solid( - dagster.ExecutionContext(), - hello_world, + assert len(hello_world.inputs) == 1 + + pipeline = PipelineDefinition( + solids=[dagster_pd.load_csv_solid('load_csv'), hello_world], + dependencies={'hello_world': { + 'num_csv': DependencyDefinition('load_csv'), + }} + ) + + pipeline_result = execute_pipeline( + pipeline, environment=create_num_csv_environment(), ) + result = pipeline_result.result_named('hello_world') + assert result.success assert result.transformed_value.to_dict('list') == { @@ -40,14 +56,21 @@ def run_hello_world(hello_world): } with get_temp_file_name() as temp_file_name: - output_result = output_single_solid( - dagster.ExecutionContext(), - hello_world, - environment=create_num_csv_environment(), - name='CSV', - arg_dict={'path': temp_file_name}, + pipeline_result = execute_pipeline( + pipeline, + environment=create_num_csv_environment( + materializations=[ + config.Materialization( + solid='hello_world', + name='CSV', + args={'path': temp_file_name}, + ) + ] + ) ) + output_result = pipeline_result.result_named('hello_world') + assert output_result.success assert pd.read_csv(temp_file_name).to_dict('list') == { @@ -58,7 +81,7 @@ def run_hello_world(hello_world): def create_definition_based_solid(): - table_input = dagster.InputDefinition('num_csv', dagster_pd.DataFrame) + table_input = InputDefinition('num_csv', dagster_pd.DataFrame) def transform_fn(_context, args): num_csv = args['num_csv'] @@ -66,19 +89,19 @@ def transform_fn(_context, args): return num_csv # supports CSV and PARQUET by default - hello_world = dagster.SolidDefinition.single_output_transform( + hello_world = SolidDefinition.single_output_transform( name='hello_world', inputs=[table_input], transform_fn=transform_fn, - output=dagster.OutputDefinition(dagster_type=dagster_pd.DataFrame) + output=OutputDefinition(dagster_type=dagster_pd.DataFrame) ) return hello_world def create_decorator_based_solid(): @solid( - inputs=[dagster.InputDefinition('num_csv', dagster_pd.DataFrame)], - output=dagster.OutputDefinition(dagster_type=dagster_pd.DataFrame), + inputs=[InputDefinition('num_csv', dagster_pd.DataFrame)], + output=OutputDefinition(dagster_type=dagster_pd.DataFrame), ) def hello_world(num_csv): num_csv['sum'] = num_csv['num1'] + num_csv['num2'] @@ -90,16 +113,3 @@ def hello_world(num_csv): def test_hello_world_decorator_style(): hello_world = create_decorator_based_solid() run_hello_world(hello_world) - result = execute_single_solid( - dagster.ExecutionContext(), - hello_world, - environment=create_num_csv_environment(), - ) - - assert result.success - - assert result.transformed_value.to_dict('list') == { - 'num1': [1, 3], - 'num2': [2, 4], - 'sum': [3, 7], - } diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py index 9eeeae4886546..f0455645ea5d4 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py @@ -1,41 +1,42 @@ import pandas as pd -import dagster -from dagster import config -from dagster.core import types -from dagster.core.definitions import ( +from dagster import ( ArgumentDefinition, DependencyDefinition, InputDefinition, MaterializationDefinition, OutputDefinition, + PipelineDefinition, + Result, SolidDefinition, - SourceDefinition, + config, + execute_pipeline, + types, ) -from dagster.core.execution import (ExecutionContext, execute_single_solid) + from dagster.utils.test import script_relative_path -def create_test_context(): - return ExecutionContext() +def define_read_csv_solid(name): + def _t_fn(_context, _inputs, config_dict): + yield Result(pd.read_csv(config_dict['path'])) + return SolidDefinition( + name=name, + inputs=[], + outputs=[OutputDefinition()], + config_def={'path': ArgumentDefinition(types.Path)}, + transform_fn=_t_fn + ) -def create_hello_world_solid_no_api(): + +def test_hello_world_pipeline_no_api(): def hello_world_transform_fn(_context, args): num_df = args['num_df'] num_df['sum'] = num_df['num1'] + num_df['num2'] return num_df - csv_input = InputDefinition( - name='num_df', - sources=[ - SourceDefinition( - source_type='CSV', - argument_def_dict={'path': ArgumentDefinition(types.Path)}, - source_fn=lambda context, arg_dict: pd.read_csv(arg_dict['path']), - ), - ], - ) + read_csv_solid = define_read_csv_solid('read_csv_solid') csv_materialization = MaterializationDefinition( name='CSV', @@ -45,29 +46,32 @@ def hello_world_transform_fn(_context, args): hello_world = SolidDefinition.single_output_transform( name='hello_world', - inputs=[csv_input], + inputs=[InputDefinition('num_df')], transform_fn=hello_world_transform_fn, output=OutputDefinition(materializations=[csv_materialization]) ) - return hello_world - -def test_hello_world_no_library_support(): - hello_world = create_hello_world_solid_no_api() + pipeline = PipelineDefinition( + solids=[read_csv_solid, hello_world], + dependencies={'hello_world': { + 'num_df': DependencyDefinition('read_csv_solid') + }} + ) - result = execute_single_solid( - create_test_context(), - hello_world, - environment=config.Environment( - sources={ - 'hello_world': { - 'num_df': config.Source('CSV', {'path': script_relative_path('num.csv')}) - } - } + pipeline_result = execute_pipeline( + pipeline, + config.Environment( + solids={ + 'read_csv_solid': config.Solid({ + 'path': script_relative_path('num.csv'), + }), + }, ), ) - assert result.success + assert pipeline_result.success + + result = pipeline_result.result_named('hello_world') assert result.transformed_value.to_dict('list') == { 'num1': [1, 3], @@ -76,19 +80,6 @@ def test_hello_world_no_library_support(): } -def create_dataframe_input(name): - return InputDefinition( - name=name, - sources=[ - SourceDefinition( - source_type='CSV', - argument_def_dict={'path': ArgumentDefinition(types.Path)}, - source_fn=lambda context, arg_dict: pd.read_csv(arg_dict['path']), - ), - ], - ) - - def create_dataframe_output(): def mat_fn(_context, arg_dict, df): df.to_csv(arg_dict['path'], index=False) @@ -104,7 +95,7 @@ def mat_fn(_context, arg_dict, df): ) -def create_hello_world_solid_composed_api(): +def create_hello_world_solid_composed_pipeline(): def transform_fn(_context, args): num_df = args['num_df'] num_df['sum'] = num_df['num1'] + num_df['num2'] @@ -112,28 +103,37 @@ def transform_fn(_context, args): hello_world = SolidDefinition.single_output_transform( name='hello_world', - inputs=[create_dataframe_input(name='num_df')], + inputs=[InputDefinition('num_df')], transform_fn=transform_fn, - output=create_dataframe_output(), + output=OutputDefinition(), + ) + + return PipelineDefinition( + solids=[define_read_csv_solid('read_hello_world'), hello_world], + dependencies={'hello_world': { + 'num_df': DependencyDefinition('read_hello_world') + }} ) - return hello_world def test_hello_world_composed(): - hello_world = create_hello_world_solid_composed_api() + pipeline = create_hello_world_solid_composed_pipeline() - result = execute_single_solid( - create_test_context(), - hello_world, + pipeline_result = execute_pipeline( + pipeline, environment=config.Environment( - sources={ - 'hello_world': { - 'num_df': config.Source('CSV', {'path': script_relative_path('num.csv')}) - } - } + solids={ + 'read_hello_world': config.Solid({ + 'path': script_relative_path('num.csv') + }), + }, ), ) + assert pipeline_result.success + + result = pipeline_result.result_named('hello_world') + assert result.success assert result.transformed_value.to_dict('list') == { @@ -151,9 +151,9 @@ def solid_one_transform(_context, args): solid_one = SolidDefinition.single_output_transform( name='solid_one', - inputs=[create_dataframe_input(name='num_df')], + inputs=[InputDefinition(name='num_df')], transform_fn=solid_one_transform, - output=create_dataframe_output(), + output=OutputDefinition(), ) def solid_two_transform(_context, args): @@ -163,14 +163,17 @@ def solid_two_transform(_context, args): solid_two = SolidDefinition.single_output_transform( name='solid_two', - inputs=[create_dataframe_input(name='sum_df')], + inputs=[InputDefinition(name='sum_df')], transform_fn=solid_two_transform, output=create_dataframe_output(), ) - pipeline = dagster.PipelineDefinition( - solids=[solid_one, solid_two], + pipeline = PipelineDefinition( + solids=[define_read_csv_solid('read_one'), solid_one, solid_two], dependencies={ + 'solid_one': { + 'num_df': DependencyDefinition('read_one'), + }, 'solid_two': { 'sum_df': DependencyDefinition('solid_one'), }, @@ -178,14 +181,14 @@ def solid_two_transform(_context, args): ) environment = config.Environment( - sources={ - 'solid_one': { - 'num_df': config.Source('CSV', {'path': script_relative_path('num.csv')}) - } + solids={ + 'read_one': config.Solid({ + 'path': script_relative_path('num.csv') + }), } ) - execute_pipeline_result = dagster.execute_pipeline( + execute_pipeline_result = execute_pipeline( pipeline, environment=environment, ) @@ -199,10 +202,10 @@ def solid_two_transform(_context, args): sum_sq_path_args = {'path': '/tmp/sum_sq.csv'} environment_two = config.Environment( - sources={ - 'solid_one': { - 'num_df': config.Source('CSV', {'path': script_relative_path('num.csv')}) - } + solids={ + 'read_one': config.Solid({ + 'path': script_relative_path('num.csv') + }), }, materializations=[ config.Materialization( @@ -213,7 +216,7 @@ def solid_two_transform(_context, args): ] ) - dagster.execute_pipeline(pipeline, environment=environment_two) + execute_pipeline(pipeline, environment=environment_two) sum_sq_df = pd.read_csv('/tmp/sum_sq.csv') diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py index baca65ac83fbb..6553818803b24 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py @@ -17,7 +17,6 @@ from dagster.core import types from dagster.core.decorators import solid from dagster.core.execution import ( - InMemoryEnv, ExecutionContext, execute_pipeline_iterator, output_single_solid, @@ -38,22 +37,33 @@ def _dataframe_solid(name, inputs, transform_fn): ) -def get_solid_transformed_value(context, solid_inst, environment): - execution_result = execute_single_solid( - context, - solid_inst, - environment=environment, +def get_solid_transformed_value(_context, solid_inst, environment): + pipeline = PipelineDefinition( + solids=[dagster_pd.load_csv_solid('load_csv'), solid_inst], + dependencies={ + solid_inst.name: { + solid_inst.inputs[0].name: DependencyDefinition('load_csv'), + } + } ) + + pipeline_result = execute_pipeline(pipeline, environment) + + execution_result = pipeline_result.result_named(solid_inst.name) + + # execution_result = execute_single_solid( + # context, + # solid_inst, + # environment=environment, + # ) return execution_result.transformed_value -def get_num_csv_environment(solid_name, materializations=None, through_solids=None): +def get_num_csv_environment(load_csv_solid_name, materializations=None, through_solids=None): return config.Environment( - sources={ - solid_name: { - 'num_csv': config.Source('CSV', args={'path': script_relative_path('num.csv')}) - }, - }, + solids={load_csv_solid_name: config.Solid({ + 'path': script_relative_path('num.csv') + })}, materializations=materializations, execution=config.Execution(through_solids=through_solids), ) @@ -63,7 +73,7 @@ def create_test_context(): return ExecutionContext() -def test_pandas_solid(): +def test_basic_pandas_solid(): csv_input = InputDefinition('num_csv', dagster_pd.DataFrame) def transform(_context, args): @@ -93,12 +103,19 @@ def materialization_fn_inst(context, arg_dict, df): output=custom_output_def, ) - output_single_solid( - create_test_context(), - single_solid, - environment=get_num_csv_environment('sum_table'), - name='CUSTOM', - arg_dict={}, + pipeline = PipelineDefinition( + solids=[dagster_pd.load_csv_solid('load_csv'), single_solid], + dependencies={single_solid.name: { + 'num_csv': DependencyDefinition('load_csv'), + }} + ) + + execute_pipeline( + pipeline, + environment=get_num_csv_environment( + 'load_csv', + materializations=[config.Materialization(solid='sum_table', name='CUSTOM', args={})] + ), ) assert test_output['df'].to_dict('list') == {'num1': [1, 3], 'num2': [2, 4], 'sum': [3, 7]} @@ -138,18 +155,34 @@ def materialization_fn_inst(context, arg_dict, df): def execute_transform_in_temp_csv_files(solid_inst): + load_csv_solid = dagster_pd.load_csv_solid('load_csv') + pipeline = PipelineDefinition( + solids=[load_csv_solid, solid_inst], + dependencies={ + solid_inst.name: { + solid_inst.inputs[0].name: DependencyDefinition('load_csv'), + } + } + ) with get_temp_file_name() as temp_file_name: - result = output_single_solid( - create_test_context(), - solid_inst, - environment=get_num_csv_environment(solid_inst.name), - name='CSV', - arg_dict={'path': temp_file_name}, + result = execute_pipeline( + pipeline, + get_num_csv_environment( + 'load_csv', + materializations=[ + config.Materialization( + solid=solid_inst.name, + name='CSV', + args={'path': temp_file_name}, + ) + ] + ), ) assert result.success output_df = pd.read_csv(temp_file_name) + return output_df @@ -187,9 +220,7 @@ def sum_sq_table(sum_df): @solid( - inputs=[ - InputDefinition('sum_table_renamed', dagster_pd.DataFrame) - ], + inputs=[InputDefinition('sum_table_renamed', dagster_pd.DataFrame)], output=OutputDefinition(dagster_type=dagster_pd.DataFrame), ) def sum_sq_table_renamed_input(sum_table_renamed): @@ -205,9 +236,7 @@ def transform(_context, args): return _dataframe_solid( name='mult_table', - inputs=[ - InputDefinition('sum_table', dagster_pd.DataFrame) - ], + inputs=[InputDefinition('sum_table', dagster_pd.DataFrame)], transform_fn=transform ) @@ -224,61 +253,19 @@ def test_pandas_csv_to_csv_decorator_api(): def test_pandas_csv_in_memory(): df = get_solid_transformed_value( - create_test_context(), + None, create_sum_table(), - get_num_csv_environment('sum_table'), + get_num_csv_environment('load_csv'), ) assert isinstance(df, pd.DataFrame) assert df.to_dict('list') == {'num1': [1, 3], 'num2': [2, 4], 'sum': [3, 7]} -@pytest.mark.skip('do not use _pipeline_solid_in_memory') -def test_two_step_pipeline_in_memory(): - sum_table_solid = create_sum_table() - mult_table_solid = create_mult_table(sum_table_solid) - pipeline = PipelineDefinition(solids=[sum_table_solid, mult_table_solid]) - context = create_test_context() - df = get_solid_transformed_value(context, sum_table_solid, get_num_csv_environment('sum_table')) - input_values = {'sum_table': df} - env = InMemoryEnv( - context, - pipeline, - {'mult_table': input_values}, - from_solids=['mult_table'], - through_solids=['mult_table'], - ) - mult_df = _pipeline_solid_in_memory( - context, env, mult_table_solid, input_values - ).transformed_value - assert mult_df.to_dict('list') == { - 'num1': [1, 3], - 'num2': [2, 4], - 'sum': [3, 7], - 'sum_squared': [9, 49] - } - - def _sum_only_pipeline(): - return PipelineDefinition(solids=[sum_table, sum_sq_table], dependencies={},) - - -@pytest.mark.skip('do not use _pipeline_solid_in_memory') -def test_two_step_pipeline_in_memory_decorator_style(): - context = create_test_context() - df = get_solid_transformed_value(context, sum_table, get_num_csv_environment('sum_table')) - input_values = {'sum_df': df} - mult_df = _pipeline_solid_in_memory( - context, - InMemoryEnv(context, _sum_only_pipeline(), {'sum_sq_table': input_values}), - sum_sq_table, - input_values, - ).transformed_value - assert mult_df.to_dict('list') == { - 'num1': [1, 3], - 'num2': [2, 4], - 'sum': [3, 7], - 'sum_squared': [9, 49] - } + return PipelineDefinition( + solids=[sum_table, sum_sq_table], + dependencies={}, + ) def test_two_input_solid(): @@ -300,15 +287,35 @@ def transform(_context, args): ) environment = config.Environment( - sources={ + solids={ + 'load_csv1': config.Solid({ + 'path': script_relative_path('num.csv') + }), + 'load_csv2': config.Solid({ + 'path': script_relative_path('num.csv') + }), + } + ) + + pipeline = PipelineDefinition( + solids=[ + dagster_pd.load_csv_solid('load_csv1'), + dagster_pd.load_csv_solid('load_csv2'), two_input_solid + ], + dependencies={ 'two_input_solid': { - 'num_csv1': config.Source('CSV', {'path': script_relative_path('num.csv')}), - 'num_csv2': config.Source('CSV', {'path': script_relative_path('num.csv')}), - }, - }, + 'num_csv1': DependencyDefinition('load_csv1'), + 'num_csv2': DependencyDefinition('load_csv2'), + } + } ) - df = get_solid_transformed_value(create_test_context(), two_input_solid, environment) + pipeline_result = execute_pipeline(pipeline, environment) + assert pipeline_result.success + + df = pipeline_result.result_named('two_input_solid').transformed_value + + # df = get_solid_transformed_value(create_test_context(), two_input_solid, environment) assert isinstance(df, pd.DataFrame) assert df.to_dict('list') == {'num1': [1, 3], 'num2': [2, 4], 'sum': [3, 7]} @@ -320,29 +327,35 @@ def test_no_transform_solid(): transform_fn=lambda _context, args: args['num_csv'], ) context = create_test_context() - df = get_solid_transformed_value(context, num_table, get_num_csv_environment('num_table')) + df = get_solid_transformed_value(context, num_table, get_num_csv_environment('load_csv')) assert df.to_dict('list') == {'num1': [1, 3], 'num2': [2, 4]} def create_diamond_pipeline(): - return PipelineDefinition(solids=list(create_diamond_dag()), dependencies=create_diamond_deps(),) + return PipelineDefinition(solids=list(create_diamond_dag()), dependencies=create_diamond_deps()) def create_diamond_deps(): return { - 'sum_table' : { - 'num_table' : DependencyDefinition('num_table'), + 'num_table': { + 'num_csv': DependencyDefinition('load_csv'), + }, + 'sum_table': { + 'num_table': DependencyDefinition('num_table'), }, - 'mult_table' : { - 'num_table' : DependencyDefinition('num_table'), + 'mult_table': { + 'num_table': DependencyDefinition('num_table'), }, - 'sum_mult_table' : { - 'sum_table' : DependencyDefinition('sum_table'), - 'mult_table' : DependencyDefinition('mult_table'), + 'sum_mult_table': { + 'sum_table': DependencyDefinition('sum_table'), + 'mult_table': DependencyDefinition('mult_table'), } } + def create_diamond_dag(): + load_csv_solid = dagster_pd.load_csv_solid('load_csv') + num_table_solid = _dataframe_solid( name='num_table', inputs=[InputDefinition('num_csv', dagster_pd.DataFrame)], @@ -357,9 +370,7 @@ def sum_transform(_context, args): sum_table_solid = _dataframe_solid( name='sum_table', - inputs=[ - InputDefinition('num_table', dagster_pd.DataFrame) - ], + inputs=[InputDefinition('num_table', dagster_pd.DataFrame)], transform_fn=sum_transform, ) @@ -371,9 +382,7 @@ def mult_transform(_context, args): mult_table_solid = _dataframe_solid( name='mult_table', - inputs=[ - InputDefinition('num_table', dagster_pd.DataFrame) - ], + inputs=[InputDefinition('num_table', dagster_pd.DataFrame)], transform_fn=mult_transform, ) @@ -394,71 +403,16 @@ def sum_mult_transform(_context, args): transform_fn=sum_mult_transform, ) - return (num_table_solid, sum_table_solid, mult_table_solid, sum_mult_table_solid) - - -@pytest.mark.skip('do not use _pipeline_solid_in_memory for these') -def test_diamond_dag_run(): - solids = create_diamond_dag() - num_table_solid, sum_table_solid, mult_table_solid, sum_mult_table_solid = solids - - pipeline = PipelineDefinition(solids=list(solids), dependencies=create_diamond_deps()) - - context = create_test_context() - - num_table_df = get_solid_transformed_value( - context, - num_table_solid, - get_num_csv_environment('num_table'), + return ( + load_csv_solid, num_table_solid, sum_table_solid, mult_table_solid, sum_mult_table_solid ) - assert num_table_df.to_dict('list') == {'num1': [1, 3], 'num2': [2, 4]} - - input_values = {'num_table': num_table_df} - sum_df = _pipeline_solid_in_memory( - context, - InMemoryEnv( - context, - pipeline, - {'sum_table': input_values}, - from_solids=['sum_table'], - through_solids=['sum_table'], - ), - sum_table_solid, - input_values, - ).transformed_value - - assert sum_df.to_dict('list') == {'num1': [1, 3], 'num2': [2, 4], 'sum': [3, 7]} - - input_values = {'num_table': num_table_df} - mult_df = _pipeline_solid_in_memory( - context, - InMemoryEnv(context, pipeline, input_values), - mult_table_solid, - input_values, - ).transformed_value - - assert mult_df.to_dict('list') == {'num1': [1, 3], 'num2': [2, 4], 'mult': [2, 12]} - - input_values = {'sum_table': sum_df, 'mult_table': mult_df} - - sum_mult_df = _pipeline_solid_in_memory( - context, InMemoryEnv(context, pipeline, input_values), sum_mult_table_solid, input_values - ).transformed_value - - assert sum_mult_df.to_dict('list') == { - 'num1': [1, 3], - 'num2': [2, 4], - 'sum': [3, 7], - 'mult': [2, 12], - 'sum_mult': [6, 84], - } def test_pandas_in_memory_diamond_pipeline(): pipeline = create_diamond_pipeline() result = execute_pipeline( pipeline, - environment=get_num_csv_environment('num_table', through_solids=['sum_mult_table']) + environment=get_num_csv_environment('load_csv', through_solids=['sum_mult_table']) ) assert result.result_named('sum_mult_table').transformed_value.to_dict('list') == { @@ -474,7 +428,7 @@ def test_pandas_output_csv_pipeline(): with get_temp_file_name() as temp_file_name: pipeline = create_diamond_pipeline() environment = get_num_csv_environment( - 'num_table', [ + 'load_csv', [ config.Materialization( solid='sum_mult_table', name='CSV', @@ -512,7 +466,7 @@ def test_pandas_output_intermediate_csv_files(): sum_file, mult_file = temp_tuple # pylint: disable=E0632 environment = get_num_csv_environment( - 'num_table', [ + 'load_csv', [ config.Materialization( solid='sum_table', name='CSV', @@ -528,7 +482,7 @@ def test_pandas_output_intermediate_csv_files(): subgraph_one_result = execute_pipeline(pipeline, environment=environment) - assert len(subgraph_one_result.result_list) == 4 + assert len(subgraph_one_result.result_list) == 5 expected_sum = { 'num1': [1, 3], @@ -549,32 +503,35 @@ def test_pandas_output_intermediate_csv_files(): mult_table_result = subgraph_one_result.result_named('mult_table') assert mult_table_result.transformed_value.to_dict('list') == expected_mult - pipeline_result = execute_pipeline( - pipeline, - environment=config.Environment( - sources={ - 'sum_mult_table': { - 'sum_table': config.Source('CSV', {'path': sum_file}), - 'mult_table': config.Source('CSV', {'path': mult_file}), - }, - }, - execution=config.Execution.single_solid('sum_mult_table'), - ), - ) - - assert pipeline_result.success - - subgraph_two_result_list = pipeline_result.result_list - - assert len(subgraph_two_result_list) == 1 - output_df = subgraph_two_result_list[0].transformed_value - assert output_df.to_dict('list') == { - 'num1': [1, 3], - 'num2': [2, 4], - 'sum': [3, 7], - 'mult': [2, 12], - 'sum_mult': [6, 84], - } + # TODO need better partial execution API + return + + # pipeline_result = execute_pipeline( + # pipeline, + # environment=config.Environment( + # sources={ + # 'sum_mult_table': { + # 'sum_table': config.Source('CSV', {'path': sum_file}), + # 'mult_table': config.Source('CSV', {'path': mult_file}), + # }, + # }, + # execution=config.Execution.single_solid('sum_mult_table'), + # ), + # ) + + # assert pipeline_result.success + + # subgraph_two_result_list = pipeline_result.result_list + + # assert len(subgraph_two_result_list) == 1 + # output_df = subgraph_two_result_list[0].transformed_value + # assert output_df.to_dict('list') == { + # 'num1': [1, 3], + # 'num2': [2, 4], + # 'sum': [3, 7], + # 'mult': [2, 12], + # 'sum_mult': [6, 84], + # } def csv_materialization(solid_name, path): @@ -602,7 +559,7 @@ def test_pandas_output_intermediate_parquet_files(): pipeline_result = execute_pipeline( pipeline, environment=get_num_csv_environment( - 'num_table', [ + 'load_csv', [ parquet_materialization('sum_table', sum_file), parquet_materialization('mult_table', mult_file), ] @@ -621,13 +578,14 @@ def test_pandas_output_intermediate_parquet_files(): def test_pandas_multiple_inputs(): - environment = config.Environment( - sources={ - 'double_sum': { - 'num_csv1': config.Source('CSV', {'path': script_relative_path('num.csv')}), - 'num_csv2': config.Source('CSV', {'path': script_relative_path('num.csv')}), - }, + solids={ + 'load_one': config.Solid({ + 'path': script_relative_path('num.csv') + }), + 'load_two': config.Solid({ + 'path': script_relative_path('num.csv') + }), }, execution=config.Execution(through_solids=['double_sum']), ) @@ -643,13 +601,24 @@ def transform_fn(_context, args): ], transform_fn=transform_fn ) - pipeline = PipelineDefinition(solids=[double_sum]) + + pipeline = PipelineDefinition( + solids=[ + dagster_pd.load_csv_solid('load_one'), + dagster_pd.load_csv_solid('load_two'), double_sum + ], + dependencies={ + 'double_sum': { + 'num_csv1': DependencyDefinition('load_one'), + 'num_csv2': DependencyDefinition('load_two'), + } + }, + ) output_df = execute_pipeline( pipeline, environment=environment, - # solid_name='double_sum', - ).result_list[0].transformed_value + ).result_named('double_sum').transformed_value assert not output_df.empty @@ -668,7 +637,7 @@ def test_pandas_multiple_outputs(): for _result in execute_pipeline_iterator( pipeline=pipeline, environment=get_num_csv_environment( - 'num_table', [ + 'load_csv', [ csv_materialization('sum_mult_table', csv_file), parquet_materialization('sum_mult_table', parquet_file), ] @@ -700,14 +669,17 @@ def test_pandas_multiple_outputs(): def test_rename_input(): result = execute_pipeline( PipelineDefinition( - solids=[sum_table, sum_sq_table_renamed_input], + solids=[dagster_pd.load_csv_solid('load_csv'), sum_table, sum_sq_table_renamed_input], dependencies={ - sum_sq_table_renamed_input.name : { - 'sum_table_renamed' : DependencyDefinition(sum_table.name), + 'sum_table': { + 'num_csv': DependencyDefinition('load_csv'), + }, + sum_sq_table_renamed_input.name: { + 'sum_table_renamed': DependencyDefinition(sum_table.name), }, }, ), - environment=get_num_csv_environment('sum_table'), + environment=get_num_csv_environment('load_csv'), ) assert result.success diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_user_error.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_user_error.py index 1a48c1e30e757..0b966360e6466 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_user_error.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_user_error.py @@ -2,13 +2,24 @@ import pytest +import pandas as pd + import dagster import dagster.pandas_kernel as dagster_pd -from dagster import (config, InputDefinition, OutputDefinition, SolidDefinition) -from dagster.core.decorators import solid, source +from dagster import ( + DependencyDefinition, + InputDefinition, + OutputDefinition, + PipelineDefinition, + SolidDefinition, + config, + execute_pipeline, + solid, +) from dagster.core.errors import DagsterInvariantViolationError from dagster.core.execution import execute_single_solid from dagster.utils.test import script_relative_path +from dagster.core.utility_solids import define_pass_mem_value def _dataframe_solid(name, inputs, transform_fn): @@ -31,65 +42,42 @@ def test_wrong_output_value(): def df_solid(num_csv): return 'not a dataframe' + pass_solid = define_pass_mem_value('pass_solid', pd.DataFrame()) + + pipeline = PipelineDefinition( + solids=[pass_solid, df_solid], + dependencies={'test_wrong_output': { + 'num_csv': DependencyDefinition('pass_solid'), + }} + ) + with pytest.raises(DagsterInvariantViolationError): - execute_single_solid( - dagster.ExecutionContext(), - df_solid, - environment=config.Environment( - sources={ - 'test_wrong_output': { - 'num_csv': config.Source('CSV', {'path': script_relative_path('num.csv')}) - }, - }, - ), + execute_pipeline( + pipeline, + environment=config.Environment(), ) def test_wrong_input_value(): - @source(name="WRONG") - def wrong_source(): - return 'not a dataframe' - @solid( name="test_wrong_input", - inputs=[InputDefinition('foo', dagster_pd.DataFrame, sources=[wrong_source])], + inputs=[InputDefinition('foo', dagster_pd.DataFrame)], output=OutputDefinition(), ) def df_solid(foo): return foo - with pytest.raises(DagsterInvariantViolationError): - execute_single_solid( - dagster.ExecutionContext(), - df_solid, - environment=config.Environment( - sources={'test_wrong_input': { - 'foo': config.Source('WRONG', {}) - }} - ) - ) - - -def test_wrong_input_arg_dict(): - csv_input = InputDefinition('num_csv', dagster_pd.DataFrame) - - def transform_fn(context, args): - return args['num_csv'] + pass_solid = define_pass_mem_value('pass_solid', 'not a dataframe') - df_solid = _dataframe_solid( - name='test_wrong_value', inputs=[csv_input], transform_fn=transform_fn + pipeline = PipelineDefinition( + solids=[pass_solid, df_solid], + dependencies={'test_wrong_input': { + 'foo': DependencyDefinition('pass_solid'), + }} ) with pytest.raises(DagsterInvariantViolationError): - execute_single_solid( - dagster.ExecutionContext(), - df_solid, - environment=config.Environment( - sources={ - 'test_wrong_value': { - 'num_jdkfjskdfjs': - config.Source('CSV', {'path': script_relative_path('num.csv')}) - }, - }, - ), + execute_pipeline( + pipeline, + environment=config.Environment(), ) diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_basic_solid.py b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_basic_solid.py index c967b6d4edffd..6ffaa398e68b3 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_basic_solid.py +++ b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_basic_solid.py @@ -1,18 +1,20 @@ import sqlalchemy as sa -import dagster + from dagster import ( - config, DependencyDefinition, + InputDefinition, + PipelineContextDefinition, PipelineDefinition, -) -from dagster.core.execution import ( - output_single_solid, + check, + config, execute_pipeline, ) +from dagster.core.utility_solids import define_pass_mem_value + from dagster.sqlalchemy_kernel.subquery_builder_experimental import ( create_sql_solid, - create_table_expression_input, + DagsterSqlTableExpression, ) from .math_test_db import in_mem_context @@ -22,7 +24,7 @@ def pipeline_test_def(solids, context, dependencies): solids=solids, context_definitions={ 'default': - dagster.PipelineContextDefinition( + PipelineContextDefinition( argument_def_dict={}, context_fn=lambda _pipeline, _args: context, ), @@ -50,52 +52,68 @@ def create_num_table(engine): def test_sql_sum_solid(): + expr_solid = define_pass_mem_value('expr', DagsterSqlTableExpression('num_table')) + sum_table_solid = create_sum_table_solid() environment = config.Environment( - sources={ - 'sum_table': { - 'num_table': config.Source('TABLENAME', {'table_name': 'num_table'}) - } - }, + materializations=[ + config.Materialization( + solid='sum_table', + name='CREATE', + args={'table_name': 'sum_table'}, + ) + ], ) - context = in_mem_context() - - result = output_single_solid( - context, - sum_table_solid, - environment=environment, - name='CREATE', - arg_dict={'table_name': 'sum_table'}, + pipeline = pipeline_test_def( + solids=[expr_solid, sum_table_solid], + context=in_mem_context(), + dependencies={'sum_table': { + 'num_table': DependencyDefinition('expr') + }}, ) + + pipeline_result = execute_pipeline(pipeline, environment) + assert pipeline_result.success + + result = pipeline_result.result_named(sum_table_solid.name) + assert result.success - results = context.resources.sa.engine.connect().execute('SELECT * FROM sum_table').fetchall() + results = result.context.resources.sa.engine.connect().execute('SELECT * FROM sum_table' + ).fetchall() assert results == [(1, 2, 3), (3, 4, 7)] def create_sum_table_solid(): return create_sql_solid( name='sum_table', - inputs=[create_table_expression_input('num_table')], + inputs=[InputDefinition('num_table')], sql_text='SELECT num1, num2, num1 + num2 as sum FROM {num_table}', ) -def create_sum_sq_pipeline(context): +def create_sum_sq_pipeline(context, expr): + check.inst_param(expr, 'expr', DagsterSqlTableExpression) + + expr_solid = define_pass_mem_value('expr', expr) + sum_solid = create_sum_table_solid() sum_sq_solid = create_sql_solid( name='sum_sq_table', - inputs=[create_table_expression_input(sum_solid.name)], + inputs=[InputDefinition(sum_solid.name)], sql_text='SELECT num1, num2, sum, sum * sum as sum_sq from {sum_table}', ) return pipeline_test_def( - solids=[sum_solid, sum_sq_solid], + solids=[expr_solid, sum_solid, sum_sq_solid], context=context, dependencies={ + sum_solid.name: { + 'num_table': DependencyDefinition('expr'), + }, sum_sq_solid.name: { sum_solid.name: DependencyDefinition(sum_solid.name), }, @@ -104,15 +122,9 @@ def create_sum_sq_pipeline(context): def test_execute_sql_sum_sq_solid(): - pipeline = create_sum_sq_pipeline(in_mem_context()) + pipeline = create_sum_sq_pipeline(in_mem_context(), DagsterSqlTableExpression('num_table')) - environment = config.Environment( - sources={ - 'sum_table': { - 'num_table': config.Source('TABLENAME', {'table_name': 'num_table'}) - } - }, - ) + environment = config.Environment() pipeline_result = execute_pipeline( pipeline, @@ -123,23 +135,18 @@ def test_execute_sql_sum_sq_solid(): result_list = pipeline_result.result_list - sum_table_sql_text = result_list[0].transformed_value.query_text + sum_table_sql_text = result_list[1].transformed_value.query_text assert sum_table_sql_text == 'SELECT num1, num2, num1 + num2 as sum FROM num_table' - sum_sq_table_sql_text = result_list[1].transformed_value.query_text + sum_sq_table_sql_text = result_list[2].transformed_value.query_text assert sum_sq_table_sql_text == 'SELECT num1, num2, sum, sum * sum as sum_sq from ' + \ '(SELECT num1, num2, num1 + num2 as sum FROM num_table)' def test_output_sql_sum_sq_solid(): - pipeline = create_sum_sq_pipeline(in_mem_context()) + pipeline = create_sum_sq_pipeline(in_mem_context(), DagsterSqlTableExpression('num_table')) environment = config.Environment( - sources={ - 'sum_table': { - 'num_table': config.Source('TABLENAME', {'table_name': 'num_table'}) - } - }, materializations=[ config.Materialization( solid='sum_sq_table', @@ -155,7 +162,7 @@ def test_output_sql_sum_sq_solid(): result_list = pipeline_result.result_list - assert len(result_list) == 2 + assert len(result_list) == 3 engine = pipeline_result.context.resources.sa.engine result_list = engine.connect().execute('SELECT * FROM sum_sq_table').fetchall() assert result_list == [(1, 2, 3, 9), (3, 4, 7, 49)] diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py index 8435fa69f5edb..b441e36599096 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py +++ b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py @@ -6,7 +6,6 @@ PipelineDefinition, OutputDefinition, SolidDefinition, - SourceDefinition, check, config, execute_pipeline, @@ -14,11 +13,12 @@ ) from dagster.sqlalchemy_kernel.templated import ( - _create_templated_sql_transform_with_output, _render_template_string, create_templated_sql_transform_solid, ) +from dagster.core.utility_solids import define_pass_value_solid + from .math_test_db import in_mem_context @@ -45,58 +45,9 @@ def pipeline_test_def(solids, context, dependencies=None): ) -def test_single_templated_sql_solid_single_table_raw_api(): - sql = '''CREATE TABLE {{sum_table.name}} - AS SELECT num1, num2, num1 + num2 as sum FROM num_table''' - - sum_table_arg = 'specific_sum_table' - - sum_table_input = InputDefinition( - name='sum_table', - sources=[ - SourceDefinition( - source_type='TABLENAME', - source_fn=lambda context, arg_dict: arg_dict, - argument_def_dict={ - 'name': ArgumentDefinition(types.String), - }, - ) - ] - ) - - sum_table_transform_solid = SolidDefinition.single_output_transform( - name='sum_table_transform', - inputs=[sum_table_input], - transform_fn=_create_templated_sql_transform_with_output(sql, 'sum_table'), - output=OutputDefinition(), - ) - - pipeline = pipeline_test_def(solids=[sum_table_transform_solid], context=in_mem_context()) - environment = config.Environment( - sources={'sum_table_transform': { - 'sum_table': table_name_source(sum_table_arg) - }}, - ) - - result = execute_pipeline(pipeline, environment=environment) - assert result.success - - assert _load_table(result.context, sum_table_arg) == [(1, 2, 3), (3, 4, 7)] - - environment_without_source = config.Environment( - sources={'sum_table_transform': { - 'sum_table': table_name_source('another_table') - }}, - ) - result_no_source = execute_pipeline(pipeline, environment=environment_without_source) - assert result_no_source.success - - assert _load_table(result_no_source.context, 'another_table') == [(1, 2, 3), (3, 4, 7)] - - def test_single_templated_sql_solid_single_table_with_api(): - sql = '''CREATE TABLE {{sum_table.name}} AS + sql = '''CREATE TABLE {{sum_table}} AS SELECT num1, num2, num1 + num2 as sum FROM num_table''' sum_table_arg = 'specific_sum_table' @@ -105,15 +56,14 @@ def test_single_templated_sql_solid_single_table_with_api(): name='sum_table_transform', sql=sql, table_arguments=['sum_table'], - output='sum_table', ) pipeline = pipeline_test_def(solids=[sum_table_transform], context=in_mem_context()) environment = config.Environment( - sources={'sum_table_transform': { - 'sum_table': table_name_source(sum_table_arg) - }} + solids={'sum_table_transform': config.Solid({ + 'sum_table': sum_table_arg + })} ) result = execute_pipeline(pipeline, environment=environment) @@ -126,54 +76,24 @@ def test_single_templated_sql_solid_double_table_raw_api(): sum_table_arg = 'specific_sum_table' num_table_arg = 'specific_num_table' - sql = '''CREATE TABLE {{sum_table.name}} AS - SELECT num1, num2, num1 + num2 as sum FROM {{num_table.name}}''' + sql = '''CREATE TABLE {{sum_table}} AS + SELECT num1, num2, num1 + num2 as sum FROM {{num_table}}''' - sum_table_input = InputDefinition( - name='sum_table', - sources=[ - SourceDefinition( - source_type='TABLENAME', - source_fn=lambda context, arg_dict: arg_dict, - argument_def_dict={ - 'name': ArgumentDefinition(types.String), - }, - ) - ] - ) - - num_table_input = InputDefinition( - name='num_table', - sources=[ - SourceDefinition( - source_type='TABLENAME', - source_fn=lambda context, arg_dict: arg_dict, - argument_def_dict={ - 'name': ArgumentDefinition(types.String), - }, - ) - ] - ) - - sum_solid = SolidDefinition.single_output_transform( + sum_solid = create_templated_sql_transform_solid( name='sum_solid', - inputs=[sum_table_input, num_table_input], - transform_fn=_create_templated_sql_transform_with_output( - sql, - 'sum_table', - ), - output=OutputDefinition(), + sql=sql, + table_arguments=['sum_table', 'num_table'], ) pipeline = pipeline_test_def(solids=[sum_solid], context=in_mem_context(num_table_arg)) environment = config.Environment( - sources={ - 'sum_solid': { - 'sum_table': table_name_source(sum_table_arg), - 'num_table': table_name_source(num_table_arg), - }, - }, + solids={ + 'sum_solid': config.Solid({ + 'sum_table': sum_table_arg, + 'num_table': num_table_arg, + }) + } ) result = execute_pipeline(pipeline, environment=environment) @@ -186,25 +106,24 @@ def test_single_templated_sql_solid_double_table_with_api(): sum_table_arg = 'specific_sum_table' num_table_arg = 'specific_num_table' - sql = '''CREATE TABLE {{sum_table.name}} AS - SELECT num1, num2, num1 + num2 as sum FROM {{num_table.name}}''' + sql = '''CREATE TABLE {{sum_table}} AS + SELECT num1, num2, num1 + num2 as sum FROM {{num_table}}''' sum_solid = create_templated_sql_transform_solid( name='sum_solid', sql=sql, table_arguments=['sum_table', 'num_table'], - output='sum_table', ) pipeline = pipeline_test_def(solids=[sum_solid], context=in_mem_context(num_table_arg)) environment = config.Environment( - sources={ - 'sum_solid': { - 'sum_table': table_name_source(sum_table_arg), - 'num_table': table_name_source(num_table_arg), - }, - }, + solids={ + 'sum_solid': config.Solid({ + 'sum_table': sum_table_arg, + 'num_table': num_table_arg, + }) + } ) result = execute_pipeline(pipeline, environment=environment) @@ -214,25 +133,23 @@ def test_single_templated_sql_solid_double_table_with_api(): def test_templated_sql_solid_pipeline(): - sum_sql_template = '''CREATE TABLE {{sum_table.name}} AS + sum_sql_template = '''CREATE TABLE {{sum_table}} AS SELECT num1, num2, num1 + num2 as sum FROM num_table''' - sum_sq_sql_template = '''CREATE TABLE {{sum_sq_table.name}} AS - SELECT num1, num2, sum, sum * sum as sum_sq FROM {{sum_table.name}}''' + sum_sq_sql_template = '''CREATE TABLE {{sum_sq_table}} AS + SELECT num1, num2, sum, sum * sum as sum_sq FROM {{sum_table}}''' sum_solid = create_templated_sql_transform_solid( name='sum_table', sql=sum_sql_template, table_arguments=['sum_table'], - output='sum_table', ) sum_sq_solid = create_templated_sql_transform_solid( name='sum_sq_table', sql=sum_sq_sql_template, - table_arguments=['sum_sq_table'], - output='sum_sq_table', - table_deps=[sum_solid], + table_arguments=['sum_table', 'sum_sq_table'], + dependant_solids=[sum_solid], ) context = in_mem_context() @@ -250,21 +167,33 @@ def test_templated_sql_solid_pipeline(): first_sum_sq_table = 'first_sum_sq_table' environment_one = config.Environment( - sources={ - 'sum_table': { - 'sum_table': table_name_source(first_sum_table) - }, - 'sum_sq_table': { - 'sum_sq_table': table_name_source(first_sum_sq_table) - }, + solids={ + 'sum_table': + config.Solid({ + 'sum_table': first_sum_table + }), + 'sum_sq_table': + config.Solid({ + 'sum_table': first_sum_table, + 'sum_sq_table': first_sum_sq_table, + }), + # { + # 'sum_table': table_name_source(first_sum_table) + # }, + # 'sum_sq_table': { + # 'sum_sq_table': table_name_source(first_sum_sq_table) + # }, } ) first_result = execute_pipeline(pipeline, environment=environment_one) assert first_result.success assert len(first_result.result_list) == 2 - assert first_result.result_list[0].transformed_value == {'name': first_sum_table} - assert first_result.result_list[1].transformed_value == {'name': first_sum_sq_table} + assert first_result.result_list[0].transformed_value == {'sum_table': first_sum_table} + assert first_result.result_list[1].transformed_value == { + 'sum_table': first_sum_table, + 'sum_sq_table': first_sum_sq_table, + } assert _load_table(first_result.context, first_sum_table) == [(1, 2, 3), (3, 4, 7)] @@ -272,18 +201,32 @@ def test_templated_sql_solid_pipeline(): # now execute subdag - pipeline_two = pipeline_test_def(solids=[sum_solid, sum_sq_solid], context=context) + pipeline_two = pipeline_test_def( + solids=[define_pass_value_solid('pass_value'), sum_sq_solid], + context=context, + dependencies={ + sum_sq_solid.name: { + sum_solid.name: DependencyDefinition('pass_value'), + }, + }, + ) second_sum_sq_table = 'second_sum_sq_table' + # This is a bit awkward ATM environment_two = config.Environment( - sources={ - 'sum_sq_table': { - 'sum_table': table_name_source(first_sum_table), - 'sum_sq_table': table_name_source(second_sum_sq_table), - }, + solids={ + 'pass_value': + config.Solid({ + 'value': 'something' + }), + 'sum_sq_table': + config.Solid({ + 'sum_table': first_sum_table, + 'sum_sq_table': second_sum_sq_table, + }), }, - execution=config.Execution.single_solid('sum_sq_table'), + execution=config.Execution(from_solids=['pass_value'], through_solids=['sum_sq_table']), ) second_result = execute_pipeline( @@ -291,28 +234,27 @@ def test_templated_sql_solid_pipeline(): environment=environment_two, ) assert second_result.success - assert len(second_result.result_list) == 1 + assert len(second_result.result_list) == 2 assert _load_table(second_result.context, second_sum_sq_table) == [(1, 2, 3, 9), (3, 4, 7, 49)] def test_templated_sql_solid_with_api(): - sql_template = '''CREATE TABLE {{sum_table.name}} AS + sql_template = '''CREATE TABLE {{sum_table}} AS SELECT num1, num2, num1 + num2 as sum FROM num_table''' sum_solid = create_templated_sql_transform_solid( name='sum_solid', sql=sql_template, table_arguments=['sum_table'], - output='sum_table', ) pipeline = pipeline_test_def(solids=[sum_solid], context=in_mem_context()) sum_table_arg = 'specific_sum_table' environment = config.Environment( - sources={'sum_solid': { - 'sum_table': table_name_source(sum_table_arg) - }} + solids={'sum_solid': config.Solid({ + 'sum_table': sum_table_arg + })}, ) result = execute_pipeline(pipeline, environment=environment) assert result.success @@ -330,16 +272,23 @@ def test_with_from_through_specifying_all_solids(): all_solid_names = [solid.name for solid in pipeline.solids] environment = config.Environment( - sources={ - 'sum_table': { - 'sum_table': table_name_source(first_sum_table), - }, - 'mult_table': { - 'mult_table': table_name_source(first_mult_table), - }, - 'sum_mult_table': { - 'sum_mult_table': table_name_source(first_sum_mult_table), - }, + solids={ + 'sum_table': + config.Solid({ + 'sum_table': first_sum_table, + }), + 'mult_table': + config.Solid({ + 'mult_table': first_mult_table, + }), + 'sum_mult_table': + config.Solid( + { + 'sum_table': first_sum_table, + 'mult_table': first_mult_table, + 'sum_mult_table': first_sum_mult_table, + } + ), }, execution=config.Execution(from_solids=all_solid_names, through_solids=all_solid_names) ) @@ -359,17 +308,24 @@ def test_multi_input_partial_execution(): first_sum_mult_table = 'first_sum_mult_table' environment = config.Environment( - sources={ - 'sum_table': { - 'sum_table': table_name_source(first_sum_table), - }, - 'mult_table': { - 'mult_table': table_name_source(first_mult_table), - }, - 'sum_mult_table': { - 'sum_mult_table': table_name_source(first_sum_mult_table), - }, - } + solids={ + 'sum_table': + config.Solid({ + 'sum_table': first_sum_table + }), + 'mult_table': + config.Solid({ + 'mult_table': first_mult_table, + }), + 'sum_mult_table': + config.Solid( + { + 'sum_table': first_sum_table, + 'mult_table': first_mult_table, + 'sum_mult_table': first_sum_mult_table, + } + ), + }, ) first_pipeline_result = execute_pipeline(pipeline, environment=environment) @@ -381,61 +337,56 @@ def test_multi_input_partial_execution(): assert _load_table(first_pipeline_result.context, first_sum_mult_table) == [(1, 3, 2), (3, 7, 12)] - second_sum_mult_table = 'second_sum_mult_table' + return + # FIXME: need better API for partial pipeline execution - environment_two = config.Environment( - sources={ - 'sum_mult_table': { - 'sum_table': table_name_source(first_sum_table), - 'mult_table': table_name_source(first_mult_table), - 'sum_mult_table': table_name_source(second_sum_mult_table) - }, - }, - execution=config.Execution.single_solid('sum_mult_table') - ) + # second_sum_mult_table = 'second_sum_mult_table' - second_pipeline_result = execute_pipeline( - pipeline, - environment=environment_two, - ) + # environment_two = config.Environment( + # sources={ + # 'sum_mult_table': { + # 'sum_table': table_name_source(first_sum_table), + # 'mult_table': table_name_source(first_mult_table), + # 'sum_mult_table': table_name_source(second_sum_mult_table) + # }, + # }, + # execution=config.Execution.single_solid('sum_mult_table') + # ) - assert second_pipeline_result.success - assert len(second_pipeline_result.result_list) == 1 - assert _load_table(second_pipeline_result.context, - second_sum_mult_table) == [(1, 3, 2), (3, 7, 12)] + # assert second_pipeline_result.success + # assert len(second_pipeline_result.result_list) == 1 + # assert _load_table(second_pipeline_result.context, + # second_sum_mult_table) == [(1, 3, 2), (3, 7, 12)] def create_multi_input_pipeline(): - sum_sql_template = '''CREATE TABLE {{sum_table.name}} AS + sum_sql_template = '''CREATE TABLE {{sum_table}} AS SELECT num1, num2, num1 + num2 as sum FROM num_table''' - mult_sql_template = '''CREATE TABLE {{mult_table.name}} AS + mult_sql_template = '''CREATE TABLE {{mult_table}} AS SELECT num1, num2, num1 * num2 as mult FROM num_table''' - sum_mult_join_template = '''CREATE TABLE {{sum_mult_table.name}} AS - SELECT {{sum_table.name}}.num1, sum, mult FROM {{sum_table.name}} - INNER JOIN {{mult_table.name}} ON {{sum_table.name}}.num1 = {{mult_table.name}}.num1''' + sum_mult_join_template = '''CREATE TABLE {{sum_mult_table}} AS + SELECT {{sum_table}}.num1, sum, mult FROM {{sum_table}} + INNER JOIN {{mult_table}} ON {{sum_table}}.num1 = {{mult_table}}.num1''' sum_solid = create_templated_sql_transform_solid( name='sum_table', sql=sum_sql_template, table_arguments=['sum_table'], - output='sum_table', ) mult_solid = create_templated_sql_transform_solid( name='mult_table', sql=mult_sql_template, table_arguments=['mult_table'], - output='mult_table', ) sum_mult_solid = create_templated_sql_transform_solid( name='sum_mult_table', sql=sum_mult_join_template, - table_arguments=['sum_mult_table'], - table_deps=[sum_solid, mult_solid], - output='sum_mult_table', + table_arguments=['sum_table', 'mult_table', 'sum_mult_table'], + dependant_solids=[sum_solid, mult_solid], ) pipeline = pipeline_test_def( @@ -454,6 +405,6 @@ def create_multi_input_pipeline(): def test_jinja(): templated_sql = '''SELECT * FROM {{some_table.name}}''' - sql = _render_template_string(templated_sql, args={'some_table': {'name': 'fill_me_in'}}) + sql = _render_template_string(templated_sql, config_dict={'some_table': {'name': 'fill_me_in'}}) assert sql == '''SELECT * FROM fill_me_in''' diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/subquery_builder_experimental.py b/python_modules/dagster/dagster/sqlalchemy_kernel/subquery_builder_experimental.py index 416e23391e303..e2a877860b7ba 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/subquery_builder_experimental.py +++ b/python_modules/dagster/dagster/sqlalchemy_kernel/subquery_builder_experimental.py @@ -1,15 +1,12 @@ import os -from dagster import check - -from dagster.core import types - -from dagster.core.definitions import ( - SolidDefinition, - InputDefinition, - SourceDefinition, +from dagster import ( ArgumentDefinition, + InputDefinition, OutputDefinition, + SolidDefinition, + check, + types, ) from dagster.utils.compatability import create_single_materialization_output @@ -77,19 +74,6 @@ def _table_name_read_fn(_context, arg_dict): return DagsterSqlTableExpression(table_name) -def _table_name_source(): - return SourceDefinition( - source_type='TABLENAME', - source_fn=_table_name_read_fn, - argument_def_dict={'table_name': ArgumentDefinition(types.String)}, - ) - - -def create_table_expression_input(name): - check.str_param(name, 'name') - return InputDefinition(name=name, sources=[_table_name_source()]) - - def create_sql_transform(sql_text): def transform_fn(_context, args): sql_texts = {} diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/templated.py b/python_modules/dagster/dagster/sqlalchemy_kernel/templated.py index 02168194833c7..6405433af94c8 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/templated.py +++ b/python_modules/dagster/dagster/sqlalchemy_kernel/templated.py @@ -1,131 +1,49 @@ import jinja2 -import dagster -from dagster import check -from dagster.core.definitions import ( - SolidDefinition, - InputDefinition, - SourceDefinition, +from dagster import ( ArgumentDefinition, + InputDefinition, + OutputDefinition, + Result, + SolidDefinition, + check, + types, ) from .common import execute_sql_text_on_context -def _create_table_input(name): - return InputDefinition( - name=name, - sources=[ - SourceDefinition( - source_type='TABLENAME', - source_fn=lambda context, arg_dict: arg_dict, - argument_def_dict={'name': ArgumentDefinition(dagster.core.types.String)}, - ) - ], - ) - - -def create_templated_sql_transform_solid( - name, - sql, - table_arguments, - output, - table_deps=None, - extra_inputs=None, -): - ''' - Create a solid that is a templated sql statement. This assumes that the sql statement - is creating or modifying a table, and that that table will be used downstream in the pipeline - for further transformations - - Usage example: - - sum_sql_template = 'CREATE TABLE {{sum_table.name}} AS ' + \ - 'SELECT num1, num2, num1 + num2 as sum FROM {{num_table.name}}' - - sum_solid = create_templated_sql_transform_solid( - name='sum_solid', - sql=sum_sql_template, - table_arguments=['sum_table', 'num_table'], - output='sum_table', - ) - - 'Table arguments' are inputs that represent a table. They have a string property "name". Each - table argument ends up as an InputDefinition into the solid. - - To invoke this you would configure an argument dictionary to look like: - - input_arg_dict = {'sum_table': {'name': 'a_sum_table'}, 'num_table': {'name': 'a_num_table'}} - - dagster.execute_pipeline(context, pipeline, input_arg_dict) - - The output table is what is flowed to the *next* solid. - - So imagine, we built another solid which depended on the previous one. - - sum_sq_sql_template = 'CREATE TABLE {{sum_sq_table.name}} AS ' + \ - 'SELECT num1, num2, sum, sum * sum as sum_sq FROM {{sum_solid.name}}' - - sum_sq_solid = create_templated_sql_transform_solid( - name='sum_sq_solid', - sql=sum_sq_sql_template, - table_arguments=['sum_sq_table'], - output='sum_sq_table', - dependencies=[sum_solid], - ) - - This solid has two inputs, one is a table_argument directly passed in, and one is - the table from the previous solid. - - Note: there is current awkwardness in this api because all inputs and solids in pipeline - must have a different name. This is why the template variable in the "sum_sq_solid" is - named "sum_solid" rather than "sum_table". - - To invoke this you would now specifiy: - - input_arg_dict = { - 'sum_table': {'name': 'a_sum_table'}, - 'num_table': {'name': 'a_num_table'}, - 'sum_sq_table': {'name': 'a_sum_sq_table'}, - } - - If you wanted to test the sum_sq_table in isolation, you could skip the num_table and just - specify the output of the sum_solid: - - input_arg_dict = { - 'sum_solid': {'name': 'a_sum_table'}, - 'sum_sq_table': {'name': 'a_sum_sq_table'}, - } - - As noted above the naming is not very intuitive right row, but the functionality works. - ''' +def create_templated_sql_transform_solid(name, sql, table_arguments, dependant_solids=None): check.str_param(name, 'name') check.str_param(sql, 'sql') check.list_param(table_arguments, 'table_arguments', of_type=str) - check.str_param(output, 'output') - table_deps = check.opt_list_param(table_deps, 'table_deps', of_type=SolidDefinition) - extra_inputs = check.opt_list_param(extra_inputs, 'extra_inputs', of_type=InputDefinition) - dep_inputs = [_create_table_input(dep.name) for dep in table_deps] - table_inputs = [_create_table_input(table) for table in table_arguments] + dependant_solids = check.opt_list_param( + dependant_solids, 'dependant_solids', of_type=SolidDefinition + ) + + config_def = {} + for table in table_arguments: + config_def[table] = ArgumentDefinition(types.String) - return SolidDefinition.single_output_transform( + return SolidDefinition( name=name, - inputs=table_inputs + dep_inputs + extra_inputs, - transform_fn=_create_templated_sql_transform_with_output(sql, output), - output=dagster.OutputDefinition(), + inputs=[InputDefinition(solid.name) for solid in dependant_solids], + config_def=config_def, + transform_fn=_create_templated_sql_transform_with_output(sql), + outputs=[OutputDefinition()], ) -def _render_template_string(template_text, args): +def _render_template_string(template_text, config_dict): template = jinja2.Environment(loader=jinja2.BaseLoader).from_string(template_text) - return template.render(**args) + return template.render(**config_dict) -def _create_templated_sql_transform_with_output(sql, output_table): - def do_transform(context, args): - rendered_sql = _render_template_string(sql, args) +def _create_templated_sql_transform_with_output(sql): + def do_transform(context, _inputs, config_dict): + rendered_sql = _render_template_string(sql, config_dict) execute_sql_text_on_context(context, rendered_sql) - return args[output_table] + yield Result(config_dict) return do_transform diff --git a/python_modules/dagster/dagster/utils/compatability.py b/python_modules/dagster/dagster/utils/compatability.py index 269948f19836c..3abb5ab9d478a 100644 --- a/python_modules/dagster/dagster/utils/compatability.py +++ b/python_modules/dagster/dagster/utils/compatability.py @@ -4,40 +4,12 @@ InputDefinition, MaterializationDefinition, OutputDefinition, - SourceDefinition, ) # This is a collection point for older APIs that should no longer be "public" # but still used in unit tests because I don't want to rewrite them all -def create_custom_source_input( - name, - source_fn, - *, - argument_def_dict=None, - expectations=None, - source_type='CUSTOM' -): - ''' - This function exist and is used a lot because separation of inputs and sources used to not - exist so most of the unit tests in the systems were written without tha abstraction. So - this exists as a bridge from the old api to the new api. - ''' - return InputDefinition( - name=name, - sources=[ - SourceDefinition( - source_type=source_type, - source_fn=source_fn, - argument_def_dict=argument_def_dict, - ) - ], - expectations=check.opt_list_param( - expectations, 'expectations', of_type=ExpectationDefinition - ) - ) - def create_single_materialization_output( name, materialization_fn, argument_def_dict, expectations=None ): diff --git a/python_modules/dagster/dagster_tests/test_config.py b/python_modules/dagster/dagster_tests/test_config.py index 0ae7df05f945c..5a68b04921603 100644 --- a/python_modules/dagster/dagster_tests/test_config.py +++ b/python_modules/dagster/dagster_tests/test_config.py @@ -17,37 +17,8 @@ def test_bad_config(): config.Materialization(solid='name', name=1, args={}) -def test_construct_sources_only_environment(): - document = ''' -sources: - solid_one: - input_one: - name: some_source - args: - foo: bar -''' - - environment = config.construct_environment(yaml.load(document)) - - assert environment.sources == { - 'solid_one': { - 'input_one': config.Source( - name='some_source', - args={'foo': 'bar'}, - ) - } - } - - def test_construct_full_environment(): document = ''' -sources: - solid_one: - input_one: - name: some_source - args: - foo: bar - materializations: - solid: solid_one name: mat_name @@ -63,14 +34,6 @@ def test_construct_full_environment(): environment = config.construct_environment(yaml.load(document)) assert environment == config.Environment( - sources={ - 'solid_one': { - 'input_one': config.Source( - name='some_source', - args={'foo': 'bar'}, - ) - } - }, materializations=[ config.Materialization( solid='solid_one', From 5bcde930add7f9668621e4a8a68d558d93536417 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Sun, 19 Aug 2018 09:01:23 -0700 Subject: [PATCH 007/103] Materializations are gone This eliminates materializations as a formalized concept. --- python_modules/dagster/dagster/__init__.py | 2 - .../dagster/dagster/cli/pipeline.py | 11 +- python_modules/dagster/dagster/config.py | 32 +-- .../dagster/dagster/core/compute_nodes.py | 76 +----- .../core/core_tests/test_decorators.py | 138 ---------- .../dagster/dagster/core/decorators.py | 115 +------- .../dagster/dagster/core/definitions.py | 46 ---- .../dagster/dagster/core/execution.py | 87 ------ python_modules/dagster/dagster/core/types.py | 7 - .../pandas_hello_world/env.yml | 10 - .../dagster_examples/sql_hello_world/env.yml | 2 - .../sql_project_example/env.yml | 2 - .../dagster/dagster/pandas_kernel/__init__.py | 64 ++--- .../test_pandas_hello_world_library_slide.py | 54 ++-- ...est_pandas_hello_world_no_library_slide.py | 85 +++--- .../pandas_kernel_tests/test_pandas_solids.py | 253 ++++++++++-------- .../test_basic_solid.py | 62 +++-- .../subquery_builder_experimental.py | 27 +- .../dagster/dagster/utils/compatability.py | 31 --- .../dagster/dagster_tests/test_config.py | 26 -- 20 files changed, 310 insertions(+), 820 deletions(-) delete mode 100644 python_modules/dagster/dagster/utils/compatability.py diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index 730acd55de51b..dc62d34678745 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -10,7 +10,6 @@ ExpectationDefinition, ExpectationResult, InputDefinition, - MaterializationDefinition, OutputDefinition, PipelineContextDefinition, PipelineDefinition, @@ -19,7 +18,6 @@ ) from dagster.core.decorators import ( - materialization, solid, with_context, ) diff --git a/python_modules/dagster/dagster/cli/pipeline.py b/python_modules/dagster/dagster/cli/pipeline.py index 2e91f9e7d34f7..b31229c87d857 100644 --- a/python_modules/dagster/dagster/cli/pipeline.py +++ b/python_modules/dagster/dagster/cli/pipeline.py @@ -143,16 +143,7 @@ def print_solid(printer, solid): printer.line('Outputs:') for output in solid.outputs: - if output.materializations: - printer.line('Materializations:') - for materialization_def in output.materializations: - arg_list = format_argument_dict(materialization_def.argument_def_dict) - with printer.with_indent(): - printer.line( - '{name}({arg_list})'.format( - name=materialization_def.name, arg_list=arg_list - ) - ) + print(output.name) def print_inputs(printer, solid): diff --git a/python_modules/dagster/dagster/config.py b/python_modules/dagster/dagster/config.py index eda5528e2bd08..0c939e4c0149c 100644 --- a/python_modules/dagster/dagster/config.py +++ b/python_modules/dagster/dagster/config.py @@ -4,18 +4,6 @@ # lifted from https://bit.ly/2HcQAuv -class Materialization(namedtuple('MaterializationData', 'solid name args output_name')): - def __new__(cls, solid, name, args): - DEFAULT_OUTPUT = 'result' - return super(Materialization, cls).__new__( - cls, - solid=check.str_param(solid, 'solid'), - name=check.str_param(name, 'name'), - args=check.dict_param(args, 'args', key_type=str), - output_name=DEFAULT_OUTPUT, - ) - - class Context(namedtuple('ContextData', 'name args')): def __new__(cls, name, args): return super(Context, cls).__new__( @@ -43,12 +31,8 @@ def single_solid(solid_name): return Execution(from_solids=[solid_name], through_solids=[solid_name]) -class Environment( - namedtuple('EnvironmentData', 'context solids materializations expectations, execution') -): - def __new__( - cls, *, solids=None, context=None, materializations=None, expectations=None, execution=None - ): +class Environment(namedtuple('EnvironmentData', 'context solids expectations, execution')): + def __new__(cls, *, solids=None, context=None, expectations=None, execution=None): check.opt_inst_param(context, 'context', Context) check.opt_inst_param(execution, 'execution', Execution) @@ -65,9 +49,6 @@ def __new__( cls, context=context, solids=check.opt_dict_param(solids, 'solids', key_type=str, value_type=Solid), - materializations=check.opt_list_param( - materializations, 'materializations', of_type=Materialization - ), expectations=expectations, execution=execution, ) @@ -101,14 +82,6 @@ def _construct_context(yml_config_object): return None -def _construct_materializations(yml_config_object): - materializations = [ - Materialization(solid=m['solid'], name=m['name'], args=m['args']) - for m in check.opt_list_elem(yml_config_object, 'materializations') - ] - return materializations - - def _coerce_to_list(value): if value is None: return None @@ -149,7 +122,6 @@ def construct_environment(yml_config_object): return Environment( solids=_construct_solids(yml_config_object), - materializations=_construct_materializations(yml_config_object), context=_construct_context(yml_config_object), execution=_construct_execution(yml_config_object), ) diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index 85e124063f638..1f3bed4e17397 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -23,12 +23,10 @@ DependencyStructure, ExpectationDefinition, InputDefinition, - MaterializationDefinition, OutputDefinition, PipelineDefinition, Result, SolidDefinition, - SolidInputHandle, SolidOutputHandle, ) @@ -42,10 +40,7 @@ from .graph import create_subgraph -from .types import ( - Any, - DagsterType, -) +from .types import DagsterType def get_single_solid_output(solid): check.inst_param(solid, 'solid', SolidDefinition) @@ -156,15 +151,12 @@ class ComputeNodeTag(Enum): INPUT_EXPECTATION = 'INPUT_EXPECTATION' OUTPUT_EXPECTATION = 'OUTPUT_EXPECTATION' JOIN = 'JOIN' - MATERIALIZATION = 'MATERIALIZATION' EXPECTATION_VALUE_OUTPUT = 'expectation_value' JOIN_OUTPUT = 'join_output' -MATERIALIZATION_INPUT = 'mat_input' EXPECTATION_INPUT = 'expectation_input' - def _execute_core_transform(context, solid_transform_fn, values_dict, config_dict): ''' Execute the user-specified transform for the solid. Wrap in an error boundary and do @@ -544,15 +536,6 @@ def create_compute_node_graph_from_env(pipeline, env): compute_nodes.append(solid_transform_cn) - for materialization in env.materializations: - mat_solid = pipeline.solid_named(materialization.solid) - mat_cn = _construct_materialization_cn( - pipeline, - materialization, - cn_output_node_map[mat_solid.output_handle(materialization.output_name)], - ) - compute_nodes.append(mat_cn) - cn_dict = {} for cn in compute_nodes: cn_dict[cn.guid] = cn @@ -626,63 +609,6 @@ def _do_expectation(context, inputs): return _do_expectation -def _construct_materialization_cn(pipeline, materialization, prev_output_handle): - check.inst_param(pipeline, 'pipeline', PipelineDefinition) - check.inst_param(materialization, 'materialization', config.Materialization) - check.inst_param(prev_output_handle, 'prev_output_handle', ComputeNodeOutputHandle) - - solid = pipeline.solid_named(materialization.solid) - output = get_single_solid_output(solid) - mat_def = output.materialization_of_type(materialization.name) - - error_context_str = 'materialization type {mat}'.format(mat=mat_def.name) - - arg_dict = validate_args( - mat_def.argument_def_dict, - materialization.args, - error_context_str, - ) - - def _compute_fn(context, inputs): - mat_def.materialization_fn( - context, - materialization.args, - inputs[MATERIALIZATION_INPUT], - ) - - - return ComputeNode( - friendly_name=f'{solid.name}.materialization.{mat_def.name}', - node_inputs=[ - ComputeNodeInput( - name=MATERIALIZATION_INPUT, - dagster_type=output.dagster_type, - prev_output_handle=prev_output_handle, - ) - ], - node_outputs=[], - arg_dict=arg_dict, - compute_fn=_compute_fn, - tag=ComputeNodeTag.MATERIALIZATION, - solid=solid, - ) - - -def _create_materialization_lambda(mat_def, materialization, output_name): - check.inst_param(mat_def, 'mat_def', MaterializationDefinition) - check.inst_param(materialization, 'materialization', config.Materialization) - check.str_param(output_name, 'output_name') - - return lambda context, inputs: Result( - output_name=output_name, - value=mat_def.materialization_fn( - context, - materialization.args, - inputs[MATERIALIZATION_INPUT], - ), - ) - - def create_compute_node_from_solid_transform(solid, node_inputs, config_args): check.inst_param(solid, 'solid', SolidDefinition) check.list_param(node_inputs, 'node_inputs', of_type=ComputeNodeInput) diff --git a/python_modules/dagster/dagster/core/core_tests/test_decorators.py b/python_modules/dagster/dagster/core/core_tests/test_decorators.py index 6cf31fce7f78a..e75c134250a12 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_decorators.py +++ b/python_modules/dagster/dagster/core/core_tests/test_decorators.py @@ -1,23 +1,19 @@ import pytest import dagster from dagster import ( - ArgumentDefinition, DependencyDefinition, InputDefinition, OutputDefinition, PipelineDefinition, config, execute_pipeline, - types, ) from dagster.core.errors import DagsterInvalidDefinitionError from dagster.core.decorators import ( - materialization, solid, with_context, ) from dagster.core.execution import ( - output_single_solid, execute_single_solid, ExecutionContext, ) @@ -110,49 +106,6 @@ def hello_world(foo_to_foo): assert result.transformed_value['foo'] == 'bar' -def test_materializations(): - test_output = {} - - @materialization(name="CONTEXT", argument_def_dict={'foo': ArgumentDefinition(types.String)}) - @with_context - def materialization_with_context(_context, data, foo): - test_output['test'] = data - - @materialization(name="NO_CONTEXT", argument_def_dict={'foo': ArgumentDefinition(types.String)}) - def materialization_no_context(data, foo): - test_output['test'] = data - - @solid( - output=OutputDefinition( - materializations=[materialization_with_context, materialization_no_context] - ) - ) - def hello(): - return {'foo': 'bar'} - - output_single_solid( - create_test_context(), - hello, - environment=config.Environment(), - name='CONTEXT', - arg_dict={'foo': 'bar'} - ) - - assert test_output['test'] == {'foo': 'bar'} - - test_output = {} - - output_single_solid( - create_test_context(), - hello, - environment=config.Environment(), - name='NO_CONTEXT', - arg_dict={'foo': 'bar'} - ) - - assert test_output['test'] == {'foo': 'bar'} - - def test_solid_definition_errors(): with pytest.raises(DagsterInvalidDefinitionError): @@ -231,91 +184,6 @@ def valid_context_2(_context, foo, bar): pass -def test_materialization_definition_errors(): - with pytest.raises(DagsterInvalidDefinitionError): - - @materialization(argument_def_dict={'foo': ArgumentDefinition(types.String)}) - @with_context - def no_data(context, foo): - pass - - with pytest.raises(DagsterInvalidDefinitionError): - - @materialization(argument_def_dict={'foo': ArgumentDefinition(types.String)}) - @with_context - def vargs(context, data, foo, *args): - pass - - with pytest.raises(DagsterInvalidDefinitionError): - - @materialization(argument_def_dict={'foo': ArgumentDefinition(types.String)}) - def wrong_name(data, bar): - pass - - with pytest.raises(DagsterInvalidDefinitionError): - - @materialization( - argument_def_dict={ - 'foo': ArgumentDefinition(types.String), - 'bar': ArgumentDefinition(types.String) - } - ) - def wrong_name_2(data, foo): - pass - - with pytest.raises(DagsterInvalidDefinitionError): - - @materialization(argument_def_dict={'foo': ArgumentDefinition(types.String)}) - @with_context - def no_context(data, foo): - pass - - with pytest.raises(DagsterInvalidDefinitionError): - - @materialization(argument_def_dict={'foo': ArgumentDefinition(types.String)}) - def yes_context(context, data, foo): - pass - - with pytest.raises(DagsterInvalidDefinitionError): - - @materialization(argument_def_dict={'foo': ArgumentDefinition(types.String)}) - def extras(data, foo, bar): - pass - - @materialization(argument_def_dict={'foo': ArgumentDefinition(types.String)}) - def valid_kwargs(data, **kwargs): - pass - - @materialization( - argument_def_dict={ - 'foo': ArgumentDefinition(types.String), - 'bar': ArgumentDefinition(types.String) - } - ) - def valid(data, foo, bar): - pass - - @materialization( - argument_def_dict={ - 'foo': ArgumentDefinition(types.String), - 'bar': ArgumentDefinition(types.String) - } - ) - @with_context - def valid_rontext(context, data, foo, bar): - pass - - @materialization( - argument_def_dict={ - 'foo': ArgumentDefinition(types.String), - 'bar': ArgumentDefinition(types.String) - } - ) - @with_context - def valid_context_2(_context, _data, foo, bar): - pass - - def test_wrong_argument_to_pipeline(): def non_solid_func(): pass @@ -337,9 +205,3 @@ def solid_desc(): pass assert solid_desc.description == 'foo' - - @materialization(description='baaz') - def materialization_desc(): - pass - - assert materialization_desc.description == 'baaz' diff --git a/python_modules/dagster/dagster/core/decorators.py b/python_modules/dagster/dagster/core/decorators.py index 491dc71f4a226..b10fd46105ca4 100644 --- a/python_modules/dagster/dagster/core/decorators.py +++ b/python_modules/dagster/dagster/core/decorators.py @@ -5,7 +5,6 @@ SolidDefinition, InputDefinition, OutputDefinition, - MaterializationDefinition, DagsterInvalidDefinitionError, ) @@ -14,7 +13,7 @@ def with_context(fn): - """Pass context as a first argument to a transform, source or materialization function. + """Pass context as a first argument to a transform. """ return _WithContext(fn) @@ -79,58 +78,6 @@ def transform(context, args): return transform -class _Materialization: - def __init__(self, name=None, argument_def_dict=None, description=None): - self.name = check.opt_str_param(name, 'name') - self.argument_def_dict = check.opt_dict_param(argument_def_dict, 'argument_def_dict') - self.description = check.opt_str_param(description, 'description') - - def __call__(self, fn): - include_context = getattr(fn, 'has_context', False) - if include_context: - fn = fn.fn - - if not self.name: - self.name = fn.__name__ - - _validate_materialization_fn(fn, self.name, self.argument_def_dict, include_context) - materialization_fn = _create_materialization_wrapper( - fn, self.argument_def_dict, include_context - ) - - return MaterializationDefinition( - name=self.name, - materialization_fn=materialization_fn, - argument_def_dict=self.argument_def_dict, - description=self.description, - ) - - -def materialization(*, name=None, argument_def_dict=None, description=None): - return _Materialization( - name=name, - argument_def_dict=argument_def_dict, - description=description, - ) - - -def _create_materialization_wrapper(fn, arg_def_dict, include_context=False): - arg_names = arg_def_dict.keys() - - @wraps(fn) - def materialization_fn(context, args, data): - kwargs = {} - for arg in arg_names: - kwargs[arg] = args[arg] - - if include_context: - return fn(context, data, **kwargs) - else: - return fn(data, **kwargs) - - return materialization_fn - - class FunctionValidationError(Exception): TYPES = { 'vararg': 1, @@ -176,66 +123,6 @@ def _validate_transform_fn(solid_name, transform_fn, inputs, expect_context=Fals raise e -def _validate_source_fn(fn, source_name, arg_def_dict, expect_context=False): - names = set(arg_def_dict.keys()) - if expect_context: - expected_positionals = ('context', ) - else: - expected_positionals = () - try: - _validate_decorated_fn(fn, names, expected_positionals) - except FunctionValidationError as e: - if e.error_type == FunctionValidationError.TYPES['vararg']: - raise DagsterInvalidDefinitionError( - f"source '{source_name}' source function has positional vararg parameter '{e.param}'. Source functions should only have keyword arguments that match argument definition names and optionally a first positional parameter named 'context'." - ) - elif e.error_type == FunctionValidationError.TYPES['missing_name']: - raise DagsterInvalidDefinitionError( - f"source '{source_name}' source function has parameter '{e.param}' that is not one of the source arguments. Source functions should only have keyword arguments that match argument definition names and optionally a first positional parameter named 'context'." - ) - elif e.error_type == FunctionValidationError.TYPES['missing_positional']: - raise DagsterInvalidDefinitionError( - f"source '{source_name}' transform function do not have required positional argument '{e.param}'. Source functions should only have keyword arguments that match argument definition names and optionally a first positional parameter named 'context" - ) - elif e.error_type == FunctionValidationError.TYPES['extra']: - undeclared_inputs_printed = ", '".join(e.missing_names) - raise DagsterInvalidDefinitionError( - f"source '{source_name}' transform function do not have parameter(s) '{undeclared_inputs_printed}', which are in source's argument definitions. Source functions should only have keyword arguments that match argument definition names and optionally a first positional parameter named 'context'" - ) - else: - raise e - - -def _validate_materialization_fn(fn, materialization_name, arg_def_dict, expect_context=False): - names = set(arg_def_dict.keys()) - if expect_context: - expected_positionals = ('context', 'data') - else: - expected_positionals = ('data', ) - try: - _validate_decorated_fn(fn, names, expected_positionals) - except FunctionValidationError as e: - if e.error_type == FunctionValidationError.TYPES['vararg']: - raise DagsterInvalidDefinitionError( - f"materialization '{materialization_name}' materialization function has positional vararg parameter '{e.param}'. Materialization functions should only have keyword arguments that match argument definition names, positional parameter 'data' and optionally a first positional parameter named 'context'." - ) - elif e.error_type == FunctionValidationError.TYPES['missing_name']: - raise DagsterInvalidDefinitionError( - f"materialization '{materialization_name}' transform function has parameter '{e.param}' that is not one of the materialization's argument definitions. Materialization functions should only have keyword arguments that match argument definition names, positional parameter 'data' and optionally a first positional parameter named 'context'." - ) - elif e.error_type == FunctionValidationError.TYPES['missing_positional']: - raise DagsterInvalidDefinitionError( - f"materialization '{materialization_name}' materialization function do not have parameter {e.param}'. Materialization functions should only have keyword arguments that match argument definition names, positional parameter 'data' and optionally a first positional parameter named 'context'." - ) - elif e.error_type == FunctionValidationError.TYPES['extra']: - undeclared_inputs_printed = ", '".join(e.missing_names) - raise DagsterInvalidDefinitionError( - f"materialization '{materialization_name}' materialization function do not have parameter(s) '{undeclared_inputs_printed}', which are in materialization's argument definitinio. Materialization functions should only have keyword arguments that match argument definition names, positional parameter 'data' and optionally a first positional parameter named 'context'." - ) - else: - raise e - - def _validate_decorated_fn(fn, names, expected_positionals): used_inputs = set() has_kwargs = False diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index 75f8542b3bb21..728aa89001ec3 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -250,44 +250,12 @@ def __init__( self.description = check.opt_str_param(description, 'description') -class MaterializationDefinition: - ''' - materialization_fn: callable - - This function defines the actual output. - - The first function argument is the result of the transform function. It can be - named anything. - - You must specify an argument with the name "arg_dict". It will be the dictionary - of arguments specified by the caller of the solid - - def materialization_fn(context, args, value): - pass - - argument_def_dict: { str: DagsterType } - Define the arguments expected by this materialization. A dictionary that maps a string - (argument name) to an argument type (defined in dagster.core.types). - - e.g.: - - argument_def_dict = { 'path' : dagster.core.types.Path } - ''' - - def __init__(self, name, materialization_fn, argument_def_dict=None, description=None): - self.name = check_valid_name(name) - self.materialization_fn = check.callable_param(materialization_fn, 'materialization_fn') - self.argument_def_dict = check_argument_def_dict(argument_def_dict) - self.description = check.opt_str_param(description, 'description') - - class OutputDefinition: # runtime type info def __init__( self, name=None, dagster_type=None, - materializations=None, expectations=None, output_callback=None, description=None @@ -298,26 +266,12 @@ def __init__( dagster_type, 'dagster_type', types.DagsterType, types.Any ) - if materializations is None and dagster_type is not None: - materializations = dagster_type.default_materializations - - self.materializations = check.opt_list_param( - materializations, 'materializations', of_type=MaterializationDefinition - ) - self.expectations = check.opt_list_param( expectations, 'expectations', of_type=ExpectationDefinition ) self.output_callback = check.opt_callable_param(output_callback, 'output_callback') self.description = check.opt_str_param(description, 'description') - def materialization_of_type(self, name): - for materialization in self.materializations: - if materialization.name == name: - return materialization - - check.failed('Did not find materialization {type}'.format(type=name)) - class SolidInputHandle(namedtuple('_SolidInputHandle', 'solid input_def')): def __new__(cls, solid, input_def): diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index 753ea9237860e..6031af51dcf23 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -9,12 +9,6 @@ the core transform, and exercise all logging and metrics tracking (outside of outputs), but they will not invoke *any* outputs (and their APIs don't allow the user to). -materialize_* - -Materializations functions do execution but also allow the user to specify materializations, -which create artifacts that are discoverable by external systems (e.g. files, database -tables, and so on). - ''' @@ -140,7 +134,6 @@ def execute_single_solid(context, solid, environment, throw_on_error=True): check.invariant(environment.execution.through_solids == []) single_solid_environment = config.Environment( - materializations=environment.materializations, expectations=environment.expectations, context=environment.context, execution=config.Execution.single_solid(solid.name), @@ -169,82 +162,6 @@ def _do_throw_on_error(execution_result): raise execution_result.dagster_user_exception -def output_single_solid( - context, - solid, - environment, - name, - arg_dict, - throw_on_error=True, -): - check.inst_param(context, 'context', ExecutionContext) - check.inst_param(solid, 'solid', SolidDefinition) - check.inst_param(environment, 'environment', config.Environment) - check.str_param(name, 'name') - check.dict_param(arg_dict, 'arg_dict', key_type=str) - check.bool_param(throw_on_error, 'throw_on_error') - - - results = list( - execute_pipeline_iterator( - PipelineDefinition( - solids=[solid], - context_definitions=_create_passthrough_context_definition(context), - ), - environment=config.Environment( - context=environment.context, - materializations=[ - config.Materialization( - solid=solid.name, name=name, args=arg_dict - ) - ], - ), - ) - ) - - for result in results: - if not result.success: - if throw_on_error: - _do_throw_on_error(result) - else: - return result - - for result in results: - if result.name == solid.name and result.tag == ComputeNodeTag.TRANSFORM: - return result - - check.invariant(len(results) == 1, 'must be one result got ' + str(len(results))) - - execution_result = results[0] - - check.invariant(execution_result.name == solid.name) - - if throw_on_error: - _do_throw_on_error(execution_result) - - return execution_result - -class DagsterEnv: - @contextmanager - def yield_context(self): - check.not_implemented('must implement in subclass') - - @property - def materializations(self): - check.not_implemented('must implement in subclass') - - @property - def evaluate_expectations(self): - check.not_implemented('must implement in subclass') - - @property - def from_solids(self): - check.not_implemented('must implement in subclass') - - @property - def through_solids(self): - check.not_implemented('must implement in subclass') - def _wrap_in_yield(thing): if isinstance(thing, ExecutionContext): def _wrap(): @@ -297,10 +214,6 @@ def yield_context(self): thing = context_definition.context_fn(self.pipeline, args_to_pass) return _wrap_in_yield(thing) - @property - def materializations(self): - return self.environment.materializations - @property def evaluate_expectations(self): return self.environment.expectations.evaluate diff --git a/python_modules/dagster/dagster/core/types.py b/python_modules/dagster/dagster/core/types.py index 967b32cdb731a..d790f3dd3f540 100644 --- a/python_modules/dagster/dagster/core/types.py +++ b/python_modules/dagster/dagster/core/types.py @@ -17,7 +17,6 @@ def is_python_valid_value(self, _value): class DagsterScalarType(DagsterType): def __init__(self, *args, **kwargs): super().__init__(*args, **kwargs) - self.default_materializations = [] class _DagsterAnyType(DagsterType): @@ -39,16 +38,10 @@ def __init__( self, name, python_type, - default_materializations=None, description=None, ): super().__init__(name, description) self.python_type = check.type_param(python_type, 'python_type') - self.default_materializations = check.opt_list_param( - default_materializations, - 'default_materializations', - of_type=dagster.MaterializationDefinition - ) def is_python_valid_value(self, value): return _nullable_isinstance(value, self.python_type) diff --git a/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/env.yml b/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/env.yml index d6c14ff14d915..b5cdc38072abf 100644 --- a/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/env.yml +++ b/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/env.yml @@ -7,13 +7,3 @@ solids: load_num_csv: config: path: "pandas_hello_world/num.csv" - -materializations: - - solid: sum_solid - name: CSV - args: - path: "/tmp/sum.csv" - - solid: sum_sq_solid - name: CSV - args: - path: "/tmp/sum_sq.csv" diff --git a/python_modules/dagster/dagster/dagster_examples/sql_hello_world/env.yml b/python_modules/dagster/dagster/dagster_examples/sql_hello_world/env.yml index 6eaa5e5433fc2..80412dc28d1f7 100644 --- a/python_modules/dagster/dagster/dagster_examples/sql_hello_world/env.yml +++ b/python_modules/dagster/dagster/dagster_examples/sql_hello_world/env.yml @@ -1,4 +1,2 @@ environment: sources: {} - -materializations: [] diff --git a/python_modules/dagster/dagster/dagster_examples/sql_project_example/env.yml b/python_modules/dagster/dagster/dagster_examples/sql_project_example/env.yml index e368746effbd7..c8187f2a6ab2c 100644 --- a/python_modules/dagster/dagster/dagster_examples/sql_project_example/env.yml +++ b/python_modules/dagster/dagster/dagster_examples/sql_project_example/env.yml @@ -1,4 +1,2 @@ environment: inputs: [] - -materializations: [] diff --git a/python_modules/dagster/dagster/pandas_kernel/__init__.py b/python_modules/dagster/dagster/pandas_kernel/__init__.py index d9ab61aee50e3..b5bcc000ad44d 100644 --- a/python_modules/dagster/dagster/pandas_kernel/__init__.py +++ b/python_modules/dagster/dagster/pandas_kernel/__init__.py @@ -6,7 +6,7 @@ from dagster import ( ArgumentDefinition, ExecutionContext, - MaterializationDefinition, + InputDefinition, OutputDefinition, Result, SolidDefinition, @@ -15,38 +15,6 @@ ) -def dataframe_csv_materialization(): - def to_csv_fn(context, arg_dict, df): - check.inst_param(df, 'df', pd.DataFrame) - check.inst_param(context, 'context', ExecutionContext) - check.dict_param(arg_dict, 'arg_dict') - path = check.str_elem(arg_dict, 'path') - - df.to_csv(path, index=False) - - return MaterializationDefinition( - name='CSV', - materialization_fn=to_csv_fn, - argument_def_dict={'path': ArgumentDefinition(types.Path)}, - ) - - -def dataframe_parquet_materialization(): - def to_parquet_fn(context, arg_dict, df): - check.inst_param(df, 'df', pd.DataFrame) - check.inst_param(context, 'context', ExecutionContext) - check.dict_param(arg_dict, 'arg_dict') - path = check.str_elem(arg_dict, 'path') - - df.to_parquet(path) - - return MaterializationDefinition( - name='PARQUET', - materialization_fn=to_parquet_fn, - argument_def_dict={'path': ArgumentDefinition(types.Path)}, - ) - - def _create_dataframe_type(): return types.PythonObjectType( name='PandasDataFrame', @@ -55,10 +23,6 @@ def _create_dataframe_type(): '''Two-dimensional size-mutable, potentially heterogeneous tabular data structure with labeled axes (rows and columns). See http://pandas.pydata.org/ ''', - default_materializations=[ - dataframe_csv_materialization(), - dataframe_parquet_materialization(), - ] ) @@ -80,3 +44,29 @@ def _t_fn(_context, _inputs, config_dict): 'path': ArgumentDefinition(types.Path), } ) + + +def to_csv_solid(name): + def _t_fn(_context, inputs, config_dict): + inputs['df'].to_csv(config_dict['path'], index=False) + + return SolidDefinition( + name=name, + inputs=[InputDefinition('df', DataFrame)], + outputs=[], + config_def={'path': ArgumentDefinition(types.Path)}, + transform_fn=_t_fn, + ) + + +def to_parquet_solid(name): + def _t_fn(_context, inputs, config_dict): + inputs['df'].to_parquet(config_dict['path']) + + return SolidDefinition( + name=name, + inputs=[InputDefinition('df', DataFrame)], + outputs=[], + config_def={'path': ArgumentDefinition(types.Path)}, + transform_fn=_t_fn, + ) diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py index d281e0f584262..52710017dc0bc 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py @@ -16,12 +16,11 @@ import dagster.pandas_kernel as dagster_pd -def create_num_csv_environment(materializations=None): +def create_num_csv_environment(): return config.Environment( solids={'load_csv': config.Solid({ 'path': script_relative_path('num.csv') })}, - materializations=materializations, ) @@ -34,10 +33,15 @@ def run_hello_world(hello_world): assert len(hello_world.inputs) == 1 pipeline = PipelineDefinition( - solids=[dagster_pd.load_csv_solid('load_csv'), hello_world], - dependencies={'hello_world': { - 'num_csv': DependencyDefinition('load_csv'), - }} + solids=[ + dagster_pd.load_csv_solid('load_csv'), + hello_world, + ], + dependencies={ + 'hello_world': { + 'num_csv': DependencyDefinition('load_csv'), + }, + } ) pipeline_result = execute_pipeline( @@ -55,18 +59,36 @@ def run_hello_world(hello_world): 'sum': [3, 7], } + pipeline_two = PipelineDefinition( + solids=[ + dagster_pd.load_csv_solid('load_csv'), + hello_world, + dagster_pd.to_csv_solid('to_csv'), + ], + dependencies={ + 'hello_world': { + 'num_csv': DependencyDefinition('load_csv'), + }, + 'to_csv': { + 'df': DependencyDefinition('hello_world'), + } + } + ) + with get_temp_file_name() as temp_file_name: + environment = config.Environment( + solids={ + 'load_csv': config.Solid({ + 'path': script_relative_path('num.csv'), + }), + 'to_csv': config.Solid({ + 'path': temp_file_name, + }) + }, + ) pipeline_result = execute_pipeline( - pipeline, - environment=create_num_csv_environment( - materializations=[ - config.Materialization( - solid='hello_world', - name='CSV', - args={'path': temp_file_name}, - ) - ] - ) + pipeline_two, + environment, ) output_result = pipeline_result.result_named('hello_world') diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py index f0455645ea5d4..ad236bda5354c 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py @@ -4,7 +4,6 @@ ArgumentDefinition, DependencyDefinition, InputDefinition, - MaterializationDefinition, OutputDefinition, PipelineDefinition, Result, @@ -30,6 +29,20 @@ def _t_fn(_context, _inputs, config_dict): ) +def define_to_csv_solid(name): + def _t_fn(_context, inputs, config_dict): + print('WRITING NOW') + inputs['df'].to_csv(config_dict['path'], index=False) + + return SolidDefinition( + name=name, + inputs=[InputDefinition('df')], + outputs=[], + config_def={'path': ArgumentDefinition(types.Path)}, + transform_fn=_t_fn, + ) + + def test_hello_world_pipeline_no_api(): def hello_world_transform_fn(_context, args): num_df = args['num_df'] @@ -38,24 +51,20 @@ def hello_world_transform_fn(_context, args): read_csv_solid = define_read_csv_solid('read_csv_solid') - csv_materialization = MaterializationDefinition( - name='CSV', - materialization_fn=lambda df, arg_dict: df.to_csv(arg_dict['path'], index=False), - argument_def_dict={'path': ArgumentDefinition(types.Path)} - ) - hello_world = SolidDefinition.single_output_transform( name='hello_world', inputs=[InputDefinition('num_df')], transform_fn=hello_world_transform_fn, - output=OutputDefinition(materializations=[csv_materialization]) + output=OutputDefinition(), ) pipeline = PipelineDefinition( solids=[read_csv_solid, hello_world], - dependencies={'hello_world': { - 'num_df': DependencyDefinition('read_csv_solid') - }} + dependencies={ + 'hello_world': { + 'num_df': DependencyDefinition('read_csv_solid'), + }, + } ) pipeline_result = execute_pipeline( @@ -80,21 +89,6 @@ def hello_world_transform_fn(_context, args): } -def create_dataframe_output(): - def mat_fn(_context, arg_dict, df): - df.to_csv(arg_dict['path'], index=False) - - return OutputDefinition( - materializations=[ - MaterializationDefinition( - name='CSV', - materialization_fn=mat_fn, - argument_def_dict={'path': ArgumentDefinition(types.Path)}, - ), - ], - ) - - def create_hello_world_solid_composed_pipeline(): def transform_fn(_context, args): num_df = args['num_df'] @@ -143,7 +137,7 @@ def test_hello_world_composed(): } -def test_pipeline(): +def test_pandas_hello_no_library(): def solid_one_transform(_context, args): num_df = args['num_df'] num_df['sum'] = num_df['num1'] + num_df['num2'] @@ -165,7 +159,7 @@ def solid_two_transform(_context, args): name='solid_two', inputs=[InputDefinition(name='sum_df')], transform_fn=solid_two_transform, - output=create_dataframe_output(), + output=OutputDefinition(), ) pipeline = PipelineDefinition( @@ -200,23 +194,42 @@ def solid_two_transform(_context, args): 'sum_sq': [9, 49], } + sum_sq_out_path = '/tmp/sum_sq.csv' + import os + if os.path.exists(sum_sq_out_path): + os.remove(sum_sq_out_path) + sum_sq_path_args = {'path': '/tmp/sum_sq.csv'} environment_two = config.Environment( solids={ 'read_one': config.Solid({ 'path': script_relative_path('num.csv') }), + 'write_two': config.Solid(sum_sq_path_args), }, - materializations=[ - config.Materialization( - solid='solid_two', - name='CSV', - args=sum_sq_path_args, - ), - ] ) - execute_pipeline(pipeline, environment=environment_two) + pipeline_two = PipelineDefinition( + solids=[ + define_read_csv_solid('read_one'), + solid_one, + solid_two, + define_to_csv_solid('write_two'), + ], + dependencies={ + 'solid_one': { + 'num_df': DependencyDefinition('read_one'), + }, + 'solid_two': { + 'sum_df': DependencyDefinition('solid_one'), + }, + 'write_two': { + 'df': DependencyDefinition('solid_two'), + } + } + ) + + execute_pipeline(pipeline_two, environment=environment_two) sum_sq_df = pd.read_csv('/tmp/sum_sq.csv') diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py index 6553818803b24..24440cfd21b3e 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py @@ -19,12 +19,10 @@ from dagster.core.execution import ( ExecutionContext, execute_pipeline_iterator, - output_single_solid, execute_pipeline, execute_single_solid, ) import dagster.pandas_kernel as dagster_pd -from dagster.utils.compatability import create_single_materialization_output from dagster.utils.test import (get_temp_file_name, get_temp_file_names, script_relative_path) @@ -51,20 +49,16 @@ def get_solid_transformed_value(_context, solid_inst, environment): execution_result = pipeline_result.result_named(solid_inst.name) - # execution_result = execute_single_solid( - # context, - # solid_inst, - # environment=environment, - # ) return execution_result.transformed_value -def get_num_csv_environment(load_csv_solid_name, materializations=None, through_solids=None): +def get_load_only_solids_config(load_csv_solid_name): + return {load_csv_solid_name: config.Solid({'path': script_relative_path('num.csv')})} + + +def get_num_csv_environment(solids_config, through_solids=None): return config.Environment( - solids={load_csv_solid_name: config.Solid({ - 'path': script_relative_path('num.csv') - })}, - materializations=materializations, + solids=solids_config, execution=config.Execution(through_solids=through_solids), ) @@ -83,24 +77,11 @@ def transform(_context, args): test_output = {} - def materialization_fn_inst(context, arg_dict, df): - assert isinstance(df, pd.DataFrame) - assert isinstance(context, ExecutionContext) - assert isinstance(arg_dict, dict) - - test_output['df'] = df - - custom_output_def = create_single_materialization_output( - name='CUSTOM', - materialization_fn=materialization_fn_inst, - argument_def_dict={}, - ) - single_solid = SolidDefinition.single_output_transform( name='sum_table', inputs=[csv_input], transform_fn=transform, - output=custom_output_def, + output=OutputDefinition(), ) pipeline = PipelineDefinition( @@ -110,15 +91,18 @@ def materialization_fn_inst(context, arg_dict, df): }} ) - execute_pipeline( + pipeline_result = execute_pipeline( pipeline, - environment=get_num_csv_environment( - 'load_csv', - materializations=[config.Materialization(solid='sum_table', name='CUSTOM', args={})] - ), + environment=get_num_csv_environment(get_load_only_solids_config('load_csv')), ) - assert test_output['df'].to_dict('list') == {'num1': [1, 3], 'num2': [2, 4], 'sum': [3, 7]} + assert pipeline_result.success + + assert pipeline_result.result_named('sum_table').transformed_value.to_dict('list') == { + 'num1': [1, 3], + 'num2': [2, 4], + 'sum': [3, 7] + } def test_pandas_csv_to_csv(): @@ -131,22 +115,11 @@ def transform(context, args): num_csv['sum'] = num_csv['num1'] + num_csv['num2'] return num_csv - def materialization_fn_inst(context, arg_dict, df): - assert isinstance(context, ExecutionContext) - path = check.str_elem(arg_dict, 'path') - df.to_csv(path, index=False) - - csv_output_def = create_single_materialization_output( - name='CSV', - materialization_fn=materialization_fn_inst, - argument_def_dict={'path': ArgumentDefinition(types.Path)} - ) - solid_def = SolidDefinition.single_output_transform( name='sum_table', inputs=[csv_input], transform_fn=transform, - output=csv_output_def, + output=OutputDefinition(dagster_type=dagster_pd.DataFrame), ) output_df = execute_transform_in_temp_csv_files(solid_def) @@ -156,11 +129,16 @@ def materialization_fn_inst(context, arg_dict, df): def execute_transform_in_temp_csv_files(solid_inst): load_csv_solid = dagster_pd.load_csv_solid('load_csv') + to_csv_solid = dagster_pd.to_csv_solid('to_csv') + pipeline = PipelineDefinition( - solids=[load_csv_solid, solid_inst], + solids=[load_csv_solid, solid_inst, to_csv_solid], dependencies={ solid_inst.name: { solid_inst.inputs[0].name: DependencyDefinition('load_csv'), + }, + 'to_csv': { + 'df': DependencyDefinition(solid_inst.name), } } ) @@ -168,14 +146,14 @@ def execute_transform_in_temp_csv_files(solid_inst): result = execute_pipeline( pipeline, get_num_csv_environment( - 'load_csv', - materializations=[ - config.Materialization( - solid=solid_inst.name, - name='CSV', - args={'path': temp_file_name}, - ) - ] + { + load_csv_solid.name: config.Solid({ + 'path': script_relative_path('num.csv') + }), + to_csv_solid.name: config.Solid({ + 'path': temp_file_name + }), + } ), ) @@ -255,7 +233,7 @@ def test_pandas_csv_in_memory(): df = get_solid_transformed_value( None, create_sum_table(), - get_num_csv_environment('load_csv'), + get_num_csv_environment(get_load_only_solids_config('load_csv')), ) assert isinstance(df, pd.DataFrame) assert df.to_dict('list') == {'num1': [1, 3], 'num2': [2, 4], 'sum': [3, 7]} @@ -327,12 +305,16 @@ def test_no_transform_solid(): transform_fn=lambda _context, args: args['num_csv'], ) context = create_test_context() - df = get_solid_transformed_value(context, num_table, get_num_csv_environment('load_csv')) + df = get_solid_transformed_value( + context, num_table, get_num_csv_environment(get_load_only_solids_config('load_csv')) + ) assert df.to_dict('list') == {'num1': [1, 3], 'num2': [2, 4]} -def create_diamond_pipeline(): - return PipelineDefinition(solids=list(create_diamond_dag()), dependencies=create_diamond_deps()) +def create_diamond_pipeline(extra_solids=None, extra_dependencies=None): + all_solids = list(create_diamond_dag()) + (extra_solids if extra_solids else []) + all_deps = {**create_diamond_deps(), **(extra_dependencies if extra_dependencies else {})} + return PipelineDefinition(solids=all_solids, dependencies=all_deps) def create_diamond_deps(): @@ -412,7 +394,9 @@ def test_pandas_in_memory_diamond_pipeline(): pipeline = create_diamond_pipeline() result = execute_pipeline( pipeline, - environment=get_num_csv_environment('load_csv', through_solids=['sum_mult_table']) + environment=get_num_csv_environment( + get_load_only_solids_config('load_csv'), through_solids=['sum_mult_table'] + ) ) assert result.result_named('sum_mult_table').transformed_value.to_dict('list') == { @@ -426,15 +410,22 @@ def test_pandas_in_memory_diamond_pipeline(): def test_pandas_output_csv_pipeline(): with get_temp_file_name() as temp_file_name: - pipeline = create_diamond_pipeline() + write_solid = dagster_pd.to_csv_solid('write_sum_mult_table') + pipeline = create_diamond_pipeline( + extra_solids=[write_solid], + extra_dependencies={write_solid.name: { + 'df': DependencyDefinition('sum_mult_table') + }} + ) environment = get_num_csv_environment( - 'load_csv', [ - config.Materialization( - solid='sum_mult_table', - name='CSV', - args={'path': temp_file_name}, - ) - ] + { + 'load_csv': config.Solid({ + 'path': script_relative_path('num.csv'), + }), + write_solid.name: config.Solid({ + 'path': temp_file_name + }), + } ) for _result in execute_pipeline_iterator(pipeline=pipeline, environment=environment): @@ -460,24 +451,37 @@ def _result_named(results, name): def test_pandas_output_intermediate_csv_files(): - pipeline = create_diamond_pipeline() with get_temp_file_names(2) as temp_tuple: sum_file, mult_file = temp_tuple # pylint: disable=E0632 + write_sum_table = dagster_pd.to_csv_solid('write_sum_table') + write_mult_table = dagster_pd.to_csv_solid('write_mult_table') + + pipeline = create_diamond_pipeline( + extra_solids=[write_sum_table, write_mult_table], + extra_dependencies={ + write_sum_table.name: { + 'df': DependencyDefinition('sum_table'), + }, + write_mult_table.name:{ + 'df': DependencyDefinition('mult_table'), + } + } + ) + environment = get_num_csv_environment( - 'load_csv', [ - config.Materialization( - solid='sum_table', - name='CSV', - args={'path': sum_file}, - ), - config.Materialization( - solid='mult_table', - name='CSV', - args={'path': mult_file}, - ), - ] + { + 'load_csv': config.Solid({ + 'path': script_relative_path('num.csv'), + }), + write_sum_table.name: config.Solid({ + 'path': sum_file + }), + write_mult_table.name: config.Solid({ + 'path': mult_file + }), + } ) subgraph_one_result = execute_pipeline(pipeline, environment=environment) @@ -534,36 +538,43 @@ def test_pandas_output_intermediate_csv_files(): # } -def csv_materialization(solid_name, path): - return config.Materialization( - solid=solid_name, - name='CSV', - args={'path': path}, - ) - - -def parquet_materialization(solid_name, path): - return config.Materialization( - solid=solid_name, - name='PARQUET', - args={'path': path}, - ) - - def test_pandas_output_intermediate_parquet_files(): pipeline = create_diamond_pipeline() with get_temp_file_names(2) as temp_tuple: # false positive on pylint error sum_file, mult_file = temp_tuple # pylint: disable=E0632 + + write_sum_table = dagster_pd.to_parquet_solid('write_sum_table') + write_mult_table = dagster_pd.to_parquet_solid('write_mult_table') + + pipeline = create_diamond_pipeline( + extra_solids=[write_sum_table, write_mult_table], + extra_dependencies={ + write_sum_table.name: { + 'df': DependencyDefinition('sum_table'), + }, + write_mult_table.name:{ + 'df': DependencyDefinition('mult_table'), + } + } + ) + + environment = get_num_csv_environment({ + 'load_csv': config.Solid({ + 'path': script_relative_path('num.csv'), + }), + write_sum_table.name: config.Solid({ + 'path': sum_file + }), + write_mult_table.name: config.Solid({ + 'path': mult_file + }), + }) + pipeline_result = execute_pipeline( pipeline, - environment=get_num_csv_environment( - 'load_csv', [ - parquet_materialization('sum_table', sum_file), - parquet_materialization('mult_table', mult_file), - ] - ), + environment, ) assert pipeline_result.success @@ -634,16 +645,34 @@ def test_pandas_multiple_outputs(): csv_file, parquet_file = temp_tuple # pylint: disable=E0632 pipeline = create_diamond_pipeline() - for _result in execute_pipeline_iterator( - pipeline=pipeline, - environment=get_num_csv_environment( - 'load_csv', [ - csv_materialization('sum_mult_table', csv_file), - parquet_materialization('sum_mult_table', parquet_file), - ] - ), - ): - pass + write_sum_mult_csv = dagster_pd.to_csv_solid('write_sum_mult_csv') + write_sum_mult_parquet = dagster_pd.to_parquet_solid('write_sum_mult_parquet') + + pipeline = create_diamond_pipeline( + extra_solids=[write_sum_mult_csv, write_sum_mult_parquet], + extra_dependencies={ + write_sum_mult_csv.name: { + 'df': DependencyDefinition('sum_mult_table'), + }, + write_sum_mult_parquet.name:{ + 'df': DependencyDefinition('sum_mult_table'), + } + } + ) + + environment = get_num_csv_environment({ + 'load_csv': config.Solid({ + 'path': script_relative_path('num.csv'), + }), + write_sum_mult_csv.name: config.Solid({ + 'path': csv_file, + }), + write_sum_mult_parquet.name: config.Solid({ + 'path': parquet_file, + }), + }) + + execute_pipeline(pipeline, environment) assert os.path.exists(csv_file) output_csv_df = pd.read_csv(csv_file) @@ -679,7 +708,7 @@ def test_rename_input(): }, }, ), - environment=get_num_csv_environment('load_csv'), + environment=get_num_csv_environment(get_load_only_solids_config('load_csv')), ) assert result.success diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_basic_solid.py b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_basic_solid.py index 6ffaa398e68b3..8b07e2aba2ac3 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_basic_solid.py +++ b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_basic_solid.py @@ -15,6 +15,7 @@ from dagster.sqlalchemy_kernel.subquery_builder_experimental import ( create_sql_solid, DagsterSqlTableExpression, + define_create_table_solid, ) from .math_test_db import in_mem_context @@ -56,22 +57,25 @@ def test_sql_sum_solid(): sum_table_solid = create_sum_table_solid() + create_sum_table = define_create_table_solid('create_sum_table_solid') + environment = config.Environment( - materializations=[ - config.Materialization( - solid='sum_table', - name='CREATE', - args={'table_name': 'sum_table'}, - ) - ], + solids={create_sum_table.name: config.Solid({ + 'table_name': 'sum_table', + })} ) pipeline = pipeline_test_def( - solids=[expr_solid, sum_table_solid], + solids=[expr_solid, sum_table_solid, create_sum_table], context=in_mem_context(), - dependencies={'sum_table': { - 'num_table': DependencyDefinition('expr') - }}, + dependencies={ + 'sum_table': { + 'num_table': DependencyDefinition('expr') + }, + create_sum_table.name: { + 'expr': DependencyDefinition('sum_table'), + } + }, ) pipeline_result = execute_pipeline(pipeline, environment) @@ -94,7 +98,7 @@ def create_sum_table_solid(): ) -def create_sum_sq_pipeline(context, expr): +def create_sum_sq_pipeline(context, expr, extra_solids=None, extra_deps=None): check.inst_param(expr, 'expr', DagsterSqlTableExpression) expr_solid = define_pass_mem_value('expr', expr) @@ -108,15 +112,18 @@ def create_sum_sq_pipeline(context, expr): ) return pipeline_test_def( - solids=[expr_solid, sum_solid, sum_sq_solid], + solids=[expr_solid, sum_solid, sum_sq_solid] + (extra_solids if extra_solids else []), context=context, dependencies={ - sum_solid.name: { - 'num_table': DependencyDefinition('expr'), - }, - sum_sq_solid.name: { - sum_solid.name: DependencyDefinition(sum_solid.name), + **{ + sum_solid.name: { + 'num_table': DependencyDefinition('expr'), + }, + sum_sq_solid.name: { + sum_solid.name: DependencyDefinition(sum_solid.name), + }, }, + **(extra_deps if extra_deps else {}) }, ) @@ -144,16 +151,19 @@ def test_execute_sql_sum_sq_solid(): def test_output_sql_sum_sq_solid(): - pipeline = create_sum_sq_pipeline(in_mem_context(), DagsterSqlTableExpression('num_table')) + create_sum_sq_table = define_create_table_solid('create_sum_sq_table') + + pipeline = create_sum_sq_pipeline( + in_mem_context(), DagsterSqlTableExpression('num_table'), [create_sum_sq_table], + {create_sum_sq_table.name: { + 'expr': DependencyDefinition('sum_sq_table') + }} + ) environment = config.Environment( - materializations=[ - config.Materialization( - solid='sum_sq_table', - name='CREATE', - args={'table_name': 'sum_sq_table'}, - ) - ], + solids={'create_sum_sq_table': config.Solid({ + 'table_name': 'sum_sq_table' + })}, ) pipeline_result = execute_pipeline(pipeline=pipeline, environment=environment) diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/subquery_builder_experimental.py b/python_modules/dagster/dagster/sqlalchemy_kernel/subquery_builder_experimental.py index e2a877860b7ba..a947852d31e79 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/subquery_builder_experimental.py +++ b/python_modules/dagster/dagster/sqlalchemy_kernel/subquery_builder_experimental.py @@ -9,8 +9,6 @@ types, ) -from dagster.utils.compatability import create_single_materialization_output - from dagster.sqlalchemy_kernel import execute_sql_text_on_context @@ -48,21 +46,24 @@ def from_target(self): return self._table_name -def create_table_output(): - def materialization_fn(context, arg_dict, sql_expr): - check.inst_param(sql_expr, 'sql_expr', DagsterSqlExpression) - check.dict_param(arg_dict, 'arg_dict') - - output_table_name = check.str_elem(arg_dict, 'table_name') +def define_create_table_solid(name): + def _materialization_fn(context, inputs, config_dict): + sql_expr = inputs['expr'] + check.inst(sql_expr, DagsterSqlExpression) + output_table_name = check.str_elem(config_dict, 'table_name') total_sql = '''CREATE TABLE {output_table_name} AS {query_text}'''.format( output_table_name=output_table_name, query_text=sql_expr.query_text ) context.resources.sa.engine.connect().execute(total_sql) - return create_single_materialization_output( - name='CREATE', - materialization_fn=materialization_fn, - argument_def_dict={'table_name': ArgumentDefinition(types.String)} + return SolidDefinition( + name=name, + inputs=[InputDefinition('expr')], + outputs=[], + transform_fn=_materialization_fn, + config_def={ + 'table_name': ArgumentDefinition(types.String), + } ) @@ -94,7 +95,7 @@ def create_sql_solid(name, inputs, sql_text): name, inputs=inputs, transform_fn=create_sql_transform(sql_text), - output=create_table_output(), + output=OutputDefinition(), ) diff --git a/python_modules/dagster/dagster/utils/compatability.py b/python_modules/dagster/dagster/utils/compatability.py deleted file mode 100644 index 3abb5ab9d478a..0000000000000 --- a/python_modules/dagster/dagster/utils/compatability.py +++ /dev/null @@ -1,31 +0,0 @@ -from dagster import ( - check, - ExpectationDefinition, - InputDefinition, - MaterializationDefinition, - OutputDefinition, -) - -# This is a collection point for older APIs that should no longer be "public" -# but still used in unit tests because I don't want to rewrite them all - - -def create_single_materialization_output( - name, materialization_fn, argument_def_dict, expectations=None -): - ''' - Similar to create_single_source_input this exists because a move in the primitive APIs. - Materializations and outputs used to not be separate concepts so this is a compatability - layer with the old api. Once the *new* api stabilizes this should be removed but it stays - for now. - ''' - return OutputDefinition( - materializations=[ - MaterializationDefinition( - name=name, - materialization_fn=materialization_fn, - argument_def_dict=argument_def_dict, - ) - ], - expectations=expectations - ) diff --git a/python_modules/dagster/dagster_tests/test_config.py b/python_modules/dagster/dagster_tests/test_config.py index 5a68b04921603..60929a58acd8c 100644 --- a/python_modules/dagster/dagster_tests/test_config.py +++ b/python_modules/dagster/dagster_tests/test_config.py @@ -4,27 +4,8 @@ from dagster import config -def test_config(): - mat = config.Materialization(solid='some_solid', name='a_mat_type', args={}) - assert isinstance(mat, config.Materialization) - assert mat.solid == 'some_solid' - assert mat.name == 'a_mat_type' - assert mat.args == {} - - -def test_bad_config(): - with pytest.raises(Exception): - config.Materialization(solid='name', name=1, args={}) - - def test_construct_full_environment(): document = ''' -materializations: - - solid: solid_one - name: mat_name - args: - baaz: quux - context: name: default args: @@ -34,12 +15,5 @@ def test_construct_full_environment(): environment = config.construct_environment(yaml.load(document)) assert environment == config.Environment( - materializations=[ - config.Materialization( - solid='solid_one', - name='mat_name', - args={'baaz': 'quux'}, - ) - ], context=config.Context('default', {'context_arg': 'context_value'}), ) From 109dc82470fa3847db3eabd8d274eabeed083209 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Sun, 19 Aug 2018 11:23:52 -0700 Subject: [PATCH 008/103] minor cleanup --- .../dagster_ge_tests/test_pandas_ge.py | 2 +- .../dagster/dagster/cli/pipeline.py | 3 +- .../core_tests/test_pipeline_execution.py | 3 -- .../dagster/dagster/core/execution.py | 40 ++----------------- 4 files changed, 7 insertions(+), 41 deletions(-) diff --git a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py index ac928a6a87a97..14bf25c923e6b 100644 --- a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py +++ b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py @@ -7,7 +7,7 @@ import dagster_ge from dagster import (InputDefinition, OutputDefinition) from dagster.core.decorators import solid -from dagster.core.execution import (DagsterExecutionFailureReason, execute_pipeline_in_memory) +from dagster.core.errors import DagsterExecutionFailureReason from dagster.utils.test import script_relative_path diff --git a/python_modules/dagster/dagster/cli/pipeline.py b/python_modules/dagster/dagster/cli/pipeline.py index b31229c87d857..d2a92997c5326 100644 --- a/python_modules/dagster/dagster/cli/pipeline.py +++ b/python_modules/dagster/dagster/cli/pipeline.py @@ -8,7 +8,8 @@ import dagster from dagster import check -from dagster.core.execution import (DagsterExecutionFailureReason, execute_pipeline_iterator) +from dagster.core.execution import execute_pipeline_iterator +from dagster.core.errors import DagsterExecutionFailureReason from dagster.core.graph import SolidGraph from dagster.graphviz import build_graphviz_graph from dagster.utils.indenting_printer import IndentingPrinter diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py index 5ccc4f9aee2fe..b87464cfa5ac0 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py @@ -1,5 +1,3 @@ -import pytest - from dagster import ( DependencyDefinition, InputDefinition, @@ -15,7 +13,6 @@ from dagster.core.graph import (create_adjacency_lists, SolidGraph) from dagster.core.execution import ( execute_pipeline_iterator, - execute_pipeline_iterator_in_memory, ExecutionContext, ExecutionStepResult, ) diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index 6031af51dcf23..b4766d1216708 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -20,7 +20,10 @@ import six -from dagster import check, config +from dagster import ( + check, + config, +) from .definitions import ( @@ -31,8 +34,6 @@ from .errors import ( DagsterUserCodeExecutionError, - DagsterExecutionFailureReason, - DagsterExpectationFailedError, DagsterInvariantViolationError, ) @@ -238,23 +239,6 @@ def execute_pipeline_iterator(pipeline, environment): DagsterEnv(pipeline, environment) ) -def execute_pipeline_iterator_in_memory( - context, - pipeline, - input_values, - *, - from_solids=None, - through_solids=None, -): - check.opt_list_param(from_solids, 'from_solids', of_type=str) - check.opt_list_param(through_solids, 'through_solids', of_type=str) - return _execute_pipeline_iterator( - context, - pipeline, - InMemoryEnv(context, pipeline, input_values, from_solids, through_solids), - ) - - def _execute_pipeline_iterator(context, pipeline, env): check.inst_param(context, 'context', ExecutionContext) check.inst_param(pipeline, 'pipeline', PipelineDefinition) @@ -307,22 +291,6 @@ def execute_pipeline( throw_on_error, ) -def execute_pipeline_in_memory( - context, - pipeline, - *, - input_values, - from_solids=None, - through_solids=None, - throw_on_error=True, -): - check.dict_param(input_values, 'input_values', key_type=str) - return _execute_pipeline( - pipeline, - InMemoryEnv(context, pipeline, input_values, from_solids, through_solids), - throw_on_error, - ) - def _execute_pipeline( pipeline, env, From 6de03015798645232347c9354032b8412b2b3e0f Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Sun, 19 Aug 2018 12:17:49 -0700 Subject: [PATCH 009/103] minor cleanup --- .../dagster/dagster/core/core_tests/test_compute_nodes.py | 4 +--- .../dagster/core/core_tests/test_pipeline_execution.py | 4 ---- 2 files changed, 1 insertion(+), 7 deletions(-) diff --git a/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py b/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py index cc6a104ece958..72dd57d82a81e 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py +++ b/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py @@ -42,8 +42,6 @@ def test_compute_noop_node(): assert len(compute_node_graph.nodes) == 1 - context = ExecutionContext() - - outputs = list(compute_node_graph.nodes[0].execute(context, {})) + outputs = list(compute_node_graph.nodes[0].execute(ExecutionContext(), {})) assert outputs[0].success_data.value == 'foo' diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py index b87464cfa5ac0..0002eee0b0a8b 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py @@ -21,10 +21,6 @@ # pylint: disable=W0212 -def create_test_context(): - return ExecutionContext() - - def _default_passthrough_transform(*args, **kwargs): check.invariant(not args, 'There should be no positional args') return list(kwargs.values())[0] From e55ae19493bbe69c7f023f0d44f85c40981f08c4 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Mon, 20 Aug 2018 18:33:10 -0700 Subject: [PATCH 010/103] version bump --- python_modules/dagster/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python_modules/dagster/setup.py b/python_modules/dagster/setup.py index 386a1fddc6b61..991b1bcc98c0b 100644 --- a/python_modules/dagster/setup.py +++ b/python_modules/dagster/setup.py @@ -18,7 +18,7 @@ def long_description(): setup( name='dagster', - version='0.1.9', + version='0.2.0', author='Elementl', author_email='schrockn@elementl.com', license='Apache-2.0', From 691db5cb5fab9e4bc535ef41c6202a8ddefdf562 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Sun, 19 Aug 2018 18:03:30 -0700 Subject: [PATCH 011/103] Define selection execution This begins to layout an approach for partial pipeline execution in a world without sources. The primary mechanism is through "create_pipeline_slice" which allows the use to specify from_solids and through_solids to define the subgraph, and then also pass in "injected_solids" where you specific a dictionary just like dependencies, but instead of DependencyDefinitions you just supply solids directly. If we can build solids directly from a string (through some sort of registry), all elements of this function call can be expressed from a config file. --- .../dagster/dagster/cli/pipeline.py | 4 +- python_modules/dagster/dagster/config.py | 37 +-- .../dagster/dagster/core/compute_nodes.py | 21 +- .../core/core_tests/test_compute_nodes.py | 10 +- .../core_tests/test_pipeline_execution.py | 47 +--- .../dagster/dagster/core/definitions.py | 136 +++++++---- .../dagster/dagster/core/execution.py | 64 ++---- python_modules/dagster/dagster/core/graph.py | 213 +++++++++++------- .../pandas_kernel_tests/test_pandas_solids.py | 90 ++++---- .../test_isolated_templated_sql_tests.py | 3 - 10 files changed, 307 insertions(+), 318 deletions(-) diff --git a/python_modules/dagster/dagster/cli/pipeline.py b/python_modules/dagster/dagster/cli/pipeline.py index d2a92997c5326..9a6df98991b48 100644 --- a/python_modules/dagster/dagster/cli/pipeline.py +++ b/python_modules/dagster/dagster/cli/pipeline.py @@ -10,7 +10,7 @@ from dagster import check from dagster.core.execution import execute_pipeline_iterator from dagster.core.errors import DagsterExecutionFailureReason -from dagster.core.graph import SolidGraph +from dagster.core.graph import ExecutionGraph from dagster.graphviz import build_graphviz_graph from dagster.utils.indenting_printer import IndentingPrinter @@ -38,7 +38,7 @@ def list_command(config): click.echo('Description:') click.echo(format_description(pipeline.description, indent=' ' * 4)) click.echo('Solids: (Execution Order)') - solid_graph = SolidGraph(pipeline.solids, pipeline.dependency_structure) + solid_graph = ExecutionGraph(pipeline.solids, pipeline.dependency_structure) for solid in solid_graph.topological_solids: click.echo(' ' + solid.name) click.echo('*************') diff --git a/python_modules/dagster/dagster/config.py b/python_modules/dagster/dagster/config.py index 0c939e4c0149c..527c7dc87b818 100644 --- a/python_modules/dagster/dagster/config.py +++ b/python_modules/dagster/dagster/config.py @@ -17,24 +17,9 @@ def __new__(cls, config_dict): cls).__new__(cls, check.dict_param(config_dict, 'config_dict', key_type=str)) -class Execution(namedtuple('ExecutionData', 'from_solids through_solids')): - def __new__(cls, from_solids=None, through_solids=None): - return super(Execution, cls).__new__( - cls, - check.opt_list_param(from_solids, 'from_solids', of_type=str), - check.opt_list_param(through_solids, 'through_solids', of_type=str), - ) - - @staticmethod - def single_solid(solid_name): - check.str_param(solid_name, 'solid_name') - return Execution(from_solids=[solid_name], through_solids=[solid_name]) - - -class Environment(namedtuple('EnvironmentData', 'context solids expectations, execution')): - def __new__(cls, *, solids=None, context=None, expectations=None, execution=None): +class Environment(namedtuple('EnvironmentData', 'context solids expectations')): + def __new__(cls, *, solids=None, context=None, expectations=None): check.opt_inst_param(context, 'context', Context) - check.opt_inst_param(execution, 'execution', Execution) if context is None: context = Context(name='default', args={}) @@ -42,15 +27,11 @@ def __new__(cls, *, solids=None, context=None, expectations=None, execution=None if expectations is None: expectations = Expectations(evaluate=True) - if execution is None: - execution = Execution() - return super(Environment, cls).__new__( cls, context=context, solids=check.opt_dict_param(solids, 'solids', key_type=str, value_type=Solid), - expectations=expectations, - execution=execution, + expectations=check.inst_param(expectations, 'expectations', Expectations) ) @staticmethod @@ -93,17 +74,6 @@ def _coerce_to_list(value): check.invariant('should not get here') -def _construct_execution(yml_config_object): - execution_obj = check.opt_dict_elem(yml_config_object, 'execution') - if execution_obj is None: - return None - - return Execution( - from_solids=_coerce_to_list(execution_obj.get('from')), - through_solids=_coerce_to_list(execution_obj.get('through')), - ) - - def _construct_solids(yml_config_object): solid_dict = check.opt_dict_elem(yml_config_object, 'solids') if solid_dict is None: @@ -123,5 +93,4 @@ def construct_environment(yml_config_object): return Environment( solids=_construct_solids(yml_config_object), context=_construct_context(yml_config_object), - execution=_construct_execution(yml_config_object), ) diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index 1f3bed4e17397..3c089d9344025 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -6,7 +6,7 @@ import toposort -from dagster import (check, config) +from dagster import check from dagster.utils.indenting_printer import IndentingPrinter from dagster.utils.logging import ( @@ -24,7 +24,6 @@ ExpectationDefinition, InputDefinition, OutputDefinition, - PipelineDefinition, Result, SolidDefinition, SolidOutputHandle, @@ -38,7 +37,7 @@ DagsterUserCodeExecutionError, ) -from .graph import create_subgraph +from .graph import ExecutionGraph from .types import DagsterType @@ -454,29 +453,23 @@ def __setitem__(self, key, val): return dict.__setitem__(self, key, val) -def create_compute_node_graph_from_env(pipeline, env): +def create_compute_node_graph_from_env(execution_graph, env): import dagster.core.execution - check.inst_param(pipeline, 'pipeline', PipelineDefinition) + check.inst_param(execution_graph, 'execution_graph', ExecutionGraph) check.inst_param(env, 'env', dagster.core.execution.DagsterEnv) - dep_structure = pipeline.dependency_structure + dependency_structure = execution_graph.dependency_structure compute_nodes = [] cn_output_node_map = ComputeNodeOutputMap() - subgraph = create_subgraph( - pipeline, - env.from_solids, - env.through_solids, - ) - - for topo_solid in subgraph.topological_solids: + for topo_solid in execution_graph.topological_solids: cn_inputs = [] for input_def in topo_solid.inputs: prev_cn_output_handle = _prev_node_handle( - dep_structure, + dependency_structure, topo_solid, input_def, cn_output_node_map, diff --git a/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py b/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py index 72dd57d82a81e..a67b88a68b576 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py +++ b/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py @@ -7,8 +7,9 @@ OutputDefinition, ) -from dagster.core.execution import DagsterEnv +from dagster.core.execution import (DagsterEnv, ExecutionGraph) from dagster.core.compute_nodes import create_compute_node_graph_from_env +from dagster.core.graph import ExecutionGraph def silencing_default_context(): @@ -37,8 +38,11 @@ def test_compute_noop_node(): environment = config.Environment() - env = DagsterEnv(pipeline, environment) - compute_node_graph = create_compute_node_graph_from_env(pipeline, env) + env = DagsterEnv(ExecutionGraph.from_pipeline(pipeline), environment) + compute_node_graph = create_compute_node_graph_from_env( + ExecutionGraph.from_pipeline(pipeline), + env, + ) assert len(compute_node_graph.nodes) == 1 diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py index 0002eee0b0a8b..882da856699cd 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py @@ -10,7 +10,7 @@ from dagster.core.definitions import DependencyStructure -from dagster.core.graph import (create_adjacency_lists, SolidGraph) +from dagster.core.graph import (_create_adjacency_lists, ExecutionGraph) from dagster.core.execution import ( execute_pipeline_iterator, ExecutionContext, @@ -65,8 +65,8 @@ def root_transform(_context, args): def _do_construct(solids, dependencies): - dependency_structure = DependencyStructure(solids, dependencies) - return create_adjacency_lists(solids, dependency_structure) + dependency_structure = DependencyStructure.from_definitions(solids, dependencies) + return _create_adjacency_lists(solids, dependency_structure) def test_empty_adjaceny_lists(): @@ -95,7 +95,8 @@ def test_single_dep_adjacency_lists(): def graph_from_solids_only(solids, dependencies): - return SolidGraph(solids, DependencyStructure(solids, dependencies)) + pipeline = PipelineDefinition(solids=solids, dependencies=dependencies) + return ExecutionGraph.from_pipeline(pipeline) def test_diamond_deps_adjaceny_lists(): @@ -176,44 +177,6 @@ def test_diamond_toposort(): assert graph.topological_order == ['A_source', 'A', 'B', 'C', 'D'] -def test_single_node_unprovided_inputs(): - node_a = create_root_solid('A') - solid_graph = graph_from_solids_only([node_a], {}) - assert solid_graph.compute_unprovided_inputs('A', ['A_input']) == set() - assert solid_graph.compute_unprovided_inputs('A', []) == set(['A_input']) - - -# need to replicate these tests with new api -# def test_diamond_toposort_unprovided_inputs(): -# solid_graph = create_diamond_graph() - -# # no inputs -# assert solid_graph.compute_unprovided_inputs('A', []) == set(['A_input']) -# assert solid_graph.compute_unprovided_inputs('B', []) == set(['A_input']) -# assert solid_graph.compute_unprovided_inputs('C', []) == set(['A_input']) -# assert solid_graph.compute_unprovided_inputs('D', []) == set(['A_input']) - -# # root input -# assert solid_graph.compute_unprovided_inputs('A', ['A_input']) == set() -# assert solid_graph.compute_unprovided_inputs('B', ['A_input']) == set() -# assert solid_graph.compute_unprovided_inputs('C', ['A_input']) == set() -# assert solid_graph.compute_unprovided_inputs('D', ['A_input']) == set() - -# # immediate input -# assert solid_graph.compute_unprovided_inputs('A', ['A_input']) == set() -# assert solid_graph.compute_unprovided_inputs('B', ['A']) == set() -# assert solid_graph.compute_unprovided_inputs('C', ['A']) == set() -# assert solid_graph.compute_unprovided_inputs('D', ['B', 'C']) == set() - -# # mixed satisified inputs -# assert solid_graph.compute_unprovided_inputs('D', ['A_input', 'C']) == set() -# assert solid_graph.compute_unprovided_inputs('D', ['B', 'A_input']) == set() - -# # mixed unsatisifed inputs -# assert solid_graph.compute_unprovided_inputs('D', ['C']) == set(['A_input']) -# assert solid_graph.compute_unprovided_inputs('D', ['B']) == set(['A_input']) - - def test_execution_subgraph_one_node(): node_a = create_root_solid('A') solid_graph = graph_from_solids_only([node_a], {}) diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index 728aa89001ec3..4949acf7f2a2c 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -90,50 +90,97 @@ def __init__(self, solid, output=DEFAULT_OUTPUT, description=None): self.description = check.opt_str_param(description, 'description') -DepTarget = namedtuple('DepTarget', 'solid_name output_name') +class InputToOutputHandleDict(dict): + def __getitem__(self, key): + check.inst_param(key, 'key', SolidInputHandle) + return dict.__getitem__(self, key) + def __setitem__(self, key, val): + check.inst_param(key, 'key', SolidInputHandle) + check.inst_param(val, 'val', SolidOutputHandle) + return dict.__setitem__(self, key, val) -class DependencyStructure: - def __init__(self, solids, dep_dict): - check.list_param(solids, 'solids', of_type=SolidDefinition) - check.dict_param(dep_dict, 'dep_dict', key_type=str, value_type=dict) - - def _solid_named(name): - for solid in solids: - if solid.name == name: - return solid - check.failed('not fouhnd') - - self._dep_lookup = defaultdict(dict) - - for solid_name, input_dict in dep_dict.items(): - check.dict_param( - input_dict, - 'input_dict', - key_type=str, - value_type=DependencyDefinition, + +def check_two_dim_str_dict(ddict, param_name, value_type): + check.dict_param(ddict, param_name, key_type=str, value_type=dict) + for sub_dict in ddict.values(): + check.dict_param(sub_dict, 'sub_dict', key_type=str, value_type=value_type) + + +def create_handle_dict(solid_dict, dep_dict): + check.dict_param(solid_dict, 'solid_dict', key_type=str, value_type=SolidDefinition) + check_two_dim_str_dict(dep_dict, 'dep_dict', DependencyDefinition) + + handle_dict = InputToOutputHandleDict() + + for solid_name, input_dict in dep_dict.items(): + for input_name, dep_def in input_dict.items(): + from_solid = solid_dict[solid_name] + to_solid = solid_dict[dep_def.solid] + handle_dict[from_solid.input_handle(input_name)] = to_solid.output_handle( + dep_def.output ) - for input_name, dep in input_dict.items(): - dep_solid = _solid_named(dep.solid) - output_def = dep_solid.output_def_named(dep.output) - self._dep_lookup[solid_name][input_name] = SolidOutputHandle(dep_solid, output_def) + return handle_dict + + +class DependencyStructure: + @staticmethod + def from_definitions(solids, dep_dict): + return DependencyStructure(create_handle_dict(_build_named_dict(solids), dep_dict)) + + def __init__(self, handle_dict): + self._handle_dict = check.inst_param(handle_dict, 'handle_dict', InputToOutputHandleDict) def has_dep(self, solid_input_handle): check.inst_param(solid_input_handle, 'solid_input_handle', SolidInputHandle) - input_name = solid_input_handle.input_def.name - return input_name in self._dep_lookup.get(solid_input_handle.solid.name, {}) + return solid_input_handle in self._handle_dict def deps_of_solid(self, solid_name): check.str_param(solid_name, 'solid_name') - return list(self._dep_lookup[solid_name].values()) + return list(self.__gen_deps_of_solid(solid_name)) + + def __gen_deps_of_solid(self, solid_name): + for input_handle, output_handle in self._handle_dict.items(): + if input_handle.solid.name == solid_name: + yield output_handle def get_dep(self, solid_input_handle): check.inst_param(solid_input_handle, 'solid_input_handle', SolidInputHandle) - return self._dep_lookup[solid_input_handle.solid.name][solid_input_handle.input_def.name] + return self._handle_dict[solid_input_handle] + + def input_handles(self): + return list(self._handle_dict.keys()) + + def items(self): + return self._handle_dict.items() + + +def _build_named_dict(things): + ddict = {} + for thing in things: + ddict[thing.name] = thing + return ddict class PipelineDefinition: + @staticmethod + def create_pipeline_slice(pipeline, from_solids, through_solids, injected_solids): + from .graph import ExecutionGraph + check.inst_param(pipeline, 'pipeline', PipelineDefinition) + check.list_param(from_solids, 'from_solids', of_type=str) + check.list_param(through_solids, 'through_solids', of_type=str) + check_two_dim_str_dict(injected_solids, 'injected_solids', SolidDefinition) + + subgraph = ExecutionGraph.from_pipeline_subset( + pipeline, + from_solids, + through_solids, + injected_solids, + ) + + return subgraph.to_pipeline() + def __init__( self, solids, name=None, description=None, context_definitions=None, dependencies=None ): @@ -159,7 +206,7 @@ def __init__( '''.format(solid=solid.__name__) ) - self.solids = check.list_param(solids, 'solids', of_type=SolidDefinition) + self._solid_dict = _build_named_dict(solids) dependencies = check.opt_dict_param( dependencies, @@ -168,34 +215,19 @@ def __init__( value_type=dict, ) - self.dependency_structure = DependencyStructure(solids, dependencies) + self.dependency_structure = DependencyStructure.from_definitions(solids, dependencies) @property - def solid_names(self): - return [solid.name for solid in self.solids] - - def get_input(self, solid_name, input_name): - for solid in self.solids: - if solid.name != solid_name: - continue - for input_def in solid.inputs: - if input_def.name == input_name: - return input_def - check.failed('not found') + def solids(self): + return list(self._solid_dict.values()) def has_solid(self, name): check.str_param(name, 'name') - for solid in self.solids: - if solid.name == name: - return True - return False + return name in self._solid_dict def solid_named(self, name): check.str_param(name, 'name') - for solid in self.solids: - if solid.name == name: - return solid - check.failed('Could not find solid named ' + name) + return self._solid_dict[name] class ExpectationResult: @@ -284,6 +316,9 @@ def __new__(cls, solid, input_def): def __str__(self): return f'SolidInputHandle(solid="{self.solid.name}", input_name="{self.input_def.name}")' + def __repr__(self): + return f'SolidInputHandle(solid="{self.solid.name}", input_name="{self.input_def.name}")' + def __hash__(self): return hash((self.solid.name, self.input_def.name)) @@ -302,6 +337,9 @@ def __new__(cls, solid, output): def __str__(self): return f'SolidOutputHandle(solid="{self.solid.name}", output.name="{self.output.name}")' + def __repr__(self): + return f'SolidOutputHandle(solid="{self.solid.name}", output.name="{self.output.name}")' + def __hash__(self): return hash((self.solid.name, self.output.name)) diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index b4766d1216708..df766d279a420 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -47,6 +47,8 @@ from .execution_context import ExecutionContext +from .graph import ExecutionGraph + class DagsterPipelineExecutionResult: def __init__( self, @@ -131,13 +133,9 @@ def execute_single_solid(context, solid, environment, throw_on_error=True): check.inst_param(environment, 'environment', config.Environment) check.bool_param(throw_on_error, 'throw_on_error') - check.invariant(environment.execution.from_solids == []) - check.invariant(environment.execution.through_solids == []) - single_solid_environment = config.Environment( expectations=environment.expectations, context=environment.context, - execution=config.Execution.single_solid(solid.name), ) pipeline_result = execute_pipeline( @@ -184,20 +182,13 @@ def _validate_environment(environment, pipeline): class DagsterEnv: - def __init__(self, pipeline, environment): + def __init__(self, execution_graph, environment): # This is not necessarily the best spot for these calls + pipeline = execution_graph.pipeline _validate_environment(environment, pipeline) self.pipeline = check.inst_param(pipeline, 'pipeline', PipelineDefinition) self.environment = check.inst_param(environment, 'environment', config.Environment) - @property - def from_solids(self): - return self.environment.execution.from_solids - - @property - def through_solids(self): - return self.environment.execution.through_solids - @contextmanager def yield_context(self): context_name = self.environment.context.name @@ -231,21 +222,18 @@ def execute_pipeline_iterator(pipeline, environment): check.inst_param(pipeline, 'pipeline', PipelineDefinition) check.inst_param(environment, 'enviroment', config.Environment) - env = DagsterEnv(pipeline, environment) + execution_graph = ExecutionGraph.from_pipeline(pipeline) + env = DagsterEnv(execution_graph, environment) with env.yield_context() as context: - return _execute_pipeline_iterator( - context, - pipeline, - DagsterEnv(pipeline, environment) - ) + return _execute_graph_iterator(context, execution_graph, env) -def _execute_pipeline_iterator(context, pipeline, env): +def _execute_graph_iterator(context, execution_graph, env): check.inst_param(context, 'context', ExecutionContext) - check.inst_param(pipeline, 'pipeline', PipelineDefinition) + check.inst_param(execution_graph, 'execution_graph', ExecutionGraph) check.inst_param(env, 'env', DagsterEnv) - cn_graph = create_compute_node_graph_from_env(pipeline, env) + cn_graph = create_compute_node_graph_from_env(execution_graph, env) cn_nodes = list(cn_graph.topological_nodes()) @@ -278,23 +266,13 @@ def _execute_pipeline_iterator(context, pipeline, env): output_name=cn_result.success_data.output_name, ) + + def execute_pipeline( pipeline, environment, *, throw_on_error=True, -): - check.inst_param(environment, 'environment', config.Environment) - return _execute_pipeline( - pipeline, - DagsterEnv(pipeline, environment), - throw_on_error, - ) - -def _execute_pipeline( - pipeline, - env, - throw_on_error=True, ): ''' "Synchronous" version of execute_pipeline_iteator. @@ -304,17 +282,25 @@ def _execute_pipeline( Note: throw_on_error is very useful in testing contexts when not testing for error conditions ''' + check.inst_param(pipeline, 'pipeline', PipelineDefinition) + check.inst_param(environment, 'environment', config.Environment) + execution_graph = ExecutionGraph.from_pipeline(pipeline) + env = DagsterEnv(execution_graph, environment) + return _execute_graph(execution_graph, env, throw_on_error) + +def _execute_graph( + execution_graph, + env, + throw_on_error=True, +): + check.inst_param(execution_graph, 'execution_graph', ExecutionGraph) check.inst_param(env, 'env', DagsterEnv) check.bool_param(throw_on_error, 'throw_on_error') results = [] with env.yield_context() as context: - for result in _execute_pipeline_iterator( - context, - pipeline, - env=env, - ): + for result in _execute_graph_iterator(context, execution_graph, env): if throw_on_error: if not result.success: _do_throw_on_error(result) diff --git a/python_modules/dagster/dagster/core/graph.py b/python_modules/dagster/dagster/core/graph.py index 4a5cc19ab199b..b0c8bf2db6838 100644 --- a/python_modules/dagster/dagster/core/graph.py +++ b/python_modules/dagster/dagster/core/graph.py @@ -1,15 +1,20 @@ +from collections import defaultdict from toposort import toposort_flatten from dagster import check from .definitions import ( + DependencyDefinition, DependencyStructure, + InputToOutputHandleDict, PipelineDefinition, SolidDefinition, + _build_named_dict, + create_handle_dict, ) -def create_adjacency_lists(solids, dep_structure): +def _create_adjacency_lists(solids, dep_structure): check.list_param(solids, 'solids', of_type=SolidDefinition) check.inst_param(dep_structure, 'dep_structure', DependencyStructure) @@ -37,29 +42,83 @@ def visit(solid_name): return (forward_edges, backward_edges) -class SolidGraph: - def __init__(self, solids, dependency_structure): +def _dependency_structure_to_dep_dict(dependency_structure): + dep_dict = defaultdict(dict) + for input_handle, output_handle in dependency_structure.items(): + dep_dict[input_handle.solid.name][input_handle.input_def.name] = DependencyDefinition( + solid=output_handle.solid.name, + output=output_handle.output.name, + ) + return dep_dict + + +class ExecutionGraph: + @staticmethod + def from_pipeline(pipeline): + check.inst_param(pipeline, 'pipeline', PipelineDefinition) + return ExecutionGraph(pipeline, pipeline.solids, pipeline.dependency_structure) + + @staticmethod + def from_pipeline_subset(pipeline, from_solids, through_solids, injected_solids): + check.inst_param(pipeline, 'pipeline', PipelineDefinition) + check.list_param(from_solids, 'from_solids', of_type=str) + check.list_param(through_solids, 'through_solids', of_type=str) + graph = ExecutionGraph.from_pipeline(pipeline) + return _create_subgraph(graph, from_solids, through_solids).augment(injected_solids) + + def to_pipeline(self): + return PipelineDefinition( + solids=self.solids, + dependencies=_dependency_structure_to_dep_dict(self.dependency_structure), + context_definitions=self.pipeline.context_definitions + ) + + def augment(self, injected_solids): + + new_deps = defaultdict(dict) + new_solids = [] + + for from_solid_name, targets_by_input in injected_solids.items(): + for from_input_name, target_solid in targets_by_input.items(): + new_solids.append(target_solid) + new_deps[from_solid_name][from_input_name] = DependencyDefinition( + solid=target_solid.name + ) + + check.list_param(new_solids, 'new_solids', of_type=SolidDefinition) + + solids = self.solids + new_solids + + solid_dict = _build_named_dict(solids) + handle_dict = InputToOutputHandleDict() + for input_handle, output_handle in self.dependency_structure.items(): + handle_dict[input_handle] = output_handle + + for input_handle, output_handle in create_handle_dict(solid_dict, new_deps).items(): + handle_dict[input_handle] = output_handle + + return ExecutionGraph(self.pipeline, solids, DependencyStructure(handle_dict)) + + def __init__(self, pipeline, solids, dependency_structure): + self.pipeline = pipeline solids = check.list_param(solids, 'solids', of_type=SolidDefinition) - self.dep_structure = check.inst_param( + self.dependency_structure = check.inst_param( dependency_structure, 'dependency_structure', DependencyStructure ) self._solid_dict = {solid.name: solid for solid in solids} + for input_handle in dependency_structure.input_handles(): + check.invariant(input_handle.solid.name in self._solid_dict) + solid_names = set([solid.name for solid in solids]) check.invariant(len(solid_names) == len(solids), 'must have unique names') - all_inputs = {} - - for solid in solids: - for input_def in solid.inputs: - # if input exists should probably ensure that it is the same - all_inputs[input_def.name] = input_def - - self._all_inputs = all_inputs - self.forward_edges, self.backward_edges = create_adjacency_lists(solids, self.dep_structure) + self.forward_edges, self.backward_edges = _create_adjacency_lists( + solids, self.dependency_structure + ) self.topological_order = toposort_flatten(self.backward_edges, sort=True) self._transitive_deps = {} @@ -72,6 +131,10 @@ def topological_solids(self): def solids(self): return list(self._solid_dict.values()) + def solid_named(self, name): + check.str_param(name, 'name') + return self._solid_dict[name] + def transitive_dependencies_of(self, solid_name): check.str_param(solid_name, 'solid_name') @@ -82,8 +145,8 @@ def transitive_dependencies_of(self, solid_name): solid = self._solid_dict[solid_name] for inp in solid.inputs: input_handle = solid.input_handle(inp.name) - if self.dep_structure.has_dep(input_handle): - output_handle = self.dep_structure.get_dep(input_handle) + if self.dependency_structure.has_dep(input_handle): + output_handle = self.dependency_structure.get_dep(input_handle) trans_deps.add(output_handle.solid.name) trans_deps.union(self.transitive_dependencies_of(output_handle.solid.name)) @@ -97,117 +160,93 @@ def _check_solid_name(self, solid_name): f'Solid {solid_name} must exist in {list(self._solid_dict.keys())}' ) - def _check_input_names(self, input_names): - check.list_param(input_names, 'input_names', of_type=str) - - for input_name in input_names: - check.param_invariant( - input_name in self._all_inputs, 'input_names', 'Input name not found' - ) - - def compute_unprovided_inputs(self, solid_name, input_names): - ''' - Given a single solid_name and a set of input_names that represent the - set of inputs provided for a computation, return the inputs that are *missing*. - This detects the case where an upstream path in the DAG of solids does not - have enough information to materialize a given output. - ''' - - self._check_solid_name(solid_name) - self._check_input_names(input_names) - - input_set = set(input_names) - - unprovided_inputs = set() - - visit_dict = {name: False for name in self._solid_dict.keys()} - - output_solid = self._solid_dict[solid_name] - - def visit(solid): - if visit_dict[solid.name]: - return - visit_dict[solid.name] = True - - for inp in solid.inputs: - if inp.name in input_set: - continue + def create_execution_subgraph(self, from_solids, to_solids): + check.list_param(from_solids, 'from_solids', of_type=str) + check.list_param(to_solids, 'to_solids', of_type=str) - input_handle = solid.input_handle(inp.name) + involved_solid_set = self._compute_involved_solid_set(from_solids, to_solids) - if self.dep_structure.has_dep(input_handle): - output_handle = self.dep_structure.get_dep(input_handle) - visit(self._solid_dict[output_handle.solid.name]) - else: - unprovided_inputs.add(inp.name) + involved_solids = [self._solid_dict[name] for name in involved_solid_set] - visit(output_solid) + handle_dict = InputToOutputHandleDict() - return unprovided_inputs + for solid in involved_solids: + for input_def in solid.inputs: + input_handle = solid.input_handle(input_def.name) + if self.dependency_structure.has_dep(input_handle): + handle_dict[input_handle] = self.dependency_structure.get_dep(input_handle) - def create_execution_subgraph(self, from_solids, to_solids): - check.list_param(from_solids, 'from_solids', of_type=str) - check.list_param(to_solids, 'to_solids', of_type=str) + return ExecutionGraph(self.pipeline, involved_solids, DependencyStructure(handle_dict)) + def _compute_involved_solid_set(self, from_solids, to_solids): from_solid_set = set(from_solids) - involved_solids = from_solid_set + involved_solid_set = from_solid_set def visit(solid): - if solid.name in involved_solids: + if solid.name in involved_solid_set: return - involved_solids.add(solid.name) + + involved_solid_set.add(solid.name) for input_def in solid.inputs: input_handle = solid.input_handle(input_def.name) - if not self.dep_structure.has_dep(input_handle): + if not self.dependency_structure.has_dep(input_handle): continue - from_solid = self.dep_structure.get_dep(input_handle).solid.name + output_handle = self.dependency_structure.get_dep(input_handle) - if from_solid in from_solid_set: + next_solid = output_handle.solid.name + if next_solid in from_solid_set: continue - visit(self._solid_dict[from_solid]) + visit(self._solid_dict[next_solid]) for to_solid in to_solids: visit(self._solid_dict[to_solid]) - return SolidGraph([self._solid_dict[name] for name in involved_solids], self.dep_structure) + return involved_solid_set -def all_depended_on_solids(pipeline): - check.inst_param(pipeline, 'pipeline', PipelineDefinition) - dep_struct = pipeline.dependency_structure - for solid in pipeline.solids: +def _build_named_dict(things): + ddict = {} + for thing in things: + ddict[thing.name] = thing + return ddict + + +def _all_depended_on_solids(execution_graph): + check.inst_param(execution_graph, 'execution_graph', ExecutionGraph) + + dependency_structure = execution_graph.dependency_structure + + for solid in execution_graph.solids: for input_def in solid.inputs: input_handle = solid.input_handle(input_def.name) - if dep_struct.has_dep(input_handle): - output_handle = dep_struct.get_dep(input_handle) - yield pipeline.solid_named(output_handle.solid.name) + if dependency_structure.has_dep(input_handle): + output_handle = dependency_structure.get_dep(input_handle) + yield execution_graph.solid_named(output_handle.solid.name) -def all_sink_solids(pipeline): - check.inst_param(pipeline, 'pipeline', PipelineDefinition) - all_names = set([solid.name for solid in pipeline.solids]) - all_depended_on_names = set([solid.name for solid in all_depended_on_solids(pipeline)]) +def _all_sink_solids(execution_graph): + check.inst_param(execution_graph, 'execution_graph', ExecutionGraph) + all_names = set([solid.name for solid in execution_graph.solids]) + all_depended_on_names = set([solid.name for solid in _all_depended_on_solids(execution_graph)]) return all_names.difference(all_depended_on_names) -def create_subgraph(pipeline, from_solids, through_solids): - check.inst_param(pipeline, 'pipeline', PipelineDefinition) +def _create_subgraph(execution_graph, from_solids, through_solids): + check.inst_param(execution_graph, 'execution_graph', ExecutionGraph) check.opt_list_param(from_solids, 'from_solids', of_type=str) check.opt_list_param(through_solids, 'through_solids', of_type=str) - solid_graph = SolidGraph(pipeline.solids, pipeline.dependency_structure) - if not through_solids: - through_solids = list(all_sink_solids(pipeline)) + through_solids = list(_all_sink_solids(execution_graph)) if not from_solids: all_deps = set() for through_solid in through_solids: - all_deps.union(solid_graph.transitive_dependencies_of(through_solid)) + all_deps.union(execution_graph.transitive_dependencies_of(through_solid)) from_solids = list(all_deps) - return solid_graph.create_execution_subgraph(from_solids, through_solids) + return execution_graph.create_execution_subgraph(from_solids, through_solids) diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py index 24440cfd21b3e..3ff6e20c0028c 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py @@ -1,10 +1,8 @@ import os -import pytest import pandas as pd from dagster import ( - ArgumentDefinition, DependencyDefinition, ExecutionContext, InputDefinition, @@ -14,13 +12,11 @@ check, config, ) -from dagster.core import types from dagster.core.decorators import solid from dagster.core.execution import ( - ExecutionContext, + ExecutionGraph, execute_pipeline_iterator, execute_pipeline, - execute_single_solid, ) import dagster.pandas_kernel as dagster_pd from dagster.utils.test import (get_temp_file_name, get_temp_file_names, script_relative_path) @@ -56,11 +52,8 @@ def get_load_only_solids_config(load_csv_solid_name): return {load_csv_solid_name: config.Solid({'path': script_relative_path('num.csv')})} -def get_num_csv_environment(solids_config, through_solids=None): - return config.Environment( - solids=solids_config, - execution=config.Execution(through_solids=through_solids), - ) +def get_num_csv_environment(solids_config): + return config.Environment(solids=solids_config) def create_test_context(): @@ -394,9 +387,7 @@ def test_pandas_in_memory_diamond_pipeline(): pipeline = create_diamond_pipeline() result = execute_pipeline( pipeline, - environment=get_num_csv_environment( - get_load_only_solids_config('load_csv'), through_solids=['sum_mult_table'] - ) + environment=get_num_csv_environment(get_load_only_solids_config('load_csv')) ) assert result.result_named('sum_mult_table').transformed_value.to_dict('list') == { @@ -507,35 +498,45 @@ def test_pandas_output_intermediate_csv_files(): mult_table_result = subgraph_one_result.result_named('mult_table') assert mult_table_result.transformed_value.to_dict('list') == expected_mult - # TODO need better partial execution API - return - - # pipeline_result = execute_pipeline( - # pipeline, - # environment=config.Environment( - # sources={ - # 'sum_mult_table': { - # 'sum_table': config.Source('CSV', {'path': sum_file}), - # 'mult_table': config.Source('CSV', {'path': mult_file}), - # }, - # }, - # execution=config.Execution.single_solid('sum_mult_table'), - # ), - # ) - - # assert pipeline_result.success - - # subgraph_two_result_list = pipeline_result.result_list - - # assert len(subgraph_two_result_list) == 1 - # output_df = subgraph_two_result_list[0].transformed_value - # assert output_df.to_dict('list') == { - # 'num1': [1, 3], - # 'num2': [2, 4], - # 'sum': [3, 7], - # 'mult': [2, 12], - # 'sum_mult': [6, 84], - # } + injected_solids = { + 'sum_mult_table': { + 'sum_table' : dagster_pd.load_csv_solid('load_sum_table'), + 'mult_table' : dagster_pd.load_csv_solid('load_mult_table'), + } + } + + pipeline_result = execute_pipeline( + PipelineDefinition.create_pipeline_slice( + pipeline, + ['sum_mult_table'], + ['sum_mult_table'], + injected_solids, + ), + environment=config.Environment( + solids={ + 'load_sum_table' : config.Solid( + {'path' : sum_file}, + ), + 'load_mult_table' : config.Solid( + {'path' : mult_file}, + ), + }, + ), + ) + + assert pipeline_result.success + + subgraph_two_result_list = pipeline_result.result_list + + assert len(subgraph_two_result_list) == 3 + output_df = pipeline_result.result_named('sum_mult_table').transformed_value + assert output_df.to_dict('list') == { + 'num1': [1, 3], + 'num2': [2, 4], + 'sum': [3, 7], + 'mult': [2, 12], + 'sum_mult': [6, 84], + } def test_pandas_output_intermediate_parquet_files(): @@ -565,10 +566,10 @@ def test_pandas_output_intermediate_parquet_files(): 'path': script_relative_path('num.csv'), }), write_sum_table.name: config.Solid({ - 'path': sum_file + 'path': sum_file }), write_mult_table.name: config.Solid({ - 'path': mult_file + 'path': mult_file }), }) @@ -598,7 +599,6 @@ def test_pandas_multiple_inputs(): 'path': script_relative_path('num.csv') }), }, - execution=config.Execution(through_solids=['double_sum']), ) def transform_fn(_context, args): diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py index b441e36599096..c5c3adedb1726 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py +++ b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py @@ -213,7 +213,6 @@ def test_templated_sql_solid_pipeline(): second_sum_sq_table = 'second_sum_sq_table' - # This is a bit awkward ATM environment_two = config.Environment( solids={ 'pass_value': @@ -226,7 +225,6 @@ def test_templated_sql_solid_pipeline(): 'sum_sq_table': second_sum_sq_table, }), }, - execution=config.Execution(from_solids=['pass_value'], through_solids=['sum_sq_table']), ) second_result = execute_pipeline( @@ -290,7 +288,6 @@ def test_with_from_through_specifying_all_solids(): } ), }, - execution=config.Execution(from_solids=all_solid_names, through_solids=all_solid_names) ) pipeline_result = execute_pipeline(pipeline, environment=environment) From 2c153791d50fc55e898545f5a0f94bd24c248dff Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Tue, 21 Aug 2018 08:50:07 -0700 Subject: [PATCH 012/103] Some usability improvements Error messages, mostly as well as some improved logging --- .../dagster_ge_tests/test_pandas_ge.py | 122 +++++++++--------- python_modules/dagster/dagster/__init__.py | 2 + .../dagster/dagster/cli/pipeline.py | 2 +- .../dagster/dagster/core/compute_nodes.py | 13 +- .../core/core_tests/test_definition_errors.py | 96 ++++++++++++++ .../core_tests/test_pipeline_execution.py | 11 -- .../dagster/dagster/core/definitions.py | 73 ++++++++--- .../dagster/dagster/core/execution.py | 15 ++- .../dagster/dagster/core/utility_solids.py | 4 +- ...est_pandas_hello_world_no_library_slide.py | 1 - .../pandas_kernel_tests/test_pandas_solids.py | 12 +- 11 files changed, 240 insertions(+), 111 deletions(-) create mode 100644 python_modules/dagster/dagster/core/core_tests/test_definition_errors.py diff --git a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py index 14bf25c923e6b..eb31fc1541247 100644 --- a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py +++ b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py @@ -5,9 +5,15 @@ import dagster import dagster.pandas_kernel as dagster_pd import dagster_ge -from dagster import (InputDefinition, OutputDefinition) -from dagster.core.decorators import solid -from dagster.core.errors import DagsterExecutionFailureReason +from dagster import ( + config, + solid, + execute_pipeline, + InputDefinition, + OutputDefinition, + DependencyDefinition, +) +from dagster.core.errors import (DagsterExecutionFailureReason, DagsterExpectationFailedError) from dagster.utils.test import script_relative_path @@ -65,72 +71,64 @@ def sum_solid_expectations_config(num_df): return _sum_solid_impl(num_df) +from dagster.core.utility_solids import define_pass_mem_value + + def test_single_node_passing_expectation(): in_df = pd.DataFrame.from_dict({'num1': [1, 3], 'num2': [2, 4]}) - pipeline = dagster.PipelineDefinition(solids=[sum_solid]) - - return - # result = execute_pipeline_in_memory( - # dagster.ExecutionContext(), - # pipeline, - # input_values={sum_solid.name: { - # 'num_df': in_df - # }}, - # ) - # assert result.success - # assert result.result_list[0].success - # assert result.result_list[0].transformed_value.to_dict('list') == { - # 'num1': [1, 3], - # 'num2': [2, 4], - # 'sum': [3, 7], - # } + pipeline = dagster.PipelineDefinition( + solids=[define_pass_mem_value('value', in_df), sum_solid], + dependencies={'sum_solid': { + 'num_df': DependencyDefinition('value') + }} + ) + + result = execute_pipeline(pipeline, config.Environment()) + assert result.success + assert len(result.result_list) == 2 + assert result.result_list[1].success + assert result.result_list[1].transformed_value.to_dict('list') == { + 'num1': [1, 3], + 'num2': [2, 4], + 'sum': [3, 7], + } def test_single_node_passing_json_config_expectations(): in_df = pd.DataFrame.from_dict({'num1': [1, 3], 'num2': [2, 4]}) - pipeline = dagster.PipelineDefinition(solids=[sum_solid_expectations_config]) - - return - # result = execute_pipeline_in_memory( - # dagster.ExecutionContext(), - # pipeline, - # input_values={sum_solid_expectations_config.name: { - # 'num_df': in_df - # }}, - # ) - # assert result.success - # assert result.result_list[0].success - # assert result.result_list[0].transformed_value.to_dict('list') == { - # 'num1': [1, 3], - # 'num2': [2, 4], - # 'sum': [3, 7], - # } + pipeline = dagster.PipelineDefinition( + solids=[define_pass_mem_value('value', in_df), sum_solid_expectations_config], + dependencies={ + sum_solid_expectations_config.name: { + 'num_df': DependencyDefinition('value') + } + } + ) + + result = execute_pipeline(pipeline, config.Environment()) + assert result.success + assert len(result.result_list) == 2 + assert result.result_list[1].success + assert result.result_list[1].transformed_value.to_dict('list') == { + 'num1': [1, 3], + 'num2': [2, 4], + 'sum': [3, 7], + } def test_single_node_failing_expectation(): in_df = pd.DataFrame.from_dict({'num1': [1, 3], 'num2': [2, 4]}) - pipeline = dagster.PipelineDefinition(solids=[sum_solid_fails_input_expectation]) - return - # result = execute_pipeline_in_memory( - # dagster.ExecutionContext(), - # pipeline, - # input_values={sum_solid_fails_input_expectation.name: { - # 'num_df': in_df - # }}, - # throw_on_error=False - # ) - # assert not result.success - - # return - # TODO redo expectation result API - # assert len(result.result_list) == 1 - # first_solid_result = result.result_list[0] - # assert not first_solid_result.success - # assert first_solid_result.reason == DagsterExecutionFailureReason.EXPECTATION_FAILURE - # assert isinstance(first_solid_result.input_expectation_results, dagster.InputExpectationResults) - # input_expt_results = first_solid_result.input_expectation_results - # assert len(input_expt_results.result_dict) == 1 - # input_expt_result = list(input_expt_results.result_dict.values())[0] - # assert isinstance(input_expt_result, dagster.InputExpectationResult) - # assert len(list(input_expt_result.passes)) == 0 - # assert len(list(input_expt_result.fails)) == 1 + pipeline = dagster.PipelineDefinition( + solids=[define_pass_mem_value('value', in_df), sum_solid_fails_input_expectation], + dependencies={ + sum_solid_fails_input_expectation.name: { + 'num_df': DependencyDefinition('value') + } + } + ) + + # NOTE: this is not what I want to API to be but at least it exercises + # the code path for now + with pytest.raises(DagsterExpectationFailedError): + result = execute_pipeline(pipeline, config.Environment()) + assert not result.success diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index dc62d34678745..9854309c6a98e 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -22,4 +22,6 @@ with_context, ) +from dagster.core.errors import (DagsterInvalidDefinitionError) + import dagster.core.types as types diff --git a/python_modules/dagster/dagster/cli/pipeline.py b/python_modules/dagster/dagster/cli/pipeline.py index 9a6df98991b48..0748e320ea0e9 100644 --- a/python_modules/dagster/dagster/cli/pipeline.py +++ b/python_modules/dagster/dagster/cli/pipeline.py @@ -38,7 +38,7 @@ def list_command(config): click.echo('Description:') click.echo(format_description(pipeline.description, indent=' ' * 4)) click.echo('Solids: (Execution Order)') - solid_graph = ExecutionGraph(pipeline.solids, pipeline.dependency_structure) + solid_graph = ExecutionGraph(pipeline, pipeline.solids, pipeline.dependency_structure) for solid in solid_graph.topological_solids: click.echo(' ' + solid.name) click.echo('*************') diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index 3c089d9344025..e312607d83262 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -156,12 +156,13 @@ class ComputeNodeTag(Enum): JOIN_OUTPUT = 'join_output' EXPECTATION_INPUT = 'expectation_input' -def _execute_core_transform(context, solid_transform_fn, values_dict, config_dict): +def _execute_core_transform(context, solid_name, solid_transform_fn, values_dict, config_dict): ''' Execute the user-specified transform for the solid. Wrap in an error boundary and do all relevant logging and metrics tracking ''' check.inst_param(context, 'context', ExecutionContext) + check.str_param(solid_name, 'solid_name') check.callable_param(solid_transform_fn, 'solid_transform_fn') check.dict_param(values_dict, 'values_dict', key_type=str) check.dict_param(config_dict, 'config_dict', key_type=str) @@ -169,10 +170,11 @@ def _execute_core_transform(context, solid_transform_fn, values_dict, config_dic error_str = 'Error occured during core transform' with _user_code_error_boundary(context, error_str): with time_execution_scope() as timer_result: - gen = solid_transform_fn(context, values_dict, config_dict) - if gen is not None: - for result in gen: - yield result + with context.value('solid', solid_name): + gen = solid_transform_fn(context, values_dict, config_dict) + if gen is not None: + for result in gen: + yield result context.metric('core_transform_time_ms', timer_result.millis) @@ -617,6 +619,7 @@ def create_compute_node_from_solid_transform(solid, node_inputs, config_args): arg_dict={}, compute_fn=lambda context, inputs: _execute_core_transform( context, + solid.name, solid.transform_fn, inputs, config_args, diff --git a/python_modules/dagster/dagster/core/core_tests/test_definition_errors.py b/python_modules/dagster/dagster/core/core_tests/test_definition_errors.py new file mode 100644 index 0000000000000..1c71c397bebdb --- /dev/null +++ b/python_modules/dagster/dagster/core/core_tests/test_definition_errors.py @@ -0,0 +1,96 @@ +import pytest + +from dagster import ( + DagsterInvalidDefinitionError, + DependencyDefinition, + PipelineDefinition, + SolidDefinition, + OutputDefinition, + InputDefinition, +) + + +def solid_a_b_list(): + return [ + SolidDefinition( + name='A', + inputs=[], + outputs=[OutputDefinition()], + transform_fn=lambda context, inputs, config: None, + config_def={}, + ), + SolidDefinition( + name='B', + inputs=[InputDefinition('b_input')], + outputs=[], + transform_fn=lambda context, inputs, config: None, + config_def={}, + ) + ] + + +def test_no_dep_specified(): + with pytest.raises(DagsterInvalidDefinitionError, message='Dependency must be specified'): + PipelineDefinition(solids=solid_a_b_list(), dependencies={}) + + +def test_circular_dep(): + with pytest.raises(DagsterInvalidDefinitionError, message='Circular reference'): + PipelineDefinition( + solids=solid_a_b_list(), + dependencies={'B': { + 'b_input': DependencyDefinition('B') + }}, + ) + + +def test_from_solid_not_there(): + with pytest.raises( + DagsterInvalidDefinitionError, + message='Solid NOTTHERE in dependency dictionary not found', + ): + PipelineDefinition( + solids=solid_a_b_list(), + dependencies={'NOTTHERE': { + 'b_input': DependencyDefinition('A') + }}, + ) + + +def test_from_non_existant_input(): + with pytest.raises( + DagsterInvalidDefinitionError, + message='Solid B does not have input not_an_input', + ): + PipelineDefinition( + solids=solid_a_b_list(), + dependencies={'B': { + 'not_an_input': DependencyDefinition('A') + }}, + ) + + +def test_to_solid_not_there(): + with pytest.raises( + DagsterInvalidDefinitionError, + message='Solid NOTTHERE in DependencyDefinition not found in solid list', + ): + PipelineDefinition( + solids=solid_a_b_list(), + dependencies={'B': { + 'b_input': DependencyDefinition('NOTTHERE') + }}, + ) + + +def test_to_solid_output_not_there(): + with pytest.raises( + DagsterInvalidDefinitionError, + message='Solid A does not have output NOTTHERE', + ): + PipelineDefinition( + solids=solid_a_b_list(), + dependencies={'B': { + 'b_input': DependencyDefinition('A', output='NOTTHERE') + }}, + ) diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py index 882da856699cd..110bf4f27a5b7 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py @@ -177,17 +177,6 @@ def test_diamond_toposort(): assert graph.topological_order == ['A_source', 'A', 'B', 'C', 'D'] -def test_execution_subgraph_one_node(): - node_a = create_root_solid('A') - solid_graph = graph_from_solids_only([node_a], {}) - - execution_graph = solid_graph.create_execution_subgraph( - from_solids=['A'], - to_solids=['A'], - ) - assert execution_graph - - def test_execution_graph_diamond(): solid_graph = create_diamond_graph() diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index 4949acf7f2a2c..e3c5d3881ec97 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -105,6 +105,7 @@ def check_two_dim_str_dict(ddict, param_name, value_type): check.dict_param(ddict, param_name, key_type=str, value_type=dict) for sub_dict in ddict.values(): check.dict_param(sub_dict, 'sub_dict', key_type=str, value_type=value_type) + return ddict def create_handle_dict(solid_dict, dep_dict): @@ -208,15 +209,58 @@ def __init__( self._solid_dict = _build_named_dict(solids) - dependencies = check.opt_dict_param( + dependencies = check_two_dim_str_dict( dependencies, 'dependencies', - key_type=str, - value_type=dict, - ) + DependencyDefinition, + ) if dependencies else {} + + for from_solid, dep_by_input in dependencies.items(): + for from_input, dep in dep_by_input.items(): + if from_solid == dep.solid: + raise DagsterInvalidDefinitionError( + f'Circular reference detected in solid {from_solid} input {from_input}.' + ) + if not from_solid in self._solid_dict: + raise DagsterInvalidDefinitionError( + f'Solid {from_solid} in dependency dictionary not found in solid list', + ) + + if not self._solid_dict[from_solid].has_input(from_input): + input_list = [ + input_def.name for input_def in self._solid_dict[from_solid].inputs + ] + raise DagsterInvalidDefinitionError( + f'Solid {from_solid} does not have input {from_input}. ' + \ + f'Input list: {input_list}' + ) + + if not dep.solid in self._solid_dict: + raise DagsterInvalidDefinitionError( + f'Solid {dep.solid} in DependencyDefinition not found in solid list', + ) + + if not self._solid_dict[dep.solid].has_output(dep.output): + raise DagsterInvalidDefinitionError( + f'Solid {dep.solid} does not have output {dep.output}', + ) self.dependency_structure = DependencyStructure.from_definitions(solids, dependencies) + for solid in solids: + for input_def in solid.inputs: + if not self.dependency_structure.has_dep(solid.input_handle(input_def.name)): + if name: + raise DagsterInvalidDefinitionError( + f'Dependency must be specified for solid {solid.name} input ' + \ + f'{input_def.name} in pipeline {name}' + ) + else: + raise DagsterInvalidDefinitionError( + f'Dependency must be specified for solid {solid.name} input ' + \ + f'{input_def.name}' + ) + @property def solids(self): return list(self._solid_dict.values()) @@ -376,6 +420,8 @@ def __init__(self, *, name, inputs, transform_fn, outputs, config_def, descripti output_handles[output.name] = SolidOutputHandle(self, output) self.output_handles = output_handles + self._input_dict = _build_named_dict(inputs) + self._output_dict = _build_named_dict(outputs) @staticmethod def single_output_transform(name, inputs, transform_fn, output, description=None): @@ -406,26 +452,19 @@ def input_names(self): def has_input(self, name): check.str_param(name, 'name') - for input_def in self.inputs: - if input_def.name == name: - return True - return False + return name in self._input_dict def input_def_named(self, name): check.str_param(name, 'name') - for input_def in self.inputs: - if input_def.name == name: - return input_def + return self._input_dict[name] - check.failed('input {name} not found'.format(name=name)) + def has_output(self, name): + check.str_param(name, 'name') + return name in self._output_dict def output_def_named(self, name): check.str_param(name, 'name') - for output in self.outputs: - if output.name == name: - return output - - check.failed('output {name} not found'.format(name=name)) + return self._output_dict[name] class __ArgumentValueSentinel: diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index df766d279a420..b8a062ccd6020 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -300,10 +300,11 @@ def _execute_graph( results = [] with env.yield_context() as context: - for result in _execute_graph_iterator(context, execution_graph, env): - if throw_on_error: - if not result.success: - _do_throw_on_error(result) - - results.append(result.copy()) - return DagsterPipelineExecutionResult(context, results) + with context.value('pipeline', execution_graph.pipeline.name or '<>'): + for result in _execute_graph_iterator(context, execution_graph, env): + if throw_on_error: + if not result.success: + _do_throw_on_error(result) + + results.append(result.copy()) + return DagsterPipelineExecutionResult(context, results) diff --git a/python_modules/dagster/dagster/core/utility_solids.py b/python_modules/dagster/dagster/core/utility_solids.py index d76188ff6fff8..421393ae94370 100644 --- a/python_modules/dagster/dagster/core/utility_solids.py +++ b/python_modules/dagster/dagster/core/utility_solids.py @@ -8,14 +8,16 @@ ) -def define_pass_value_solid(name): +def define_pass_value_solid(name, description=None): check.str_param(name, 'name') + check.opt_str_param(description, 'description') def _value_t_fn(_context, _inputs, config_dict): yield Result(config_dict['value']) return SolidDefinition( name=name, + description=description, inputs=[], outputs=[OutputDefinition(dagster_type=types.String)], config_def={'value': ArgumentDefinition(types.String)}, diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py index ad236bda5354c..c9a86b3a12c3a 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py @@ -31,7 +31,6 @@ def _t_fn(_context, _inputs, config_dict): def define_to_csv_solid(name): def _t_fn(_context, inputs, config_dict): - print('WRITING NOW') inputs['df'].to_csv(config_dict['path'], index=False) return SolidDefinition( diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py index 3ff6e20c0028c..b1f9a5cd744a2 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py @@ -514,12 +514,12 @@ def test_pandas_output_intermediate_csv_files(): ), environment=config.Environment( solids={ - 'load_sum_table' : config.Solid( - {'path' : sum_file}, - ), - 'load_mult_table' : config.Solid( - {'path' : mult_file}, - ), + 'load_sum_table': config.Solid({ + 'path': sum_file + }, ), + 'load_mult_table': config.Solid({ + 'path': mult_file + }, ), }, ), ) From 852bbd78c4fe1d638edc7e745e0f9edfbdd971df Mon Sep 17 00:00:00 2001 From: Mikhail Novikov Date: Wed, 22 Aug 2018 14:04:19 +0200 Subject: [PATCH 013/103] 0.2.0 dagit (#91) Adapt dagit to work with 0.2.0 --- python_modules/dagit/dagit/schema.py | 135 ++++++------ python_modules/dagit/dagit/webapp/schema.json | 2 +- .../dagit/dagit/webapp/src/Argumented.tsx | 194 +++++++++--------- .../dagit/dagit/webapp/src/Pipeline.tsx | 27 ++- .../dagit/dagit/webapp/src/Solid.tsx | 82 ++++---- .../dagit/webapp/src/SolidTypeSignature.tsx | 16 +- .../dagit/webapp/src/graph/PipelineGraph.tsx | 19 +- .../dagit/webapp/src/graph/SolidNode.tsx | 68 +++--- .../webapp/src/graph/getFullSolidLayout.ts | 66 +++--- .../src/graph/types/PipelineGraphFragment.ts | 22 +- .../src/graph/types/SolidNodeFragment.ts | 22 +- .../dagit/dagit/webapp/src/types/AppQuery.ts | 56 ++--- .../webapp/src/types/PipelineFragment.ts | 56 ++--- .../webapp/src/types/PipelinesFragment.ts | 56 ++--- .../dagit/webapp/src/types/SolidFragment.ts | 56 ++--- .../webapp/src/types/SolidListItemFragment.ts | 9 +- .../src/types/SolidTypeSignatureFragment.ts | 9 +- python_modules/dagit/webapp/src/Arguments.tsx | 50 +++++ .../webapp/src/types/ArgumentsFragment.ts | 31 +++ .../dagster/dagster/core/definitions.py | 18 +- 20 files changed, 526 insertions(+), 468 deletions(-) create mode 100644 python_modules/dagit/webapp/src/Arguments.tsx create mode 100644 python_modules/dagit/webapp/src/types/ArgumentsFragment.ts diff --git a/python_modules/dagit/dagit/schema.py b/python_modules/dagit/dagit/schema.py index 2c4df434d9e4b..53f193edd5828 100644 --- a/python_modules/dagit/dagit/schema.py +++ b/python_modules/dagit/dagit/schema.py @@ -31,7 +31,12 @@ def __init__(self, pipeline): self._pipeline = pipeline def resolve_solids(self, info): - return [Solid(solid) for solid in self._pipeline.solids] + return [ + Solid( + solid, self._pipeline.dependency_structure.deps_of_solid_with_input(solid.name), + self._pipeline.dependency_structure.depended_by_of_solid(solid.name) + ) for solid in self._pipeline.solids + ] def resolve_context(self, info): return [ @@ -60,73 +65,102 @@ class Solid(graphene.ObjectType): name = graphene.NonNull(graphene.String) description = graphene.String() inputs = graphene.NonNull(graphene.List(lambda: graphene.NonNull(Input))) - output = graphene.Field(lambda: graphene.NonNull(Output)) + outputs = graphene.NonNull(graphene.List(lambda: graphene.NonNull(Output))) + config = graphene.NonNull(graphene.List(lambda: graphene.NonNull(Argument))) - def __init__(self, solid): + def __init__(self, solid, depends_on=None, depended_by=None): super(Solid, self).__init__(name=solid.name, description=solid.description) self._solid = solid + if depends_on: + self._depends_on = { + input_handle.input_def.name: output_handle + for input_handle, output_handle in depends_on.items() + } + else: + self.depends_on = {} + + if depended_by: + self._depended_by = { + output_handle.output.name: input_handles + for output_handle, input_handles in depended_by.items() + } + else: + self._depended_by = {} def resolve_inputs(self, info): - return [Input(input_definition) for input_definition in self._solid.inputs] + return [ + Input(input_definition, self, self._depends_on.get(input_definition.name)) + for input_definition in self._solid.inputs + ] + + def resolve_outputs(self, info): + return [ + Output(output_definition, self, self._depended_by.get(output_definition.name, [])) + for output_definition in self._solid.outputs + ] - def resolve_output(self, info): - return Output(self._solid.output) + def resolve_config(self, info): + return [ + Argument(name=name, argument=argument) + for name, argument in self._solid.config_dict_def.items() + ] class Input(graphene.ObjectType): + solid = graphene.NonNull(lambda: Solid) name = graphene.NonNull(graphene.String) description = graphene.String() type = graphene.NonNull(lambda: Type) - sources = graphene.NonNull(graphene.List(lambda: graphene.NonNull(Source))) - depends_on = graphene.Field(lambda: Solid) expectations = graphene.NonNull(graphene.List(lambda: graphene.NonNull(Expectation))) + depends_on = graphene.Field(lambda: Output) - def __init__(self, input_definition): + def __init__(self, input_definition, solid, depends_on): super(Input, self).__init__( - name=input_definition.name, description=input_definition.description + name=input_definition.name, + description=input_definition.description, + solid=solid, ) self._input_definition = input_definition + self._depends_on = depends_on def resolve_type(self, info): return Type(dagster_type=self._input_definition.dagster_type) - def resolve_sources(self, info): - return [Source(source) for source in self._input_definition.sources] - - def resolve_depends_on(self, info): - if self._input_definition.depends_on: - return Solid(self._input_definition.depends_on) - else: - return None - def resolve_expectations(self, info): if self._input_definition.expectations: return [Expectation(expectation for expectation in self._input_definition.expectations)] else: return [] + def resolve_depends_on(self, info): + return Output( + self._depends_on.output, + Solid(self._depends_on.solid), + # XXX(freiksenet): This is not right + [] + ) + class Output(graphene.ObjectType): + solid = graphene.NonNull(lambda: Solid) + name = graphene.NonNull(graphene.String) + description = graphene.String() type = graphene.NonNull(lambda: Type) - materializations = graphene.NonNull(graphene.List(lambda: graphene.NonNull(Materialization))) expectations = graphene.NonNull(graphene.List(lambda: graphene.NonNull(Expectation))) + depended_by = graphene.List(lambda: graphene.NonNull(Input)) - def __init__(self, output_definition): - super(Output, self).__init__() + def __init__(self, output_definition, solid, depended_by): + super(Output, self).__init__( + name=output_definition.name, + description=output_definition.description, + solid=solid, + ) self._output_definition = output_definition + self._depended_by = depended_by def resolve_type(self, info): return Type(dagster_type=self._output_definition.dagster_type) - def resolve_materializations(self, info): - if self._output_definition.materializations: - return [ - Materialization(materialization) - for materialization in self._output_definition.materializations - ] - else: - return [] - def resolve_expectations(self, info): if self._output_definition.expectations: return [ @@ -135,39 +169,14 @@ def resolve_expectations(self, info): else: return [] - -class Source(graphene.ObjectType): - # XXX(freiksenet): maybe rename to name? - source_type = graphene.NonNull(graphene.String) - description = graphene.String() - arguments = graphene.NonNull(graphene.List(lambda: graphene.NonNull(Argument))) - - def __init__(self, source): - super(Source, self).__init__(source_type=source.source_type, description=source.description) - self._source = source - - def resolve_arguments(self, info): - return [ - Argument(name=name, argument=argument) - for name, argument in self._source.argument_def_dict.items() - ] - - -class Materialization(graphene.ObjectType): - name = graphene.NonNull(graphene.String) - description = graphene.String() - arguments = graphene.NonNull(graphene.List(lambda: graphene.NonNull(Argument))) - - def __init__(self, materialization): - super(Materialization, self).__init__( - name=materialization.name, description=materialization.description - ) - self._materialization = materialization - - def resolve_arguments(self, info): + def resolve_depends_on(self, info): return [ - Argument(name=name, argument=argument) - for name, argument in self._materialization.argument_def_dict.items() + Input( + depended_by.input_def, + Solid(depended_by.solid), + # XXX(freiksenet): This is not right + None + ) for depended_by in self._depended_by ] diff --git a/python_modules/dagit/dagit/webapp/schema.json b/python_modules/dagit/dagit/webapp/schema.json index ccfa773456e57..caf47807e1813 100644 --- a/python_modules/dagit/dagit/webapp/schema.json +++ b/python_modules/dagit/dagit/webapp/schema.json @@ -1 +1 @@ -{"queryType":{"name":"Query"},"mutationType":null,"subscriptionType":null,"types":[{"kind":"OBJECT","name":"Query","description":null,"fields":[{"name":"pipeline","description":null,"args":[{"name":"name","description":null,"type":{"kind":"SCALAR","name":"String","ofType":null},"defaultValue":null}],"type":{"kind":"OBJECT","name":"Pipeline","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"pipelines","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Pipeline","ofType":null}}}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"SCALAR","name":"String","description":"The `String` scalar type represents textual data, represented as UTF-8 character sequences. The String type is most often used by GraphQL to represent free-form human-readable text.","fields":null,"inputFields":null,"interfaces":null,"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Pipeline","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"solids","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Solid","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"context","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"PipelineContext","ofType":null}}}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Solid","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"inputs","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Input","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"output","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Output","ofType":null}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Input","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"type","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Type","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"sources","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Source","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"dependsOn","description":null,"args":[],"type":{"kind":"OBJECT","name":"Solid","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"expectations","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Expectation","ofType":null}}}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Type","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Source","description":null,"fields":[{"name":"sourceType","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"arguments","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Argument","ofType":null}}}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Argument","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"type","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Type","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"isOptional","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"SCALAR","name":"Boolean","description":"The `Boolean` scalar type represents `true` or `false`.","fields":null,"inputFields":null,"interfaces":null,"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Expectation","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Output","description":null,"fields":[{"name":"type","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Type","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"materializations","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Materialization","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"expectations","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Expectation","ofType":null}}}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Materialization","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"arguments","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Argument","ofType":null}}}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"PipelineContext","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"arguments","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Argument","ofType":null}}}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"__Schema","description":"A GraphQL Schema defines the capabilities of a GraphQL server. It exposes all available types and directives on the server, as well as the entry points for query, mutation, and subscription operations.","fields":[{"name":"types","description":"A list of all types supported by this server.","args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Type","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"queryType","description":"The type that query operations will be rooted at.","args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Type","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"mutationType","description":"If this server supports mutation, the type that mutation operations will be rooted at.","args":[],"type":{"kind":"OBJECT","name":"__Type","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"subscriptionType","description":"If this server support subscription, the type that subscription operations will be rooted at.","args":[],"type":{"kind":"OBJECT","name":"__Type","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"directives","description":"A list of all directives supported by this server.","args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Directive","ofType":null}}}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"__Type","description":"The fundamental unit of any GraphQL Schema is the type. There are many kinds of types in GraphQL as represented by the `__TypeKind` enum.\n\nDepending on the kind of a type, certain fields describe information about that type. Scalar types provide no information beyond a name and description, while Enum types provide their values. Object and Interface types provide the fields they describe. Abstract types, Union and Interface, provide the Object types possible at runtime. List and NonNull types compose other types.","fields":[{"name":"kind","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"ENUM","name":"__TypeKind","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"name","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"fields","description":null,"args":[{"name":"includeDeprecated","description":null,"type":{"kind":"SCALAR","name":"Boolean","ofType":null},"defaultValue":"false"}],"type":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Field","ofType":null}}},"isDeprecated":false,"deprecationReason":null},{"name":"interfaces","description":null,"args":[],"type":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Type","ofType":null}}},"isDeprecated":false,"deprecationReason":null},{"name":"possibleTypes","description":null,"args":[],"type":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Type","ofType":null}}},"isDeprecated":false,"deprecationReason":null},{"name":"enumValues","description":null,"args":[{"name":"includeDeprecated","description":null,"type":{"kind":"SCALAR","name":"Boolean","ofType":null},"defaultValue":"false"}],"type":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__EnumValue","ofType":null}}},"isDeprecated":false,"deprecationReason":null},{"name":"inputFields","description":null,"args":[],"type":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__InputValue","ofType":null}}},"isDeprecated":false,"deprecationReason":null},{"name":"ofType","description":null,"args":[],"type":{"kind":"OBJECT","name":"__Type","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"ENUM","name":"__TypeKind","description":"An enum describing what kind of type a given `__Type` is.","fields":null,"inputFields":null,"interfaces":null,"enumValues":[{"name":"SCALAR","description":"Indicates this type is a scalar.","isDeprecated":false,"deprecationReason":null},{"name":"OBJECT","description":"Indicates this type is an object. `fields` and `interfaces` are valid fields.","isDeprecated":false,"deprecationReason":null},{"name":"INTERFACE","description":"Indicates this type is an interface. `fields` and `possibleTypes` are valid fields.","isDeprecated":false,"deprecationReason":null},{"name":"UNION","description":"Indicates this type is a union. `possibleTypes` is a valid field.","isDeprecated":false,"deprecationReason":null},{"name":"ENUM","description":"Indicates this type is an enum. `enumValues` is a valid field.","isDeprecated":false,"deprecationReason":null},{"name":"INPUT_OBJECT","description":"Indicates this type is an input object. `inputFields` is a valid field.","isDeprecated":false,"deprecationReason":null},{"name":"LIST","description":"Indicates this type is a list. `ofType` is a valid field.","isDeprecated":false,"deprecationReason":null},{"name":"NON_NULL","description":"Indicates this type is a non-null. `ofType` is a valid field.","isDeprecated":false,"deprecationReason":null}],"possibleTypes":null},{"kind":"OBJECT","name":"__Field","description":"Object and Interface types are described by a list of Fields, each of which has a name, potentially a list of arguments, and a return type.","fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"args","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__InputValue","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"type","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Type","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"isDeprecated","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"deprecationReason","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"__InputValue","description":"Arguments provided to Fields or Directives and the input fields of an InputObject are represented as Input Values which describe their type and optionally a default value.","fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"type","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Type","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"defaultValue","description":"A GraphQL-formatted string representing the default value for this input value.","args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"__EnumValue","description":"One possible value for a given Enum. Enum values are unique values, not a placeholder for a string or numeric value. However an Enum value is returned in a JSON response as a string.","fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"isDeprecated","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"deprecationReason","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"__Directive","description":"A Directive provides a way to describe alternate runtime execution and type validation behavior in a GraphQL document.\n\nIn some cases, you need to provide options to alter GraphQL's execution behavior in ways field arguments will not suffice, such as conditionally including or skipping a field. Directives provide this by describing additional information to the executor.","fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"locations","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"ENUM","name":"__DirectiveLocation","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"args","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__InputValue","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"onOperation","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"isDeprecated":true,"deprecationReason":"Use `locations`."},{"name":"onFragment","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"isDeprecated":true,"deprecationReason":"Use `locations`."},{"name":"onField","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"isDeprecated":true,"deprecationReason":"Use `locations`."}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"ENUM","name":"__DirectiveLocation","description":"A Directive can be adjacent to many parts of the GraphQL language, a __DirectiveLocation describes one such possible adjacencies.","fields":null,"inputFields":null,"interfaces":null,"enumValues":[{"name":"QUERY","description":"Location adjacent to a query operation.","isDeprecated":false,"deprecationReason":null},{"name":"MUTATION","description":"Location adjacent to a mutation operation.","isDeprecated":false,"deprecationReason":null},{"name":"SUBSCRIPTION","description":"Location adjacent to a subscription operation.","isDeprecated":false,"deprecationReason":null},{"name":"FIELD","description":"Location adjacent to a field.","isDeprecated":false,"deprecationReason":null},{"name":"FRAGMENT_DEFINITION","description":"Location adjacent to a fragment definition.","isDeprecated":false,"deprecationReason":null},{"name":"FRAGMENT_SPREAD","description":"Location adjacent to a fragment spread.","isDeprecated":false,"deprecationReason":null},{"name":"INLINE_FRAGMENT","description":"Location adjacent to an inline fragment.","isDeprecated":false,"deprecationReason":null},{"name":"SCHEMA","description":"Location adjacent to a schema definition.","isDeprecated":false,"deprecationReason":null},{"name":"SCALAR","description":"Location adjacent to a scalar definition.","isDeprecated":false,"deprecationReason":null},{"name":"OBJECT","description":"Location adjacent to an object type definition.","isDeprecated":false,"deprecationReason":null},{"name":"FIELD_DEFINITION","description":"Location adjacent to a field definition.","isDeprecated":false,"deprecationReason":null},{"name":"ARGUMENT_DEFINITION","description":"Location adjacent to an argument definition.","isDeprecated":false,"deprecationReason":null},{"name":"INTERFACE","description":"Location adjacent to an interface definition.","isDeprecated":false,"deprecationReason":null},{"name":"UNION","description":"Location adjacent to a union definition.","isDeprecated":false,"deprecationReason":null},{"name":"ENUM","description":"Location adjacent to an enum definition.","isDeprecated":false,"deprecationReason":null},{"name":"ENUM_VALUE","description":"Location adjacent to an enum value definition.","isDeprecated":false,"deprecationReason":null},{"name":"INPUT_OBJECT","description":"Location adjacent to an input object type definition.","isDeprecated":false,"deprecationReason":null},{"name":"INPUT_FIELD_DEFINITION","description":"Location adjacent to an input object field definition.","isDeprecated":false,"deprecationReason":null}],"possibleTypes":null}],"directives":[{"name":"include","description":"Directs the executor to include this field or fragment only when the `if` argument is true.","locations":["FIELD","FRAGMENT_SPREAD","INLINE_FRAGMENT"],"args":[{"name":"if","description":"Included when true.","type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"defaultValue":null}]},{"name":"skip","description":"Directs the executor to skip this field or fragment when the `if` argument is true.","locations":["FIELD","FRAGMENT_SPREAD","INLINE_FRAGMENT"],"args":[{"name":"if","description":"Skipped when true.","type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"defaultValue":null}]}]} \ No newline at end of file +{"queryType":{"name":"Query"},"mutationType":null,"subscriptionType":null,"types":[{"kind":"OBJECT","name":"Query","description":null,"fields":[{"name":"pipeline","description":null,"args":[{"name":"name","description":null,"type":{"kind":"SCALAR","name":"String","ofType":null},"defaultValue":null}],"type":{"kind":"OBJECT","name":"Pipeline","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"pipelines","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Pipeline","ofType":null}}}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"SCALAR","name":"String","description":"The `String` scalar type represents textual data, represented as UTF-8 character sequences. The String type is most often used by GraphQL to represent free-form human-readable text.","fields":null,"inputFields":null,"interfaces":null,"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Pipeline","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"solids","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Solid","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"context","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"PipelineContext","ofType":null}}}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Solid","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"inputs","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Input","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"outputs","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Output","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"config","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Argument","ofType":null}}}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Input","description":null,"fields":[{"name":"solid","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Solid","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"type","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Type","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"expectations","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Expectation","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"dependsOn","description":null,"args":[],"type":{"kind":"OBJECT","name":"Output","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Type","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Expectation","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Output","description":null,"fields":[{"name":"solid","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Solid","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"type","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Type","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"expectations","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Expectation","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"dependedBy","description":null,"args":[],"type":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Input","ofType":null}}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Argument","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"type","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Type","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"isOptional","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"SCALAR","name":"Boolean","description":"The `Boolean` scalar type represents `true` or `false`.","fields":null,"inputFields":null,"interfaces":null,"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"PipelineContext","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"arguments","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Argument","ofType":null}}}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"__Schema","description":"A GraphQL Schema defines the capabilities of a GraphQL server. It exposes all available types and directives on the server, as well as the entry points for query, mutation, and subscription operations.","fields":[{"name":"types","description":"A list of all types supported by this server.","args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Type","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"queryType","description":"The type that query operations will be rooted at.","args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Type","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"mutationType","description":"If this server supports mutation, the type that mutation operations will be rooted at.","args":[],"type":{"kind":"OBJECT","name":"__Type","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"subscriptionType","description":"If this server support subscription, the type that subscription operations will be rooted at.","args":[],"type":{"kind":"OBJECT","name":"__Type","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"directives","description":"A list of all directives supported by this server.","args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Directive","ofType":null}}}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"__Type","description":"The fundamental unit of any GraphQL Schema is the type. There are many kinds of types in GraphQL as represented by the `__TypeKind` enum.\n\nDepending on the kind of a type, certain fields describe information about that type. Scalar types provide no information beyond a name and description, while Enum types provide their values. Object and Interface types provide the fields they describe. Abstract types, Union and Interface, provide the Object types possible at runtime. List and NonNull types compose other types.","fields":[{"name":"kind","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"ENUM","name":"__TypeKind","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"name","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"fields","description":null,"args":[{"name":"includeDeprecated","description":null,"type":{"kind":"SCALAR","name":"Boolean","ofType":null},"defaultValue":"false"}],"type":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Field","ofType":null}}},"isDeprecated":false,"deprecationReason":null},{"name":"interfaces","description":null,"args":[],"type":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Type","ofType":null}}},"isDeprecated":false,"deprecationReason":null},{"name":"possibleTypes","description":null,"args":[],"type":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Type","ofType":null}}},"isDeprecated":false,"deprecationReason":null},{"name":"enumValues","description":null,"args":[{"name":"includeDeprecated","description":null,"type":{"kind":"SCALAR","name":"Boolean","ofType":null},"defaultValue":"false"}],"type":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__EnumValue","ofType":null}}},"isDeprecated":false,"deprecationReason":null},{"name":"inputFields","description":null,"args":[],"type":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__InputValue","ofType":null}}},"isDeprecated":false,"deprecationReason":null},{"name":"ofType","description":null,"args":[],"type":{"kind":"OBJECT","name":"__Type","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"ENUM","name":"__TypeKind","description":"An enum describing what kind of type a given `__Type` is.","fields":null,"inputFields":null,"interfaces":null,"enumValues":[{"name":"SCALAR","description":"Indicates this type is a scalar.","isDeprecated":false,"deprecationReason":null},{"name":"OBJECT","description":"Indicates this type is an object. `fields` and `interfaces` are valid fields.","isDeprecated":false,"deprecationReason":null},{"name":"INTERFACE","description":"Indicates this type is an interface. `fields` and `possibleTypes` are valid fields.","isDeprecated":false,"deprecationReason":null},{"name":"UNION","description":"Indicates this type is a union. `possibleTypes` is a valid field.","isDeprecated":false,"deprecationReason":null},{"name":"ENUM","description":"Indicates this type is an enum. `enumValues` is a valid field.","isDeprecated":false,"deprecationReason":null},{"name":"INPUT_OBJECT","description":"Indicates this type is an input object. `inputFields` is a valid field.","isDeprecated":false,"deprecationReason":null},{"name":"LIST","description":"Indicates this type is a list. `ofType` is a valid field.","isDeprecated":false,"deprecationReason":null},{"name":"NON_NULL","description":"Indicates this type is a non-null. `ofType` is a valid field.","isDeprecated":false,"deprecationReason":null}],"possibleTypes":null},{"kind":"OBJECT","name":"__Field","description":"Object and Interface types are described by a list of Fields, each of which has a name, potentially a list of arguments, and a return type.","fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"args","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__InputValue","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"type","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Type","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"isDeprecated","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"deprecationReason","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"__InputValue","description":"Arguments provided to Fields or Directives and the input fields of an InputObject are represented as Input Values which describe their type and optionally a default value.","fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"type","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Type","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"defaultValue","description":"A GraphQL-formatted string representing the default value for this input value.","args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"__EnumValue","description":"One possible value for a given Enum. Enum values are unique values, not a placeholder for a string or numeric value. However an Enum value is returned in a JSON response as a string.","fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"isDeprecated","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"deprecationReason","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"__Directive","description":"A Directive provides a way to describe alternate runtime execution and type validation behavior in a GraphQL document.\n\nIn some cases, you need to provide options to alter GraphQL's execution behavior in ways field arguments will not suffice, such as conditionally including or skipping a field. Directives provide this by describing additional information to the executor.","fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"locations","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"ENUM","name":"__DirectiveLocation","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"args","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__InputValue","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"onOperation","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"isDeprecated":true,"deprecationReason":"Use `locations`."},{"name":"onFragment","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"isDeprecated":true,"deprecationReason":"Use `locations`."},{"name":"onField","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"isDeprecated":true,"deprecationReason":"Use `locations`."}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"ENUM","name":"__DirectiveLocation","description":"A Directive can be adjacent to many parts of the GraphQL language, a __DirectiveLocation describes one such possible adjacencies.","fields":null,"inputFields":null,"interfaces":null,"enumValues":[{"name":"QUERY","description":"Location adjacent to a query operation.","isDeprecated":false,"deprecationReason":null},{"name":"MUTATION","description":"Location adjacent to a mutation operation.","isDeprecated":false,"deprecationReason":null},{"name":"SUBSCRIPTION","description":"Location adjacent to a subscription operation.","isDeprecated":false,"deprecationReason":null},{"name":"FIELD","description":"Location adjacent to a field.","isDeprecated":false,"deprecationReason":null},{"name":"FRAGMENT_DEFINITION","description":"Location adjacent to a fragment definition.","isDeprecated":false,"deprecationReason":null},{"name":"FRAGMENT_SPREAD","description":"Location adjacent to a fragment spread.","isDeprecated":false,"deprecationReason":null},{"name":"INLINE_FRAGMENT","description":"Location adjacent to an inline fragment.","isDeprecated":false,"deprecationReason":null},{"name":"SCHEMA","description":"Location adjacent to a schema definition.","isDeprecated":false,"deprecationReason":null},{"name":"SCALAR","description":"Location adjacent to a scalar definition.","isDeprecated":false,"deprecationReason":null},{"name":"OBJECT","description":"Location adjacent to an object type definition.","isDeprecated":false,"deprecationReason":null},{"name":"FIELD_DEFINITION","description":"Location adjacent to a field definition.","isDeprecated":false,"deprecationReason":null},{"name":"ARGUMENT_DEFINITION","description":"Location adjacent to an argument definition.","isDeprecated":false,"deprecationReason":null},{"name":"INTERFACE","description":"Location adjacent to an interface definition.","isDeprecated":false,"deprecationReason":null},{"name":"UNION","description":"Location adjacent to a union definition.","isDeprecated":false,"deprecationReason":null},{"name":"ENUM","description":"Location adjacent to an enum definition.","isDeprecated":false,"deprecationReason":null},{"name":"ENUM_VALUE","description":"Location adjacent to an enum value definition.","isDeprecated":false,"deprecationReason":null},{"name":"INPUT_OBJECT","description":"Location adjacent to an input object type definition.","isDeprecated":false,"deprecationReason":null},{"name":"INPUT_FIELD_DEFINITION","description":"Location adjacent to an input object field definition.","isDeprecated":false,"deprecationReason":null}],"possibleTypes":null}],"directives":[{"name":"include","description":"Directs the executor to include this field or fragment only when the `if` argument is true.","locations":["FIELD","FRAGMENT_SPREAD","INLINE_FRAGMENT"],"args":[{"name":"if","description":"Included when true.","type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"defaultValue":null}]},{"name":"skip","description":"Directs the executor to skip this field or fragment when the `if` argument is true.","locations":["FIELD","FRAGMENT_SPREAD","INLINE_FRAGMENT"],"args":[{"name":"if","description":"Skipped when true.","type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"defaultValue":null}]}]} \ No newline at end of file diff --git a/python_modules/dagit/dagit/webapp/src/Argumented.tsx b/python_modules/dagit/dagit/webapp/src/Argumented.tsx index 48f5ff1f040ae..96f14e818b245 100644 --- a/python_modules/dagit/dagit/webapp/src/Argumented.tsx +++ b/python_modules/dagit/dagit/webapp/src/Argumented.tsx @@ -1,104 +1,104 @@ -import * as React from "react"; -import gql from "graphql-tag"; -import styled from "styled-components"; -import { H5, H6, Text, UL, Code, Collapse } from "@blueprintjs/core"; -import SpacedCard from "./SpacedCard"; -import TypeWithTooltip from "./TypeWithTooltip"; -import Description from "./Description"; -import { SourceFragment } from "./types/SourceFragment"; -import { MaterializationFragment } from "./types/MaterializationFragment"; -import { PipelineContextFragment } from "./types/PipelineContextFragment"; +// import * as React from "react"; +// import gql from "graphql-tag"; +// import styled from "styled-components"; +// import { H5, H6, Text, UL, Code, Collapse } from "@blueprintjs/core"; +// import SpacedCard from "./SpacedCard"; +// import TypeWithTooltip from "./TypeWithTooltip"; +// import Description from "./Description"; +// import { SourceFragment } from "./types/SourceFragment"; +// import { MaterializationFragment } from "./types/MaterializationFragment"; +// import { PipelineContextFragment } from "./types/PipelineContextFragment"; -interface IArgumentedProps { - // XXX(freiksenet): Fix - item: SourceFragment & MaterializationFragment & PipelineContextFragment; - renderCard?: (props: any) => React.ReactNode; -} +// interface IArgumentedProps { +// // XXX(freiksenet): Fix +// item: SourceFragment & MaterializationFragment & PipelineContextFragment; +// renderCard?: (props: any) => React.ReactNode; +// } -export default class Argumented extends React.Component { - static fragments = { - SourceFragment: gql` - fragment SourceFragment on Source { - name: sourceType - description - arguments { - name - description - type { - ...TypeFragment - } - isOptional - } - } +// export default class Argumented extends React.Component { +// static fragments = { +// SourceFragment: gql` +// fragment SourceFragment on Source { +// name: sourceType +// description +// arguments { +// name +// description +// type { +// ...TypeFragment +// } +// isOptional +// } +// } - ${TypeWithTooltip.fragments.TypeFragment} - `, - MaterializationFragment: gql` - fragment MaterializationFragment on Materialization { - name - description - arguments { - name - description - type { - ...TypeFragment - } - isOptional - } - } +// ${TypeWithTooltip.fragments.TypeFragment} +// `, +// MaterializationFragment: gql` +// fragment MaterializationFragment on Materialization { +// name +// description +// arguments { +// name +// description +// type { +// ...TypeFragment +// } +// isOptional +// } +// } - ${TypeWithTooltip.fragments.TypeFragment} - `, - PipelineContextFragment: gql` - fragment PipelineContextFragment on PipelineContext { - name - description - arguments { - name - description - type { - ...TypeFragment - } - isOptional - } - } +// ${TypeWithTooltip.fragments.TypeFragment} +// `, +// PipelineContextFragment: gql` +// fragment PipelineContextFragment on PipelineContext { +// name +// description +// arguments { +// name +// description +// type { +// ...TypeFragment +// } +// isOptional +// } +// } - ${TypeWithTooltip.fragments.TypeFragment} - ` - }; +// ${TypeWithTooltip.fragments.TypeFragment} +// ` +// }; - public render() { - const renderCard = - this.props.renderCard || ((props: any) => ); - return renderCard({ - elevation: 2, - children: ( - <> -
- {this.props.item.name} -
- - - -
Arguments
-
    - {this.props.item.arguments.map((argument: any, i: number) => ( -
  • - {argument.name} {argument.isOptional ? "(optional)" : null}{" "} - - - - -
  • - ))} -
- - ) - }); - } -} +// public render() { +// const renderCard = +// this.props.renderCard || ((props: any) => ); +// return renderCard({ +// elevation: 2, +// children: ( +// <> +//
+// {this.props.item.name} +//
+// +// +// +//
Arguments
+//
    +// {this.props.item.arguments.map((argument: any, i: number) => ( +//
  • +// {argument.name} {argument.isOptional ? "(optional)" : null}{" "} +// +// +// +// +//
  • +// ))} +//
+// +// ) +// }); +// } +// } -const DescriptionWrapper = styled.div` - max-width: 400px; - margin-bottom: 10px; -`; +// const DescriptionWrapper = styled.div` +// max-width: 400px; +// margin-bottom: 10px; +// `; diff --git a/python_modules/dagit/dagit/webapp/src/Pipeline.tsx b/python_modules/dagit/dagit/webapp/src/Pipeline.tsx index 6966beefb646d..b3345175e6cf8 100644 --- a/python_modules/dagit/dagit/webapp/src/Pipeline.tsx +++ b/python_modules/dagit/dagit/webapp/src/Pipeline.tsx @@ -4,9 +4,9 @@ import styled from "styled-components"; import { History } from "history"; import { Switch, Route, match } from "react-router"; import { Link } from "react-router-dom"; -import { Card, H2, H5, Text, Code, UL } from "@blueprintjs/core"; +import { Card, H2, H5, Text, Code, UL, H6 } from "@blueprintjs/core"; import SpacedCard from "./SpacedCard"; -import Argumented from "./Argumented"; +import Arguments from "./Arguments"; import Solid from "./Solid"; import PipelineGraph from "./graph/PipelineGraph"; import { Breadcrumbs, Breadcrumb } from "./Breadcrumbs"; @@ -32,15 +32,19 @@ export default class Pipeline extends React.Component { ...SolidListItemFragment } context { - ...PipelineContextFragment + name + description + arguments { + ...ArgumentsFragment + } } ...PipelineGraphFragment } ${Solid.fragments.SolidFragment} - ${Argumented.fragments.PipelineContextFragment} ${PipelineGraph.fragments.PipelineGraphFragment} ${SolidListItem.fragments.SolidListItemFragment} + ${Arguments.fragments.ArgumentsFragment} ` }; @@ -98,11 +102,16 @@ export default class Pipeline extends React.Component { renderContext() { return this.props.pipeline.context.map((context: any, i: number) => ( - } - /> + +
+ {context.name} +
+ + + +
Arguments
+ +
)); } diff --git a/python_modules/dagit/dagit/webapp/src/Solid.tsx b/python_modules/dagit/dagit/webapp/src/Solid.tsx index 001ead942772e..8227309ff49a8 100644 --- a/python_modules/dagit/dagit/webapp/src/Solid.tsx +++ b/python_modules/dagit/dagit/webapp/src/Solid.tsx @@ -3,7 +3,7 @@ import gql from "graphql-tag"; import styled from "styled-components"; import { Link } from "react-router-dom"; import { H5, H6, Text, Colors, Code, UL } from "@blueprintjs/core"; -import Argumented from "./Argumented"; +import Arguments from "./Arguments"; import SpacedCard from "./SpacedCard"; import SolidTypeSignature from "./SolidTypeSignature"; import TypeWithTooltip from "./TypeWithTooltip"; @@ -21,7 +21,12 @@ export default class Solid extends React.Component { ...SolidTypeSignatureFragment name description + config { + ...ArgumentsFragment + } inputs { + name + description type { ...TypeFragment } @@ -29,16 +34,16 @@ export default class Solid extends React.Component { name description } - name - description - sources { - ...SourceFragment - } dependsOn { name + solid { + name + } } } - output { + outputs { + name + description type { ...TypeFragment } @@ -46,9 +51,6 @@ export default class Solid extends React.Component { name description } - materializations { - ...MaterializationFragment - } expectations { name description @@ -56,13 +58,25 @@ export default class Solid extends React.Component { } } - ${Argumented.fragments.SourceFragment} - ${Argumented.fragments.MaterializationFragment} ${TypeWithTooltip.fragments.TypeFragment} ${SolidTypeSignature.fragments.SolidTypeSignatureFragment} + ${Arguments.fragments.ArgumentsFragment} ` }; + renderConfig() { + if (this.props.solid.config.length > 0) { + return ( + <> +
Config
+ + + ); + } else { + return null; + } + } + renderInputs() { return this.props.solid.inputs.map((input, i: number) => ( @@ -90,50 +104,31 @@ export default class Solid extends React.Component { ))} - {input.sources.length > 0 ?
Sources
: null} - {input.sources.map((source: any, i: number) => ( - } - /> - ))}
)); } - renderOutput() { - return ( - -
Output
+ renderOutputs() { + return this.props.solid.outputs.map((output, i: number) => ( + +
+ Input {output.name} +
- + - {this.props.solid.output.expectations.length > 0 ? ( -
Expectations
- ) : null} + + {output.expectations.length > 0 ?
Expectations
: null}
    - {this.props.solid.output.expectations.map((expectation, i) => ( + {output.expectations.map((expectation, i) => (
  • {expectation.name}
  • ))}
- {this.props.solid.output.materializations.length > 0 ? ( -
Materializations
- ) : null} - {this.props.solid.output.materializations.map( - (materialization: any, i: number) => ( - } - /> - ) - )}
- ); + )); } public render() { @@ -148,10 +143,11 @@ export default class Solid extends React.Component { + {this.renderConfig()} {this.renderInputs()} - {this.renderOutput()} + {this.renderOutputs()}
); diff --git a/python_modules/dagit/dagit/webapp/src/SolidTypeSignature.tsx b/python_modules/dagit/dagit/webapp/src/SolidTypeSignature.tsx index 93a89461476c2..77029f0a51122 100644 --- a/python_modules/dagit/dagit/webapp/src/SolidTypeSignature.tsx +++ b/python_modules/dagit/dagit/webapp/src/SolidTypeSignature.tsx @@ -16,7 +16,8 @@ export default class SolidTypeSignature extends React.Component< static fragments = { SolidTypeSignatureFragment: gql` fragment SolidTypeSignatureFragment on Solid { - output { + outputs { + name type { ...TypeFragment } @@ -35,19 +36,20 @@ export default class SolidTypeSignature extends React.Component< render() { const inputSide = this.props.solid.inputs.map((input, i) => ( - + {input.name}: {i < this.props.solid.inputs.length - 1 ? ", " : ""} )); - const outputSide = ( - - + const outputSide = this.props.solid.outputs.map((output, i) => ( + + {output.name}: + {i < this.props.solid.outputs.length - 1 ? ", " : ""} - ); + )); return ( - ({inputSide}) ⇒ {outputSide} + ({inputSide}) ⇒ ({outputSide}) ); } diff --git a/python_modules/dagit/dagit/webapp/src/graph/PipelineGraph.tsx b/python_modules/dagit/dagit/webapp/src/graph/PipelineGraph.tsx index 8f4be2b65890c..82f88b9fdfe7d 100644 --- a/python_modules/dagit/dagit/webapp/src/graph/PipelineGraph.tsx +++ b/python_modules/dagit/dagit/webapp/src/graph/PipelineGraph.tsx @@ -51,7 +51,7 @@ export default class PipelineGraph extends React.Component< renderConnections(layout: IFullPipelineLayout) { const connections: Array<{ - from: string; + from: { solidName: string; outputName: string }; to: { solidName: string; inputName: string }; }> = []; @@ -59,7 +59,10 @@ export default class PipelineGraph extends React.Component< solid.inputs.forEach(input => { if (input.dependsOn) { connections.push({ - from: input.dependsOn.name, + from: { + solidName: input.dependsOn.solid.name, + outputName: input.dependsOn.name + }, to: { solidName: solid.name, inputName: input.name @@ -70,12 +73,18 @@ export default class PipelineGraph extends React.Component< }); const links = connections.map( - ({ from, to: { solidName, inputName } }, i) => ( + ( + { + from: { solidName: outputSolidName, outputName }, + to: { solidName: inputSolidName, inputName } + }, + i + ) => ( d.x} y={(d: { x: number; y: number }) => d.y} diff --git a/python_modules/dagit/dagit/webapp/src/graph/SolidNode.tsx b/python_modules/dagit/dagit/webapp/src/graph/SolidNode.tsx index 4804b9a5fe53f..5d133fdc4e416 100644 --- a/python_modules/dagit/dagit/webapp/src/graph/SolidNode.tsx +++ b/python_modules/dagit/dagit/webapp/src/graph/SolidNode.tsx @@ -23,20 +23,18 @@ export default class SolidNode extends React.Component { type { name } - sources { - name: sourceType - } dependsOn { name + solid { + name + } } } - output { + outputs { + name type { name } - materializations { - name - } expectations { name description @@ -83,41 +81,43 @@ export default class SolidNode extends React.Component { }); } - renderOutput() { - const { - layout: { x, y, width, height } - } = this.props.layout.output; + renderOutputs() { + return this.props.solid.outputs.map((output, i) => { + const { + layout: { x, y, width, height } + } = this.props.layout.outputs[output.name]; - return ( - - -
({this.props.solid.output.type.name})
-
-
- ); + +
+ {output.name} ({output.type.name}) +
+
+ + ); + }); } renderSelectedBox() { if (this.props.selected) { - const width = - this.props.layout.solid.width + - this.props.layout.output.layout.width * 2 + - 20; + const width = this.props.layout.solid.width + 200 * 2 + 20; const height = this.props.layout.solid.height + 20; return ( { {this.renderInputs()} - {this.renderOutput()} + {this.renderOutputs()} ); } diff --git a/python_modules/dagit/dagit/webapp/src/graph/getFullSolidLayout.ts b/python_modules/dagit/dagit/webapp/src/graph/getFullSolidLayout.ts index 60c282df469e5..88c4cc81ddb0a 100644 --- a/python_modules/dagit/dagit/webapp/src/graph/getFullSolidLayout.ts +++ b/python_modules/dagit/dagit/webapp/src/graph/getFullSolidLayout.ts @@ -16,9 +16,11 @@ export interface IFullSolidLayout { port: IPoint; }; }; - output: { - layout: ILayout; - port: IPoint; + outputs: { + [outputName: string]: { + layout: ILayout; + port: IPoint; + }; }; } @@ -32,8 +34,14 @@ interface ILayoutSolid { name: string; dependsOn: { name: string; + solid: { + name: string; + }; } | null; }>; + outputs: Array<{ + name: string; + }>; } export interface ILayout { @@ -76,15 +84,12 @@ export function getDagrePipelineLayout( const layout = layoutSolid({ solid, x: 0, y: 0 }); g.setNode(solid.name, { height: layout.solid.height, - width: - layout.solid.width + - layout.output.layout.width * 2 - - INPUT_OUTPUT_INSET * 2 + width: layout.solid.width + INPUT_WIDTH * 2 - INPUT_OUTPUT_INSET * 2 }); solid.inputs.forEach(input => { if (input.dependsOn) { - g.setEdge(input.dependsOn.name, solid.name); + g.setEdge(input.dependsOn.solid.name, solid.name); } }); }); @@ -131,19 +136,10 @@ function layoutSolid({ x: solidX, y: solidY, width: SOLID_WIDTH, - height: SOLID_BASE_HEIGHT + (INPUT_HEIGHT + INPUT_GAP) * solid.inputs.length - }; - const outputX = solidX + SOLID_WIDTH - INPUT_OUTPUT_INSET; - const outputY = solidY + INPUT_GAP; - const outputLayout: ILayout = { - x: outputX, - y: outputY, - width: OUTPUT_WIDTH, - height: OUTPUT_HEIGHT - }; - const outputPort: IPoint = { - x: outputX + OUTPUT_WIDTH, - y: outputY + OUTPUT_HEIGHT / 2 + height: + SOLID_BASE_HEIGHT + + (INPUT_HEIGHT + INPUT_GAP) * + Math.max(solid.inputs.length, solid.outputs.length) }; const inputs: { [inputName: string]: { @@ -168,12 +164,32 @@ function layoutSolid({ }; }); + const outputs: { + [outputName: string]: { + layout: ILayout; + port: IPoint; + }; + } = {}; + solid.outputs.forEach((output, i) => { + const outputX = solidX + SOLID_WIDTH - INPUT_OUTPUT_INSET; + const outputY = solidY + INPUT_GAP + (OUTPUT_HEIGHT + INPUT_GAP) * i; + outputs[output.name] = { + layout: { + x: outputX, + y: outputY, + width: OUTPUT_WIDTH, + height: OUTPUT_HEIGHT + }, + port: { + x: outputX, + y: outputY + OUTPUT_HEIGHT / 2 + } + }; + }); + return { solid: solidLayout, inputs, - output: { - layout: outputLayout, - port: outputPort - } + outputs }; } diff --git a/python_modules/dagit/dagit/webapp/src/graph/types/PipelineGraphFragment.ts b/python_modules/dagit/dagit/webapp/src/graph/types/PipelineGraphFragment.ts index 62c158053dc96..2747ecd0ec630 100644 --- a/python_modules/dagit/dagit/webapp/src/graph/types/PipelineGraphFragment.ts +++ b/python_modules/dagit/dagit/webapp/src/graph/types/PipelineGraphFragment.ts @@ -11,44 +11,40 @@ export interface PipelineGraphFragment_solids_inputs_type { name: string; } -export interface PipelineGraphFragment_solids_inputs_sources { +export interface PipelineGraphFragment_solids_inputs_dependsOn_solid { name: string; } export interface PipelineGraphFragment_solids_inputs_dependsOn { name: string; + solid: PipelineGraphFragment_solids_inputs_dependsOn_solid; } export interface PipelineGraphFragment_solids_inputs { name: string; type: PipelineGraphFragment_solids_inputs_type; - sources: PipelineGraphFragment_solids_inputs_sources[]; dependsOn: PipelineGraphFragment_solids_inputs_dependsOn | null; } -export interface PipelineGraphFragment_solids_output_type { +export interface PipelineGraphFragment_solids_outputs_type { name: string; } -export interface PipelineGraphFragment_solids_output_materializations { - name: string; -} - -export interface PipelineGraphFragment_solids_output_expectations { +export interface PipelineGraphFragment_solids_outputs_expectations { name: string; description: string | null; } -export interface PipelineGraphFragment_solids_output { - type: PipelineGraphFragment_solids_output_type; - materializations: PipelineGraphFragment_solids_output_materializations[]; - expectations: PipelineGraphFragment_solids_output_expectations[]; +export interface PipelineGraphFragment_solids_outputs { + name: string; + type: PipelineGraphFragment_solids_outputs_type; + expectations: PipelineGraphFragment_solids_outputs_expectations[]; } export interface PipelineGraphFragment_solids { name: string; inputs: PipelineGraphFragment_solids_inputs[]; - output: PipelineGraphFragment_solids_output; + outputs: PipelineGraphFragment_solids_outputs[]; } export interface PipelineGraphFragment { diff --git a/python_modules/dagit/dagit/webapp/src/graph/types/SolidNodeFragment.ts b/python_modules/dagit/dagit/webapp/src/graph/types/SolidNodeFragment.ts index 057b5fbdf7b3b..9eb3ab0aefef9 100644 --- a/python_modules/dagit/dagit/webapp/src/graph/types/SolidNodeFragment.ts +++ b/python_modules/dagit/dagit/webapp/src/graph/types/SolidNodeFragment.ts @@ -11,44 +11,40 @@ export interface SolidNodeFragment_inputs_type { name: string; } -export interface SolidNodeFragment_inputs_sources { +export interface SolidNodeFragment_inputs_dependsOn_solid { name: string; } export interface SolidNodeFragment_inputs_dependsOn { name: string; + solid: SolidNodeFragment_inputs_dependsOn_solid; } export interface SolidNodeFragment_inputs { name: string; type: SolidNodeFragment_inputs_type; - sources: SolidNodeFragment_inputs_sources[]; dependsOn: SolidNodeFragment_inputs_dependsOn | null; } -export interface SolidNodeFragment_output_type { +export interface SolidNodeFragment_outputs_type { name: string; } -export interface SolidNodeFragment_output_materializations { - name: string; -} - -export interface SolidNodeFragment_output_expectations { +export interface SolidNodeFragment_outputs_expectations { name: string; description: string | null; } -export interface SolidNodeFragment_output { - type: SolidNodeFragment_output_type; - materializations: SolidNodeFragment_output_materializations[]; - expectations: SolidNodeFragment_output_expectations[]; +export interface SolidNodeFragment_outputs { + name: string; + type: SolidNodeFragment_outputs_type; + expectations: SolidNodeFragment_outputs_expectations[]; } export interface SolidNodeFragment { name: string; inputs: SolidNodeFragment_inputs[]; - output: SolidNodeFragment_output; + outputs: SolidNodeFragment_outputs[]; } /* tslint:disable */ diff --git a/python_modules/dagit/dagit/webapp/src/types/AppQuery.ts b/python_modules/dagit/dagit/webapp/src/types/AppQuery.ts index 6c4e3b9a9ebda..fbc2706651ec1 100644 --- a/python_modules/dagit/dagit/webapp/src/types/AppQuery.ts +++ b/python_modules/dagit/dagit/webapp/src/types/AppQuery.ts @@ -7,38 +7,21 @@ // GraphQL query operation: AppQuery // ==================================================== -export interface AppQuery_pipelines_solids_output_type { +export interface AppQuery_pipelines_solids_outputs_type { name: string; description: string | null; } -export interface AppQuery_pipelines_solids_output_expectations { +export interface AppQuery_pipelines_solids_outputs_expectations { name: string; description: string | null; } -export interface AppQuery_pipelines_solids_output_materializations_arguments_type { +export interface AppQuery_pipelines_solids_outputs { name: string; + type: AppQuery_pipelines_solids_outputs_type; description: string | null; -} - -export interface AppQuery_pipelines_solids_output_materializations_arguments { - name: string; - description: string | null; - type: AppQuery_pipelines_solids_output_materializations_arguments_type; - isOptional: boolean; -} - -export interface AppQuery_pipelines_solids_output_materializations { - name: string; - description: string | null; - arguments: AppQuery_pipelines_solids_output_materializations_arguments[]; -} - -export interface AppQuery_pipelines_solids_output { - type: AppQuery_pipelines_solids_output_type; - expectations: AppQuery_pipelines_solids_output_expectations[]; - materializations: AppQuery_pipelines_solids_output_materializations[]; + expectations: AppQuery_pipelines_solids_outputs_expectations[]; } export interface AppQuery_pipelines_solids_inputs_type { @@ -51,42 +34,41 @@ export interface AppQuery_pipelines_solids_inputs_expectations { description: string | null; } -export interface AppQuery_pipelines_solids_inputs_sources_arguments_type { +export interface AppQuery_pipelines_solids_inputs_dependsOn_solid { name: string; - description: string | null; } -export interface AppQuery_pipelines_solids_inputs_sources_arguments { +export interface AppQuery_pipelines_solids_inputs_dependsOn { name: string; - description: string | null; - type: AppQuery_pipelines_solids_inputs_sources_arguments_type; - isOptional: boolean; + solid: AppQuery_pipelines_solids_inputs_dependsOn_solid; } -export interface AppQuery_pipelines_solids_inputs_sources { +export interface AppQuery_pipelines_solids_inputs { name: string; + type: AppQuery_pipelines_solids_inputs_type; description: string | null; - arguments: AppQuery_pipelines_solids_inputs_sources_arguments[]; + expectations: AppQuery_pipelines_solids_inputs_expectations[]; + dependsOn: AppQuery_pipelines_solids_inputs_dependsOn | null; } -export interface AppQuery_pipelines_solids_inputs_dependsOn { +export interface AppQuery_pipelines_solids_config_type { name: string; + description: string | null; } -export interface AppQuery_pipelines_solids_inputs { +export interface AppQuery_pipelines_solids_config { name: string; - type: AppQuery_pipelines_solids_inputs_type; - expectations: AppQuery_pipelines_solids_inputs_expectations[]; description: string | null; - sources: AppQuery_pipelines_solids_inputs_sources[]; - dependsOn: AppQuery_pipelines_solids_inputs_dependsOn | null; + type: AppQuery_pipelines_solids_config_type; + isOptional: boolean; } export interface AppQuery_pipelines_solids { - output: AppQuery_pipelines_solids_output; + outputs: AppQuery_pipelines_solids_outputs[]; inputs: AppQuery_pipelines_solids_inputs[]; name: string; description: string | null; + config: AppQuery_pipelines_solids_config[]; } export interface AppQuery_pipelines_context_arguments_type { diff --git a/python_modules/dagit/dagit/webapp/src/types/PipelineFragment.ts b/python_modules/dagit/dagit/webapp/src/types/PipelineFragment.ts index 9d5a106291d4f..d9fc4d53120cc 100644 --- a/python_modules/dagit/dagit/webapp/src/types/PipelineFragment.ts +++ b/python_modules/dagit/dagit/webapp/src/types/PipelineFragment.ts @@ -7,38 +7,21 @@ // GraphQL fragment: PipelineFragment // ==================================================== -export interface PipelineFragment_solids_output_type { +export interface PipelineFragment_solids_outputs_type { name: string; description: string | null; } -export interface PipelineFragment_solids_output_expectations { +export interface PipelineFragment_solids_outputs_expectations { name: string; description: string | null; } -export interface PipelineFragment_solids_output_materializations_arguments_type { +export interface PipelineFragment_solids_outputs { name: string; + type: PipelineFragment_solids_outputs_type; description: string | null; -} - -export interface PipelineFragment_solids_output_materializations_arguments { - name: string; - description: string | null; - type: PipelineFragment_solids_output_materializations_arguments_type; - isOptional: boolean; -} - -export interface PipelineFragment_solids_output_materializations { - name: string; - description: string | null; - arguments: PipelineFragment_solids_output_materializations_arguments[]; -} - -export interface PipelineFragment_solids_output { - type: PipelineFragment_solids_output_type; - expectations: PipelineFragment_solids_output_expectations[]; - materializations: PipelineFragment_solids_output_materializations[]; + expectations: PipelineFragment_solids_outputs_expectations[]; } export interface PipelineFragment_solids_inputs_type { @@ -51,42 +34,41 @@ export interface PipelineFragment_solids_inputs_expectations { description: string | null; } -export interface PipelineFragment_solids_inputs_sources_arguments_type { +export interface PipelineFragment_solids_inputs_dependsOn_solid { name: string; - description: string | null; } -export interface PipelineFragment_solids_inputs_sources_arguments { +export interface PipelineFragment_solids_inputs_dependsOn { name: string; - description: string | null; - type: PipelineFragment_solids_inputs_sources_arguments_type; - isOptional: boolean; + solid: PipelineFragment_solids_inputs_dependsOn_solid; } -export interface PipelineFragment_solids_inputs_sources { +export interface PipelineFragment_solids_inputs { name: string; + type: PipelineFragment_solids_inputs_type; description: string | null; - arguments: PipelineFragment_solids_inputs_sources_arguments[]; + expectations: PipelineFragment_solids_inputs_expectations[]; + dependsOn: PipelineFragment_solids_inputs_dependsOn | null; } -export interface PipelineFragment_solids_inputs_dependsOn { +export interface PipelineFragment_solids_config_type { name: string; + description: string | null; } -export interface PipelineFragment_solids_inputs { +export interface PipelineFragment_solids_config { name: string; - type: PipelineFragment_solids_inputs_type; - expectations: PipelineFragment_solids_inputs_expectations[]; description: string | null; - sources: PipelineFragment_solids_inputs_sources[]; - dependsOn: PipelineFragment_solids_inputs_dependsOn | null; + type: PipelineFragment_solids_config_type; + isOptional: boolean; } export interface PipelineFragment_solids { - output: PipelineFragment_solids_output; + outputs: PipelineFragment_solids_outputs[]; inputs: PipelineFragment_solids_inputs[]; name: string; description: string | null; + config: PipelineFragment_solids_config[]; } export interface PipelineFragment_context_arguments_type { diff --git a/python_modules/dagit/dagit/webapp/src/types/PipelinesFragment.ts b/python_modules/dagit/dagit/webapp/src/types/PipelinesFragment.ts index 06004781d09a7..2ab384ed98a20 100644 --- a/python_modules/dagit/dagit/webapp/src/types/PipelinesFragment.ts +++ b/python_modules/dagit/dagit/webapp/src/types/PipelinesFragment.ts @@ -7,38 +7,21 @@ // GraphQL fragment: PipelinesFragment // ==================================================== -export interface PipelinesFragment_solids_output_type { +export interface PipelinesFragment_solids_outputs_type { name: string; description: string | null; } -export interface PipelinesFragment_solids_output_expectations { +export interface PipelinesFragment_solids_outputs_expectations { name: string; description: string | null; } -export interface PipelinesFragment_solids_output_materializations_arguments_type { +export interface PipelinesFragment_solids_outputs { name: string; + type: PipelinesFragment_solids_outputs_type; description: string | null; -} - -export interface PipelinesFragment_solids_output_materializations_arguments { - name: string; - description: string | null; - type: PipelinesFragment_solids_output_materializations_arguments_type; - isOptional: boolean; -} - -export interface PipelinesFragment_solids_output_materializations { - name: string; - description: string | null; - arguments: PipelinesFragment_solids_output_materializations_arguments[]; -} - -export interface PipelinesFragment_solids_output { - type: PipelinesFragment_solids_output_type; - expectations: PipelinesFragment_solids_output_expectations[]; - materializations: PipelinesFragment_solids_output_materializations[]; + expectations: PipelinesFragment_solids_outputs_expectations[]; } export interface PipelinesFragment_solids_inputs_type { @@ -51,42 +34,41 @@ export interface PipelinesFragment_solids_inputs_expectations { description: string | null; } -export interface PipelinesFragment_solids_inputs_sources_arguments_type { +export interface PipelinesFragment_solids_inputs_dependsOn_solid { name: string; - description: string | null; } -export interface PipelinesFragment_solids_inputs_sources_arguments { +export interface PipelinesFragment_solids_inputs_dependsOn { name: string; - description: string | null; - type: PipelinesFragment_solids_inputs_sources_arguments_type; - isOptional: boolean; + solid: PipelinesFragment_solids_inputs_dependsOn_solid; } -export interface PipelinesFragment_solids_inputs_sources { +export interface PipelinesFragment_solids_inputs { name: string; + type: PipelinesFragment_solids_inputs_type; description: string | null; - arguments: PipelinesFragment_solids_inputs_sources_arguments[]; + expectations: PipelinesFragment_solids_inputs_expectations[]; + dependsOn: PipelinesFragment_solids_inputs_dependsOn | null; } -export interface PipelinesFragment_solids_inputs_dependsOn { +export interface PipelinesFragment_solids_config_type { name: string; + description: string | null; } -export interface PipelinesFragment_solids_inputs { +export interface PipelinesFragment_solids_config { name: string; - type: PipelinesFragment_solids_inputs_type; - expectations: PipelinesFragment_solids_inputs_expectations[]; description: string | null; - sources: PipelinesFragment_solids_inputs_sources[]; - dependsOn: PipelinesFragment_solids_inputs_dependsOn | null; + type: PipelinesFragment_solids_config_type; + isOptional: boolean; } export interface PipelinesFragment_solids { - output: PipelinesFragment_solids_output; + outputs: PipelinesFragment_solids_outputs[]; inputs: PipelinesFragment_solids_inputs[]; name: string; description: string | null; + config: PipelinesFragment_solids_config[]; } export interface PipelinesFragment_context_arguments_type { diff --git a/python_modules/dagit/dagit/webapp/src/types/SolidFragment.ts b/python_modules/dagit/dagit/webapp/src/types/SolidFragment.ts index b9c2076b1319f..599c55342195a 100644 --- a/python_modules/dagit/dagit/webapp/src/types/SolidFragment.ts +++ b/python_modules/dagit/dagit/webapp/src/types/SolidFragment.ts @@ -7,38 +7,21 @@ // GraphQL fragment: SolidFragment // ==================================================== -export interface SolidFragment_output_type { +export interface SolidFragment_outputs_type { name: string; description: string | null; } -export interface SolidFragment_output_expectations { +export interface SolidFragment_outputs_expectations { name: string; description: string | null; } -export interface SolidFragment_output_materializations_arguments_type { +export interface SolidFragment_outputs { name: string; + type: SolidFragment_outputs_type; description: string | null; -} - -export interface SolidFragment_output_materializations_arguments { - name: string; - description: string | null; - type: SolidFragment_output_materializations_arguments_type; - isOptional: boolean; -} - -export interface SolidFragment_output_materializations { - name: string; - description: string | null; - arguments: SolidFragment_output_materializations_arguments[]; -} - -export interface SolidFragment_output { - type: SolidFragment_output_type; - expectations: SolidFragment_output_expectations[]; - materializations: SolidFragment_output_materializations[]; + expectations: SolidFragment_outputs_expectations[]; } export interface SolidFragment_inputs_type { @@ -51,42 +34,41 @@ export interface SolidFragment_inputs_expectations { description: string | null; } -export interface SolidFragment_inputs_sources_arguments_type { +export interface SolidFragment_inputs_dependsOn_solid { name: string; - description: string | null; } -export interface SolidFragment_inputs_sources_arguments { +export interface SolidFragment_inputs_dependsOn { name: string; - description: string | null; - type: SolidFragment_inputs_sources_arguments_type; - isOptional: boolean; + solid: SolidFragment_inputs_dependsOn_solid; } -export interface SolidFragment_inputs_sources { +export interface SolidFragment_inputs { name: string; + type: SolidFragment_inputs_type; description: string | null; - arguments: SolidFragment_inputs_sources_arguments[]; + expectations: SolidFragment_inputs_expectations[]; + dependsOn: SolidFragment_inputs_dependsOn | null; } -export interface SolidFragment_inputs_dependsOn { +export interface SolidFragment_config_type { name: string; + description: string | null; } -export interface SolidFragment_inputs { +export interface SolidFragment_config { name: string; - type: SolidFragment_inputs_type; - expectations: SolidFragment_inputs_expectations[]; description: string | null; - sources: SolidFragment_inputs_sources[]; - dependsOn: SolidFragment_inputs_dependsOn | null; + type: SolidFragment_config_type; + isOptional: boolean; } export interface SolidFragment { - output: SolidFragment_output; + outputs: SolidFragment_outputs[]; inputs: SolidFragment_inputs[]; name: string; description: string | null; + config: SolidFragment_config[]; } /* tslint:disable */ diff --git a/python_modules/dagit/dagit/webapp/src/types/SolidListItemFragment.ts b/python_modules/dagit/dagit/webapp/src/types/SolidListItemFragment.ts index 6de2c8f922c1e..4c90ca6945b31 100644 --- a/python_modules/dagit/dagit/webapp/src/types/SolidListItemFragment.ts +++ b/python_modules/dagit/dagit/webapp/src/types/SolidListItemFragment.ts @@ -7,13 +7,14 @@ // GraphQL fragment: SolidListItemFragment // ==================================================== -export interface SolidListItemFragment_output_type { +export interface SolidListItemFragment_outputs_type { name: string; description: string | null; } -export interface SolidListItemFragment_output { - type: SolidListItemFragment_output_type; +export interface SolidListItemFragment_outputs { + name: string; + type: SolidListItemFragment_outputs_type; } export interface SolidListItemFragment_inputs_type { @@ -29,7 +30,7 @@ export interface SolidListItemFragment_inputs { export interface SolidListItemFragment { name: string; description: string | null; - output: SolidListItemFragment_output; + outputs: SolidListItemFragment_outputs[]; inputs: SolidListItemFragment_inputs[]; } diff --git a/python_modules/dagit/dagit/webapp/src/types/SolidTypeSignatureFragment.ts b/python_modules/dagit/dagit/webapp/src/types/SolidTypeSignatureFragment.ts index 5716f2910182e..bce09193d4305 100644 --- a/python_modules/dagit/dagit/webapp/src/types/SolidTypeSignatureFragment.ts +++ b/python_modules/dagit/dagit/webapp/src/types/SolidTypeSignatureFragment.ts @@ -7,13 +7,14 @@ // GraphQL fragment: SolidTypeSignatureFragment // ==================================================== -export interface SolidTypeSignatureFragment_output_type { +export interface SolidTypeSignatureFragment_outputs_type { name: string; description: string | null; } -export interface SolidTypeSignatureFragment_output { - type: SolidTypeSignatureFragment_output_type; +export interface SolidTypeSignatureFragment_outputs { + name: string; + type: SolidTypeSignatureFragment_outputs_type; } export interface SolidTypeSignatureFragment_inputs_type { @@ -27,7 +28,7 @@ export interface SolidTypeSignatureFragment_inputs { } export interface SolidTypeSignatureFragment { - output: SolidTypeSignatureFragment_output; + outputs: SolidTypeSignatureFragment_outputs[]; inputs: SolidTypeSignatureFragment_inputs[]; } diff --git a/python_modules/dagit/webapp/src/Arguments.tsx b/python_modules/dagit/webapp/src/Arguments.tsx new file mode 100644 index 0000000000000..4fc45d41d2656 --- /dev/null +++ b/python_modules/dagit/webapp/src/Arguments.tsx @@ -0,0 +1,50 @@ +import * as React from "react"; +import gql from "graphql-tag"; +import styled from "styled-components"; +import { H5, H6, Text, UL, Code, Collapse } from "@blueprintjs/core"; +import SpacedCard from "./SpacedCard"; +import TypeWithTooltip from "./TypeWithTooltip"; +import Description from "./Description"; +import { ArgumentsFragment } from "./types/ArgumentsFragment"; + +interface ArgumentsProps { + arguments: Array; +} + +export default class Arguments extends React.Component { + static fragments = { + ArgumentsFragment: gql` + fragment ArgumentsFragment on Argument { + name + description + type { + ...TypeFragment + } + isOptional + } + + ${TypeWithTooltip.fragments.TypeFragment} + ` + }; + + public render() { + return ( +
    + {this.props.arguments.map((argument: any, i: number) => ( +
  • + {argument.name} {argument.isOptional ? "(optional)" : null}{" "} + + + + +
  • + ))} +
+ ); + } +} + +const DescriptionWrapper = styled.div` + max-width: 400px; + margin-bottom: 10px; +`; diff --git a/python_modules/dagit/webapp/src/types/ArgumentsFragment.ts b/python_modules/dagit/webapp/src/types/ArgumentsFragment.ts new file mode 100644 index 0000000000000..a0aed84ee9063 --- /dev/null +++ b/python_modules/dagit/webapp/src/types/ArgumentsFragment.ts @@ -0,0 +1,31 @@ + + +/* tslint:disable */ +// This file was automatically generated and should not be edited. + +// ==================================================== +// GraphQL fragment: ArgumentsFragment +// ==================================================== + +export interface ArgumentsFragment_type { + name: string; + description: string | null; +} + +export interface ArgumentsFragment { + name: string; + description: string | null; + type: ArgumentsFragment_type; + isOptional: boolean; +} + +/* tslint:disable */ +// This file was automatically generated and should not be edited. + +//============================================================== +// START Enums and Input Objects +//============================================================== + +//============================================================== +// END Enums and Input Objects +//============================================================== \ No newline at end of file diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index e3c5d3881ec97..6076cd5f9f0e6 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -139,12 +139,26 @@ def has_dep(self, solid_input_handle): def deps_of_solid(self, solid_name): check.str_param(solid_name, 'solid_name') - return list(self.__gen_deps_of_solid(solid_name)) + return list(handles[1] for handles in self.__gen_deps_of_solid(solid_name)) + + def deps_of_solid_with_input(self, solid_name): + check.str_param(solid_name, 'solid_name') + return dict(self.__gen_deps_of_solid(solid_name)) def __gen_deps_of_solid(self, solid_name): for input_handle, output_handle in self._handle_dict.items(): if input_handle.solid.name == solid_name: - yield output_handle + yield (input_handle, output_handle) + + def depended_by_of_solid(self, solid_name): + check.str_param(solid_name, 'solid_name') + result = {} + for input_handle, output_handle in self._handle_dict.items(): + if output_handle.solid.name == solid_name: + if output_handle not in result: + result[output_handle] = [] + result[output_handle].extend(input_handle) + return result def get_dep(self, solid_input_handle): check.inst_param(solid_input_handle, 'solid_input_handle', SolidInputHandle) From f8d070b6e6f00c84377d7c2691313d536d74279f Mon Sep 17 00:00:00 2001 From: Mikhail Novikov Date: Wed, 22 Aug 2018 15:19:44 +0300 Subject: [PATCH 014/103] Update publish script --- python_modules/dagit/bin/publish.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/python_modules/dagit/bin/publish.sh b/python_modules/dagit/bin/publish.sh index ee425a4a99e28..10abbd212262e 100755 --- a/python_modules/dagit/bin/publish.sh +++ b/python_modules/dagit/bin/publish.sh @@ -1,4 +1,8 @@ #!/bin/bash +cd ./dagit/webapp +yarn install +yarn build +cd ../../ python3 setup.py sdist bdist_wheel twine upload dist/* From 4c7822fc86526cd1f3793012ddeb95b3d548fb10 Mon Sep 17 00:00:00 2001 From: Mikhail Novikov Date: Wed, 22 Aug 2018 15:25:42 +0300 Subject: [PATCH 015/103] Remove extra stuff + linting --- .../dagit/webapp/src/graph/PipelineGraph.tsx | 13 ++---------- .../dagit/webapp/src/graph/PipelineLegend.tsx | 21 ------------------- python_modules/dagit/webapp/src/Arguments.tsx | 3 +-- 3 files changed, 3 insertions(+), 34 deletions(-) delete mode 100644 python_modules/dagit/dagit/webapp/src/graph/PipelineLegend.tsx diff --git a/python_modules/dagit/dagit/webapp/src/graph/PipelineGraph.tsx b/python_modules/dagit/dagit/webapp/src/graph/PipelineGraph.tsx index 82f88b9fdfe7d..d690e2b0a3577 100644 --- a/python_modules/dagit/dagit/webapp/src/graph/PipelineGraph.tsx +++ b/python_modules/dagit/dagit/webapp/src/graph/PipelineGraph.tsx @@ -1,7 +1,7 @@ import * as React from "react"; import gql from "graphql-tag"; import styled from "styled-components"; -import { Card, Colors } from "@blueprintjs/core"; +import { Colors } from "@blueprintjs/core"; import { LinkHorizontal as Link } from "@vx/shape"; import PanAndZoom from "./PanAndZoom"; import SolidNode from "./SolidNode"; @@ -35,7 +35,7 @@ export default class PipelineGraph extends React.Component< }; renderSolids(layout: IFullPipelineLayout) { - return this.props.pipeline.solids.map((solid, i) => { + return this.props.pipeline.solids.map(solid => { const solidLayout = layout.solids[solid.name]; return ( PipelineColorScale(datum)} - labelFormat={(label: string) => `${label.toUpperCase()}`} - /> - ); - } -} diff --git a/python_modules/dagit/webapp/src/Arguments.tsx b/python_modules/dagit/webapp/src/Arguments.tsx index 4fc45d41d2656..c0702292706fb 100644 --- a/python_modules/dagit/webapp/src/Arguments.tsx +++ b/python_modules/dagit/webapp/src/Arguments.tsx @@ -1,8 +1,7 @@ import * as React from "react"; import gql from "graphql-tag"; import styled from "styled-components"; -import { H5, H6, Text, UL, Code, Collapse } from "@blueprintjs/core"; -import SpacedCard from "./SpacedCard"; +import { UL } from "@blueprintjs/core"; import TypeWithTooltip from "./TypeWithTooltip"; import Description from "./Description"; import { ArgumentsFragment } from "./types/ArgumentsFragment"; From 26635072c3eabe2197c84fcb09d37b5f5c800fc3 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Thu, 23 Aug 2018 07:13:23 -0700 Subject: [PATCH 016/103] Changes to support clarify 1) Make expectations work for cases with named outputs 2) Change output UI to render Output instead of Input 3) Equality semantics for DependencyDefinition (nice for testing) 4) Rename create_pipeline_slice to create_sub_pipeline (marginally better) 5) Add create_single_solid_pipeline --- .../dagit/dagit/webapp/src/Solid.tsx | 2 +- .../dagster_ge_tests/test_pandas_ge.py | 8 +-- .../dagster/dagster/core/compute_nodes.py | 17 ++++--- .../core/core_tests/test_decorators.py | 4 +- .../core/core_tests/test_definitions.py | 9 ++++ .../core/core_tests/test_multiple_outputs.py | 49 +++++++++++++++++++ .../core/core_tests/test_naming_collisions.py | 18 ++++++- .../core_tests/test_pipeline_execution.py | 4 +- .../dagster/dagster/core/definitions.py | 47 +++++++++++++----- .../dagster/dagster/core/utility_solids.py | 19 +------ .../pandas_kernel_tests/test_pandas_solids.py | 2 +- .../test_pandas_user_error.py | 6 +-- .../test_basic_solid.py | 6 +-- .../test_isolated_templated_sql_tests.py | 18 +++---- 14 files changed, 145 insertions(+), 64 deletions(-) create mode 100644 python_modules/dagster/dagster/core/core_tests/test_definitions.py diff --git a/python_modules/dagit/dagit/webapp/src/Solid.tsx b/python_modules/dagit/dagit/webapp/src/Solid.tsx index 8227309ff49a8..5ea9ce9cd0161 100644 --- a/python_modules/dagit/dagit/webapp/src/Solid.tsx +++ b/python_modules/dagit/dagit/webapp/src/Solid.tsx @@ -112,7 +112,7 @@ export default class Solid extends React.Component { return this.props.solid.outputs.map((output, i: number) => (
- Input {output.name} + Output {output.name}
diff --git a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py index eb31fc1541247..412d354314cb6 100644 --- a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py +++ b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py @@ -71,13 +71,13 @@ def sum_solid_expectations_config(num_df): return _sum_solid_impl(num_df) -from dagster.core.utility_solids import define_pass_mem_value +from dagster.core.utility_solids import define_stub_solid def test_single_node_passing_expectation(): in_df = pd.DataFrame.from_dict({'num1': [1, 3], 'num2': [2, 4]}) pipeline = dagster.PipelineDefinition( - solids=[define_pass_mem_value('value', in_df), sum_solid], + solids=[define_stub_solid('value', in_df), sum_solid], dependencies={'sum_solid': { 'num_df': DependencyDefinition('value') }} @@ -97,7 +97,7 @@ def test_single_node_passing_expectation(): def test_single_node_passing_json_config_expectations(): in_df = pd.DataFrame.from_dict({'num1': [1, 3], 'num2': [2, 4]}) pipeline = dagster.PipelineDefinition( - solids=[define_pass_mem_value('value', in_df), sum_solid_expectations_config], + solids=[define_stub_solid('value', in_df), sum_solid_expectations_config], dependencies={ sum_solid_expectations_config.name: { 'num_df': DependencyDefinition('value') @@ -119,7 +119,7 @@ def test_single_node_passing_json_config_expectations(): def test_single_node_failing_expectation(): in_df = pd.DataFrame.from_dict({'num1': [1, 3], 'num2': [2, 4]}) pipeline = dagster.PipelineDefinition( - solids=[define_pass_mem_value('value', in_df), sum_solid_fails_input_expectation], + solids=[define_stub_solid('value', in_df), sum_solid_fails_input_expectation], dependencies={ sum_solid_fails_input_expectation.name: { 'num_df': DependencyDefinition('value') diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index e312607d83262..4005442074571 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -293,7 +293,6 @@ def output_named(self, name): check.failed(f'output {name} not found') - def execute_compute_nodes(context, compute_nodes): check.inst_param(context, 'context', ExecutionContext) check.list_param(compute_nodes, 'compute_nodes', of_type=ComputeNode) @@ -304,9 +303,13 @@ def execute_compute_nodes(context, compute_nodes): for node_input in compute_node.node_inputs: prev_output_handle = node_input.prev_output_handle if prev_output_handle not in intermediate_results: + + target_cn = prev_output_handle.compute_node + check.failed( f'Could not find handle {prev_output_handle} in results. ' + \ - f'current node: {compute_node.friendly_name}' + f'current node: {compute_node.friendly_name}\n' + \ + f'target node: {target_cn.friendly_name}' ) input_value = intermediate_results[prev_output_handle].success_data.value input_values[node_input.name] = input_value @@ -357,24 +360,25 @@ def topological_nodes(self): yield self.cn_dict[cn_guid] -def create_expectation_cn(solid, expectation_def, friendly_name, tag, prev_node_output_handle): +def create_expectation_cn(solid, expectation_def, friendly_name, tag, prev_node_output_handle, inout_def): check.inst_param(solid, 'solid', SolidDefinition) check.inst_param(expectation_def, 'input_expct_def', ExpectationDefinition) check.inst_param(prev_node_output_handle, 'prev_node_output_handle', ComputeNodeOutputHandle) + check.inst_param(inout_def, 'inout_def', (InputDefinition, OutputDefinition)) - output = get_single_solid_output(solid) + value_type = inout_def.dagster_type return SingleSyncOutputComputeNode( friendly_name=friendly_name, node_inputs=[ ComputeNodeInput( name=EXPECTATION_INPUT, - dagster_type=output.dagster_type, + dagster_type=value_type, prev_output_handle=prev_node_output_handle, ) ], node_outputs=[ - ComputeNodeOutput(name=EXPECTATION_VALUE_OUTPUT, dagster_type=output.dagster_type), + ComputeNodeOutput(name=EXPECTATION_VALUE_OUTPUT, dagster_type=value_type), ], arg_dict={}, sync_compute_fn=_create_expectation_lambda( @@ -408,6 +412,7 @@ def create_expectations_cn_graph(solid, inout_def, prev_node_output_handle, tag) friendly_name=f'{solid.name}.{inout_def.name}.expectation.{expectation_def.name}', tag=tag, prev_node_output_handle=prev_node_output_handle, + inout_def=inout_def, ) input_expect_nodes.append(expect_compute_node) compute_nodes.append(expect_compute_node) diff --git a/python_modules/dagster/dagster/core/core_tests/test_decorators.py b/python_modules/dagster/dagster/core/core_tests/test_decorators.py index e75c134250a12..22ef6188dcb2c 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_decorators.py +++ b/python_modules/dagster/dagster/core/core_tests/test_decorators.py @@ -17,7 +17,7 @@ execute_single_solid, ExecutionContext, ) -from dagster.core.utility_solids import define_pass_mem_value +from dagster.core.utility_solids import define_stub_solid # This file tests a lot of parameter name stuff # So these warnings are spurious @@ -88,7 +88,7 @@ def hello_world(foo_to_foo): return foo_to_foo pipeline = PipelineDefinition( - solids=[define_pass_mem_value('test_value', {'foo': 'bar'}), hello_world], + solids=[define_stub_solid('test_value', {'foo': 'bar'}), hello_world], dependencies={'hello_world': { 'foo_to_foo': DependencyDefinition('test_value'), }} diff --git a/python_modules/dagster/dagster/core/core_tests/test_definitions.py b/python_modules/dagster/dagster/core/core_tests/test_definitions.py new file mode 100644 index 0000000000000..f70685c937f6a --- /dev/null +++ b/python_modules/dagster/dagster/core/core_tests/test_definitions.py @@ -0,0 +1,9 @@ +from dagster import (DependencyDefinition) + + +def test_deps_equal(): + assert DependencyDefinition('foo') == DependencyDefinition('foo') + assert DependencyDefinition('foo') != DependencyDefinition('bar') + + assert DependencyDefinition('foo', 'bar') == DependencyDefinition('foo', 'bar') + assert DependencyDefinition('foo', 'bar') != DependencyDefinition('foo', 'quuz') diff --git a/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py b/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py index 0d37fc66cccdd..7b0a3bebbee4c 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py +++ b/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py @@ -1,4 +1,6 @@ from dagster import ( + ExpectationDefinition, + ExpectationResult, OutputDefinition, PipelineDefinition, Result, @@ -35,3 +37,50 @@ def _t_fn(_context, _inputs, _config_dict): assert result.result_list[1].name == 'multiple_outputs' assert result.result_list[1].output_name == 'output_two' assert result.result_list[1].transformed_value == 'bar' + + +def test_multiple_outputs_expectations(): + called = {} + + def _expect_fn_one(*_args, **_kwargs): + called['expectation_one'] = True + return ExpectationResult(success=True) + + def _expect_fn_two(*_args, **_kwargs): + called['expectation_two'] = True + return ExpectationResult(success=True) + + def _transform_fn(*_args, **_kwargs): + yield Result('foo', 'output_one') + yield Result('bar', 'output_two') + + solid = SolidDefinition( + name='multiple_outputs', + inputs=[], + outputs=[ + OutputDefinition( + name='output_one', + expectations=[ + ExpectationDefinition(name='some_expectation', expectation_fn=_expect_fn_one) + ] + ), + OutputDefinition( + name='output_two', + expectations=[ + ExpectationDefinition( + name='some_other_expectation', expectation_fn=_expect_fn_two + ) + ], + ), + ], + config_def={}, + transform_fn=_transform_fn, + ) + + pipeline = PipelineDefinition(solids=[solid]) + + result = execute_pipeline(pipeline, config.Environment()) + + assert result.success + assert called['expectation_one'] + assert called['expectation_two'] diff --git a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py index 42bf9aa0b4ee8..93b7608283ec7 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py +++ b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py @@ -8,12 +8,28 @@ PipelineDefinition, Result, SolidDefinition, + check, config, execute_pipeline, types, ) -from dagster.core.utility_solids import define_pass_value_solid + +def define_pass_value_solid(name, description=None): + check.str_param(name, 'name') + check.opt_str_param(description, 'description') + + def _value_t_fn(_context, _inputs, config_dict): + yield Result(config_dict['value']) + + return SolidDefinition( + name=name, + description=description, + inputs=[], + outputs=[OutputDefinition(dagster_type=types.String)], + config_def={'value': ArgumentDefinition(types.String)}, + transform_fn=_value_t_fn, + ) def test_execute_solid_with_input_same_name(): diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py index 110bf4f27a5b7..bdf754fb46b84 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py @@ -156,11 +156,11 @@ def test_disconnected_graphs_adjaceny_lists(): assert backwards_edges == {'B': {'A'}, 'A': set(), 'D': {'C'}, 'C': set()} -from dagster.core.utility_solids import define_pass_mem_value +from dagster.core.utility_solids import define_stub_solid def create_diamond_solids(): - a_source = define_pass_mem_value('A_source', [input_set('A_input')]) + a_source = define_stub_solid('A_source', [input_set('A_input')]) node_a = create_root_solid('A') node_b = create_solid_with_deps('B', node_a) node_c = create_solid_with_deps('C', node_a) diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index 6076cd5f9f0e6..1023692b21879 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -47,10 +47,10 @@ def check_valid_name(name): return name -def check_argument_def_dict(argument_def_dict): +def check_argument_def_dict(config_def_dict): return check.dict_param( - argument_def_dict, - 'argument_def_dict', + config_def_dict, + 'config_def_dict', key_type=str, value_type=ArgumentDefinition, ) @@ -83,11 +83,14 @@ def _default_context_fn(_pipeline, args): return {'default': default_context_def} -class DependencyDefinition: - def __init__(self, solid, output=DEFAULT_OUTPUT, description=None): - self.solid = check.str_param(solid, 'solid') - self.output = check.str_param(output, 'output') - self.description = check.opt_str_param(description, 'description') +class DependencyDefinition(namedtuple('_DependencyDefinition', 'solid output description')): + def __new__(cls, solid, output=DEFAULT_OUTPUT, description=None): + return super(DependencyDefinition, cls).__new__( + cls, + check.str_param(solid, 'solid'), + check.str_param(output, 'output'), + check.opt_str_param(description, 'description'), + ) class InputToOutputHandleDict(dict): @@ -108,6 +111,13 @@ def check_two_dim_str_dict(ddict, param_name, value_type): return ddict +def check_opt_two_dim_str_dict(ddict, param_name, value_type): + ddict = check.opt_dict_param(ddict, param_name, key_type=str, value_type=dict) + for sub_dict in ddict.values(): + check.dict_param(sub_dict, 'sub_dict', key_type=str, value_type=value_type) + return ddict + + def create_handle_dict(solid_dict, dep_dict): check.dict_param(solid_dict, 'solid_dict', key_type=str, value_type=SolidDefinition) check_two_dim_str_dict(dep_dict, 'dep_dict', DependencyDefinition) @@ -180,12 +190,23 @@ def _build_named_dict(things): class PipelineDefinition: @staticmethod - def create_pipeline_slice(pipeline, from_solids, through_solids, injected_solids): + def create_single_solid_pipeline(pipeline, solid_name, injected_solids=None): + return PipelineDefinition.create_sub_pipeline( + pipeline, + [solid_name], + [solid_name], + injected_solids, + ) + + @staticmethod + def create_sub_pipeline(pipeline, from_solids, through_solids, injected_solids=None): from .graph import ExecutionGraph check.inst_param(pipeline, 'pipeline', PipelineDefinition) check.list_param(from_solids, 'from_solids', of_type=str) check.list_param(through_solids, 'through_solids', of_type=str) - check_two_dim_str_dict(injected_solids, 'injected_solids', SolidDefinition) + injected_solids = check_opt_two_dim_str_dict( + injected_solids, 'injected_solids', SolidDefinition + ) subgraph = ExecutionGraph.from_pipeline_subset( pipeline, @@ -438,7 +459,9 @@ def __init__(self, *, name, inputs, transform_fn, outputs, config_def, descripti self._output_dict = _build_named_dict(outputs) @staticmethod - def single_output_transform(name, inputs, transform_fn, output, description=None): + def single_output_transform( + name, inputs, transform_fn, output, config_def=None, description=None + ): def _new_transform_fn(context, inputs, _config_dict): value = transform_fn(context, inputs) yield Result(output_name=DEFAULT_OUTPUT, value=value) @@ -448,7 +471,7 @@ def _new_transform_fn(context, inputs, _config_dict): inputs=inputs, transform_fn=_new_transform_fn, outputs=[output], - config_def={}, + config_def=check.opt_dict_param(config_def, 'config_def'), description=description, ) diff --git a/python_modules/dagster/dagster/core/utility_solids.py b/python_modules/dagster/dagster/core/utility_solids.py index 421393ae94370..c05bff14582fc 100644 --- a/python_modules/dagster/dagster/core/utility_solids.py +++ b/python_modules/dagster/dagster/core/utility_solids.py @@ -8,24 +8,7 @@ ) -def define_pass_value_solid(name, description=None): - check.str_param(name, 'name') - check.opt_str_param(description, 'description') - - def _value_t_fn(_context, _inputs, config_dict): - yield Result(config_dict['value']) - - return SolidDefinition( - name=name, - description=description, - inputs=[], - outputs=[OutputDefinition(dagster_type=types.String)], - config_def={'value': ArgumentDefinition(types.String)}, - transform_fn=_value_t_fn, - ) - - -def define_pass_mem_value(name, value): +def define_stub_solid(name, value): check.str_param(name, 'name') def _value_t_fn(_context, _inputs, _config_dict): diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py index b1f9a5cd744a2..4527f9acb52e9 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py @@ -506,7 +506,7 @@ def test_pandas_output_intermediate_csv_files(): } pipeline_result = execute_pipeline( - PipelineDefinition.create_pipeline_slice( + PipelineDefinition.create_sub_pipeline( pipeline, ['sum_mult_table'], ['sum_mult_table'], diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_user_error.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_user_error.py index 0b966360e6466..12875e58865d2 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_user_error.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_user_error.py @@ -19,7 +19,7 @@ from dagster.core.errors import DagsterInvariantViolationError from dagster.core.execution import execute_single_solid from dagster.utils.test import script_relative_path -from dagster.core.utility_solids import define_pass_mem_value +from dagster.core.utility_solids import define_stub_solid def _dataframe_solid(name, inputs, transform_fn): @@ -42,7 +42,7 @@ def test_wrong_output_value(): def df_solid(num_csv): return 'not a dataframe' - pass_solid = define_pass_mem_value('pass_solid', pd.DataFrame()) + pass_solid = define_stub_solid('pass_solid', pd.DataFrame()) pipeline = PipelineDefinition( solids=[pass_solid, df_solid], @@ -67,7 +67,7 @@ def test_wrong_input_value(): def df_solid(foo): return foo - pass_solid = define_pass_mem_value('pass_solid', 'not a dataframe') + pass_solid = define_stub_solid('pass_solid', 'not a dataframe') pipeline = PipelineDefinition( solids=[pass_solid, df_solid], diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_basic_solid.py b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_basic_solid.py index 8b07e2aba2ac3..68bef85bc435f 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_basic_solid.py +++ b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_basic_solid.py @@ -10,7 +10,7 @@ execute_pipeline, ) -from dagster.core.utility_solids import define_pass_mem_value +from dagster.core.utility_solids import define_stub_solid from dagster.sqlalchemy_kernel.subquery_builder_experimental import ( create_sql_solid, @@ -53,7 +53,7 @@ def create_num_table(engine): def test_sql_sum_solid(): - expr_solid = define_pass_mem_value('expr', DagsterSqlTableExpression('num_table')) + expr_solid = define_stub_solid('expr', DagsterSqlTableExpression('num_table')) sum_table_solid = create_sum_table_solid() @@ -101,7 +101,7 @@ def create_sum_table_solid(): def create_sum_sq_pipeline(context, expr, extra_solids=None, extra_deps=None): check.inst_param(expr, 'expr', DagsterSqlTableExpression) - expr_solid = define_pass_mem_value('expr', expr) + expr_solid = define_stub_solid('expr', expr) sum_solid = create_sum_table_solid() diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py index c5c3adedb1726..442106719a564 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py +++ b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py @@ -17,7 +17,7 @@ create_templated_sql_transform_solid, ) -from dagster.core.utility_solids import define_pass_value_solid +from dagster.core.utility_solids import define_stub_solid from .math_test_db import in_mem_context @@ -202,7 +202,7 @@ def test_templated_sql_solid_pipeline(): # now execute subdag pipeline_two = pipeline_test_def( - solids=[define_pass_value_solid('pass_value'), sum_sq_solid], + solids=[define_stub_solid('pass_value', 'TODO'), sum_sq_solid], context=context, dependencies={ sum_sq_solid.name: { @@ -213,17 +213,13 @@ def test_templated_sql_solid_pipeline(): second_sum_sq_table = 'second_sum_sq_table' + sum_sq_args = { + 'sum_table': first_sum_table, + 'sum_sq_table': second_sum_sq_table, + } environment_two = config.Environment( solids={ - 'pass_value': - config.Solid({ - 'value': 'something' - }), - 'sum_sq_table': - config.Solid({ - 'sum_table': first_sum_table, - 'sum_sq_table': second_sum_sq_table, - }), + 'sum_sq_table': config.Solid(sum_sq_args), }, ) From f40fa33df50d127d197c4fb1d433787fe497bcd1 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Thu, 23 Aug 2018 17:10:53 -0700 Subject: [PATCH 017/103] Catch some common errors with multiple outputs 1) Catch when someone returns an output that is not specified 2) Disallow returniing multiple outputs of the same name --- python_modules/dagster/dagster/__init__.py | 5 +- .../dagster/dagster/core/compute_nodes.py | 62 ++++++++++++++----- .../core/core_tests/test_multiple_outputs.py | 46 ++++++++++++++ 3 files changed, 98 insertions(+), 15 deletions(-) diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index 9854309c6a98e..4f98d26986193 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -22,6 +22,9 @@ with_context, ) -from dagster.core.errors import (DagsterInvalidDefinitionError) +from dagster.core.errors import ( + DagsterInvalidDefinitionError, + DagsterInvariantViolationError, +) import dagster.core.types as types diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index 4005442074571..f8e63b212f680 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -156,6 +156,7 @@ class ComputeNodeTag(Enum): JOIN_OUTPUT = 'join_output' EXPECTATION_INPUT = 'expectation_input' + def _execute_core_transform(context, solid_name, solid_transform_fn, values_dict, config_dict): ''' Execute the user-specified transform for the solid. Wrap in an error boundary and do @@ -205,22 +206,28 @@ def __init__(self, friendly_name, node_inputs, node_outputs, arg_dict, compute_f node_input_dict = {} for node_input in node_inputs: node_input_dict[node_input.name] = node_input - self.node_input_dict = node_input_dict + self._node_input_dict = node_input_dict self.node_outputs = check.list_param( node_outputs, 'node_outputs', of_type=ComputeNodeOutput ) + + node_output_dict = {} + for node_output in node_outputs: + node_output_dict[node_output.name] = node_output + + self._node_output_dict = node_output_dict self.arg_dict = check.dict_param(arg_dict, 'arg_dict', key_type=str) self.compute_fn = check.callable_param(compute_fn, 'compute_fn') self.tag = check.inst_param(tag, 'tag', ComputeNodeTag) self.solid = check.inst_param(solid, 'solid', SolidDefinition) - def node_named(self, name): + def has_node(self, name): check.str_param(name, 'name') - for node_output in self.node_outputs: - if node_output.name == name: - return node_output + return name in self._node_output_dict - check.failed(f'{name} not found') + def node_named(self, name): + check.str_param(name, 'name') + return self._node_output_dict[name] def _create_compute_node_result(self, result): check.inst_param(result, 'result', Result) @@ -251,7 +258,7 @@ def execute(self, context, inputs): # do runtime type checks of inputs versus node inputs for input_name, input_value in inputs.items(): - compute_node_input = self.node_input_dict[input_name] + compute_node_input = self._node_input_dict[input_name] if not compute_node_input.dagster_type.is_python_valid_value(input_value): raise DagsterInvariantViolationError( f'''Solid {self.solid.name} input {input_name} @@ -261,18 +268,36 @@ def execute(self, context, inputs): error_str = 'TODO error string' + seen_outputs = set() + try: + with _user_code_error_boundary(context, error_str): gen = self.compute_fn(context, inputs) - if gen is None: - check.invariant(not self.node_outputs) - return + if gen is None: + check.invariant(not self.node_outputs) + return + + results = list(gen) + + for result in results: + if not self.has_node(result.output_name): + raise DagsterInvariantViolationError( + f'''Core transform for {self.solid.name} returned an output + {result.output_name} that does not exist. The available + outputs are {list([output.name for output in self.solid.outputs])}''' + ) - results = list(gen) + if result.output_name in seen_outputs: + raise DagsterInvariantViolationError( + f'''Core transform for {self.solid.name} returned an output + {result.output_name} multiple times''' + ) - for result in results: - yield self._create_compute_node_result(result) + seen_outputs.add(result.output_name) + + yield self._create_compute_node_result(result) except DagsterUserCodeExecutionError as dagster_user_exception: yield ComputeNodeResult.failure_result( @@ -293,6 +318,7 @@ def output_named(self, name): check.failed(f'output {name} not found') + def execute_compute_nodes(context, compute_nodes): check.inst_param(context, 'context', ExecutionContext) check.list_param(compute_nodes, 'compute_nodes', of_type=ComputeNode) @@ -360,7 +386,15 @@ def topological_nodes(self): yield self.cn_dict[cn_guid] -def create_expectation_cn(solid, expectation_def, friendly_name, tag, prev_node_output_handle, inout_def): +def create_expectation_cn( + solid, + expectation_def, + friendly_name, + tag, + prev_node_output_handle, + inout_def, +): + check.inst_param(solid, 'solid', SolidDefinition) check.inst_param(expectation_def, 'input_expct_def', ExpectationDefinition) check.inst_param(prev_node_output_handle, 'prev_node_output_handle', ComputeNodeOutputHandle) diff --git a/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py b/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py index 7b0a3bebbee4c..fe561bbda8439 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py +++ b/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py @@ -1,4 +1,7 @@ +import pytest + from dagster import ( + DagsterInvariantViolationError, ExpectationDefinition, ExpectationResult, OutputDefinition, @@ -84,3 +87,46 @@ def _transform_fn(*_args, **_kwargs): assert result.success assert called['expectation_one'] assert called['expectation_two'] + + +def test_wrong_multiple_output(): + def _t_fn(_context, _inputs, _config_dict): + yield Result(output_name='mismatch', value='foo') + + solid = SolidDefinition( + name='multiple_outputs', + inputs=[], + outputs=[ + OutputDefinition(name='output_one'), + ], + config_def={}, + transform_fn=_t_fn, + ) + + pipeline = PipelineDefinition(solids=[solid]) + + with pytest.raises(DagsterInvariantViolationError): + execute_pipeline(pipeline, config.Environment()) + + +def test_multiple_outputs_of_same_name_disallowed(): + # make this illegal until it is supported + + def _t_fn(_context, _inputs, _config_dict): + yield Result(output_name='output_one', value='foo') + yield Result(output_name='output_one', value='foo') + + solid = SolidDefinition( + name='multiple_outputs', + inputs=[], + outputs=[ + OutputDefinition(name='output_one'), + ], + config_def={}, + transform_fn=_t_fn, + ) + + pipeline = PipelineDefinition(solids=[solid]) + + with pytest.raises(DagsterInvariantViolationError): + execute_pipeline(pipeline, config.Environment()) \ No newline at end of file From f2fa39b7b4b4ca8d4af81bd46476556cddffee50 Mon Sep 17 00:00:00 2001 From: Mikhail Novikov Date: Fri, 24 Aug 2018 15:57:00 +0300 Subject: [PATCH 018/103] Better pan and zoom --- .../dagit/webapp/src/graph/PipelineGraph.tsx | 51 ++++++++++++++++++- 1 file changed, 49 insertions(+), 2 deletions(-) diff --git a/python_modules/dagit/dagit/webapp/src/graph/PipelineGraph.tsx b/python_modules/dagit/dagit/webapp/src/graph/PipelineGraph.tsx index d690e2b0a3577..2b81268cd3aee 100644 --- a/python_modules/dagit/dagit/webapp/src/graph/PipelineGraph.tsx +++ b/python_modules/dagit/dagit/webapp/src/graph/PipelineGraph.tsx @@ -17,9 +17,14 @@ interface IPipelineGraphProps { onClickSolid?: (solidName: string) => void; } +interface IPipelineGraphState { + graphWidth: number | null; + graphHeight: number | null; +} + export default class PipelineGraph extends React.Component< IPipelineGraphProps, - {} + IPipelineGraphState > { static fragments = { PipelineGraphFragment: gql` @@ -34,6 +39,36 @@ export default class PipelineGraph extends React.Component< ` }; + state = { + graphWidth: null, + graphHeight: null + }; + + graphWrapper: React.RefObject = React.createRef(); + + componentDidMount() { + if (this.graphWrapper.current) { + this.setState({ + graphWidth: this.graphWrapper.current.clientWidth, + graphHeight: this.graphWrapper.current.clientHeight + }); + } + } + + componentDidUpdate() { + if (this.graphWrapper.current) { + if ( + this.state.graphWidth !== this.graphWrapper.current.clientWidth || + this.state.graphHeight !== this.graphWrapper.current.clientHeight + ) { + this.setState({ + graphWidth: this.graphWrapper.current.clientWidth, + graphHeight: this.graphWrapper.current.clientHeight + }); + } + } + } + renderSolids(layout: IFullPipelineLayout) { return this.props.pipeline.solids.map(solid => { const solidLayout = layout.solids[solid.name]; @@ -98,8 +133,20 @@ export default class PipelineGraph extends React.Component< render() { const layout = getDagrePipelineLayout(this.props.pipeline); + let minScale; + const { graphWidth, graphHeight } = this.state; + if (graphWidth !== null && graphHeight !== null) { + if (graphWidth > graphHeight) { + minScale = graphWidth / (layout.width + 200); + } else { + minScale = graphHeight / (layout.height - 300); + } + } else { + minScale = 0.1; + } + return ( - + Date: Thu, 23 Aug 2018 17:01:15 +0300 Subject: [PATCH 019/103] Decorator support for multiple outputs --- .../core/core_tests/test_compute_nodes.py | 2 +- .../core/core_tests/test_custom_context.py | 12 ++-- .../core/core_tests/test_decorators.py | 28 ++++---- .../dagster/dagster/core/decorators.py | 61 ++++++++++++----- .../pandas_hello_world/pipeline.py | 6 +- .../test_pandas_hello_world_library_slide.py | 2 +- .../pandas_kernel_tests/test_pandas_solids.py | 67 ++++++++++--------- .../test_pandas_user_error.py | 4 +- 8 files changed, 108 insertions(+), 74 deletions(-) diff --git a/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py b/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py index a67b88a68b576..67cc0c17fef4e 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py +++ b/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py @@ -22,7 +22,7 @@ def silencing_default_context(): } -@solid(name='noop', inputs=[], output=OutputDefinition()) +@solid(name='noop', inputs=[], outputs=[OutputDefinition()]) def noop_solid(): return 'foo' diff --git a/python_modules/dagster/dagster/core/core_tests/test_custom_context.py b/python_modules/dagster/dagster/core/core_tests/test_custom_context.py index 1fe54908f3a3d..d1e59dc8c7b0f 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_custom_context.py +++ b/python_modules/dagster/dagster/core/core_tests/test_custom_context.py @@ -22,7 +22,7 @@ def test_default_context(): @solid( inputs=[], - output=OutputDefinition(), + outputs=[OutputDefinition()], ) @with_context def default_context_transform(context): @@ -38,7 +38,7 @@ def default_context_transform(context): def test_default_context_with_log_level(): @solid( inputs=[], - output=OutputDefinition(), + outputs=[OutputDefinition()], ) @with_context def default_context_transform(context): @@ -62,7 +62,7 @@ def test_default_value(): def _get_args_test_solid(arg_name, arg_value): @solid( inputs=[], - output=OutputDefinition(), + outputs=[OutputDefinition()], ) @with_context def args_test(context): @@ -96,7 +96,7 @@ def args_test(context): def test_custom_contexts(): @solid( inputs=[], - output=OutputDefinition(), + outputs=[OutputDefinition()], ) @with_context def custom_context_transform(context): @@ -136,7 +136,7 @@ def test_yield_context(): @solid( inputs=[], - output=OutputDefinition(), + outputs=[OutputDefinition()], ) @with_context def custom_context_transform(context): @@ -176,7 +176,7 @@ def _yield_context(_pipeline, args): def test_invalid_context(): @solid( inputs=[], - output=OutputDefinition(), + outputs=[OutputDefinition()], ) def never_transform(): raise Exception('should never execute') diff --git a/python_modules/dagster/dagster/core/core_tests/test_decorators.py b/python_modules/dagster/dagster/core/core_tests/test_decorators.py index 22ef6188dcb2c..5e6b4653ea828 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_decorators.py +++ b/python_modules/dagster/dagster/core/core_tests/test_decorators.py @@ -34,7 +34,7 @@ def create_empty_test_env(): def test_solid(): - @solid() + @solid(outputs=[OutputDefinition()]) def hello_world(): return {'foo': 'bar'} @@ -50,7 +50,7 @@ def hello_world(): def test_solid_with_name(): - @solid(name="foobar") + @solid(name="foobar", outputs=[OutputDefinition()]) def hello_world(): return {'foo': 'bar'} @@ -66,7 +66,7 @@ def hello_world(): def test_solid_with_context(): - @solid(name="foobar") + @solid(name="foobar", outputs=[OutputDefinition()]) @with_context def hello_world(_context): return {'foo': 'bar'} @@ -83,7 +83,7 @@ def hello_world(_context): def test_solid_with_input(): - @solid(inputs=[InputDefinition(name="foo_to_foo")]) + @solid(inputs=[InputDefinition(name="foo_to_foo")], outputs=[OutputDefinition()]) def hello_world(foo_to_foo): return foo_to_foo @@ -109,14 +109,14 @@ def hello_world(foo_to_foo): def test_solid_definition_errors(): with pytest.raises(DagsterInvalidDefinitionError): - @solid(inputs=[InputDefinition(name="foo")], output=OutputDefinition()) + @solid(inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()]) @with_context def vargs(_context, foo, *args): pass with pytest.raises(DagsterInvalidDefinitionError): - @solid(inputs=[InputDefinition(name="foo")], output=OutputDefinition()) + @solid(inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()]) def wrong_name(bar): pass @@ -125,34 +125,34 @@ def wrong_name(bar): @solid( inputs=[InputDefinition(name="foo"), InputDefinition(name="bar")], - output=OutputDefinition() + outputs=[OutputDefinition()] ) def wrong_name_2(foo): pass with pytest.raises(DagsterInvalidDefinitionError): - @solid(inputs=[InputDefinition(name="foo")], output=OutputDefinition()) + @solid(inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()]) @with_context def no_context(foo): pass with pytest.raises(DagsterInvalidDefinitionError): - @solid(inputs=[InputDefinition(name="foo")], output=OutputDefinition()) + @solid(inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()]) def yes_context(_context, foo): pass with pytest.raises(DagsterInvalidDefinitionError): - @solid(inputs=[InputDefinition(name="foo")], output=OutputDefinition()) + @solid(inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()]) def extras(foo, bar): pass @solid( inputs=[InputDefinition(name="foo"), InputDefinition(name="bar")], - output=OutputDefinition() + outputs=[OutputDefinition()] ) def valid_kwargs(**kwargs): pass @@ -160,7 +160,7 @@ def valid_kwargs(**kwargs): @solid( inputs=[InputDefinition(name="foo"), InputDefinition(name="bar")], - output=OutputDefinition() + outputs=[OutputDefinition()] ) def valid(foo, bar): pass @@ -168,7 +168,7 @@ def valid(foo, bar): @solid( inputs=[InputDefinition(name="foo"), InputDefinition(name="bar")], - output=OutputDefinition() + outputs=[OutputDefinition()] ) @with_context def valid_rontext(context, foo, bar): @@ -177,7 +177,7 @@ def valid_rontext(context, foo, bar): @solid( inputs=[InputDefinition(name="foo"), InputDefinition(name="bar")], - output=OutputDefinition() + outputs=[OutputDefinition()] ) @with_context def valid_context_2(_context, foo, bar): diff --git a/python_modules/dagster/dagster/core/decorators.py b/python_modules/dagster/dagster/core/decorators.py index b10fd46105ca4..4bc1bfdc4ca47 100644 --- a/python_modules/dagster/dagster/core/decorators.py +++ b/python_modules/dagster/dagster/core/decorators.py @@ -1,4 +1,5 @@ import inspect +from collections import namedtuple from functools import wraps from dagster import check from .definitions import ( @@ -6,12 +7,22 @@ InputDefinition, OutputDefinition, DagsterInvalidDefinitionError, + Result, + check_argument_def_dict, ) # Error messages are long # pylint: disable=C0301 +class MultipleResults(namedtuple('_MultipleResults', 'results')): + def __new__(cls, results): + return super(MultipleResults, cls).__new__( + cls, + check.list_param(results, 'results', Result), + ) + + def with_context(fn): """Pass context as a first argument to a transform. """ @@ -28,15 +39,22 @@ def has_context(self): class _Solid: - def __init__(self, name=None, inputs=None, output=None, description=None): + def __init__( + self, + name=None, + inputs=None, + outputs=None, + description=None, + config_def=None, + ): self.name = check.opt_str_param(name, 'name') self.inputs = check.opt_list_param(inputs, 'inputs', InputDefinition) - - check.opt_inst_param(output, 'output', OutputDefinition) - if not output: - output = OutputDefinition() - self.output = output + self.outputs = check.opt_list_param(outputs, 'outputs', OutputDefinition) self.description = check.opt_str_param(description, 'description') + if config_def: + self.config_def = check_argument_def_dict(config_def) + else: + self.config_def = {} def __call__(self, fn): expect_context = getattr(fn, 'has_context', False) @@ -47,33 +65,46 @@ def __call__(self, fn): self.name = fn.__name__ _validate_transform_fn(self.name, fn, self.inputs, expect_context) - transform_fn = _create_transform_wrapper(fn, self.inputs, expect_context) - return SolidDefinition.single_output_transform( + transform_fn = _create_transform_wrapper(fn, self.inputs, self.outputs, expect_context) + return SolidDefinition( name=self.name, inputs=self.inputs, - output=self.output, + outputs=self.outputs, transform_fn=transform_fn, + config_def=self.config_def, description=self.description, ) -def solid(*, name=None, inputs=None, output=None, description=None): - return _Solid(name=name, inputs=inputs, output=output, description=description) +def solid(*, name=None, inputs=None, outputs=None, description=None): + return _Solid(name=name, inputs=inputs, outputs=outputs, description=description) -def _create_transform_wrapper(fn, inputs, include_context=False): +def _create_transform_wrapper(fn, inputs, outputs, include_context=False): input_names = [input.name for input in inputs] @wraps(fn) - def transform(context, args): + def transform(context, args, config): kwargs = {} for input_name in input_names: kwargs[input_name] = args[input_name] if include_context: - return fn(context, **kwargs) + result = fn(context, **kwargs) + else: + result = fn(**kwargs) + if inspect.isgenerator(result): + yield from result else: - return fn(**kwargs) + if isinstance(result, Result): + yield result + elif isinstance(result, MultipleResults): + yield from MultipleResults.results + elif len(outputs) == 1: + yield Result(value=result, output_name=outputs[0].name) + elif result is not None: + # XXX(freiksenet) + raise Exception('Output for a solid without an output.') return transform diff --git a/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py b/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py index 7a92fc1b71e56..1f51895ea21ed 100644 --- a/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py +++ b/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py @@ -11,7 +11,7 @@ @solid( inputs=[InputDefinition('num', dagster_pd.DataFrame)], - output=OutputDefinition(dagster_type=dagster_pd.DataFrame) + outputs=[OutputDefinition(dagster_type=dagster_pd.DataFrame)] ) def sum_solid(num): sum_df = num.copy() @@ -21,7 +21,7 @@ def sum_solid(num): @solid( inputs=[InputDefinition('sum_df', dagster_pd.DataFrame)], - output=OutputDefinition(dagster_type=dagster_pd.DataFrame) + outputs=[OutputDefinition(dagster_type=dagster_pd.DataFrame)] ) def sum_sq_solid(sum_df): sum_sq_df = sum_df.copy() @@ -31,7 +31,7 @@ def sum_sq_solid(sum_df): @solid( inputs=[InputDefinition('sum_sq_solid', dagster_pd.DataFrame)], - output=OutputDefinition(dagster_type=dagster_pd.DataFrame) + outputs=[OutputDefinition(dagster_type=dagster_pd.DataFrame)] ) def always_fails_solid(**_kwargs): raise Exception('I am a programmer and I make error') diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py index 52710017dc0bc..81862e5daba34 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py @@ -123,7 +123,7 @@ def transform_fn(_context, args): def create_decorator_based_solid(): @solid( inputs=[InputDefinition('num_csv', dagster_pd.DataFrame)], - output=OutputDefinition(dagster_type=dagster_pd.DataFrame), + outputs=[OutputDefinition(dagster_type=dagster_pd.DataFrame)], ) def hello_world(num_csv): num_csv['sum'] = num_csv['num1'] + num_csv['num2'] diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py index 4527f9acb52e9..abf7c25c9563e 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py @@ -173,7 +173,7 @@ def transform(_context, args): @solid( inputs=[InputDefinition('num_csv', dagster_pd.DataFrame)], - output=OutputDefinition(dagster_type=dagster_pd.DataFrame), + outputs=[OutputDefinition(dagster_type=dagster_pd.DataFrame)], ) def sum_table(num_csv): check.inst_param(num_csv, 'num_csv', pd.DataFrame) @@ -183,7 +183,7 @@ def sum_table(num_csv): @solid( inputs=[InputDefinition('sum_df', dagster_pd.DataFrame)], - output=OutputDefinition(dagster_type=dagster_pd.DataFrame), + outputs=[OutputDefinition(dagster_type=dagster_pd.DataFrame)], ) def sum_sq_table(sum_df): sum_df['sum_squared'] = sum_df['sum'] * sum_df['sum'] @@ -192,7 +192,7 @@ def sum_sq_table(sum_df): @solid( inputs=[InputDefinition('sum_table_renamed', dagster_pd.DataFrame)], - output=OutputDefinition(dagster_type=dagster_pd.DataFrame), + outputs=[OutputDefinition(dagster_type=dagster_pd.DataFrame)], ) def sum_sq_table_renamed_input(sum_table_renamed): sum_table_renamed['sum_squared'] = sum_table_renamed['sum'] * sum_table_renamed['sum'] @@ -386,8 +386,7 @@ def sum_mult_transform(_context, args): def test_pandas_in_memory_diamond_pipeline(): pipeline = create_diamond_pipeline() result = execute_pipeline( - pipeline, - environment=get_num_csv_environment(get_load_only_solids_config('load_csv')) + pipeline, environment=get_num_csv_environment(get_load_only_solids_config('load_csv')) ) assert result.result_named('sum_mult_table').transformed_value.to_dict('list') == { @@ -455,7 +454,7 @@ def test_pandas_output_intermediate_csv_files(): write_sum_table.name: { 'df': DependencyDefinition('sum_table'), }, - write_mult_table.name:{ + write_mult_table.name: { 'df': DependencyDefinition('mult_table'), } } @@ -500,8 +499,8 @@ def test_pandas_output_intermediate_csv_files(): injected_solids = { 'sum_mult_table': { - 'sum_table' : dagster_pd.load_csv_solid('load_sum_table'), - 'mult_table' : dagster_pd.load_csv_solid('load_mult_table'), + 'sum_table': dagster_pd.load_csv_solid('load_sum_table'), + 'mult_table': dagster_pd.load_csv_solid('load_mult_table'), } } @@ -555,23 +554,25 @@ def test_pandas_output_intermediate_parquet_files(): write_sum_table.name: { 'df': DependencyDefinition('sum_table'), }, - write_mult_table.name:{ + write_mult_table.name: { 'df': DependencyDefinition('mult_table'), } } ) - environment = get_num_csv_environment({ - 'load_csv': config.Solid({ - 'path': script_relative_path('num.csv'), - }), - write_sum_table.name: config.Solid({ - 'path': sum_file - }), - write_mult_table.name: config.Solid({ - 'path': mult_file - }), - }) + environment = get_num_csv_environment( + { + 'load_csv': config.Solid({ + 'path': script_relative_path('num.csv'), + }), + write_sum_table.name: config.Solid({ + 'path': sum_file + }), + write_mult_table.name: config.Solid({ + 'path': mult_file + }), + } + ) pipeline_result = execute_pipeline( pipeline, @@ -654,23 +655,25 @@ def test_pandas_multiple_outputs(): write_sum_mult_csv.name: { 'df': DependencyDefinition('sum_mult_table'), }, - write_sum_mult_parquet.name:{ + write_sum_mult_parquet.name: { 'df': DependencyDefinition('sum_mult_table'), } } ) - environment = get_num_csv_environment({ - 'load_csv': config.Solid({ - 'path': script_relative_path('num.csv'), - }), - write_sum_mult_csv.name: config.Solid({ - 'path': csv_file, - }), - write_sum_mult_parquet.name: config.Solid({ - 'path': parquet_file, - }), - }) + environment = get_num_csv_environment( + { + 'load_csv': config.Solid({ + 'path': script_relative_path('num.csv'), + }), + write_sum_mult_csv.name: config.Solid({ + 'path': csv_file, + }), + write_sum_mult_parquet.name: config.Solid({ + 'path': parquet_file, + }), + } + ) execute_pipeline(pipeline, environment) diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_user_error.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_user_error.py index 12875e58865d2..02dc821855a66 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_user_error.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_user_error.py @@ -37,7 +37,7 @@ def test_wrong_output_value(): @solid( name="test_wrong_output", inputs=[csv_input], - output=OutputDefinition(dagster_type=dagster_pd.DataFrame) + outputs=[OutputDefinition(dagster_type=dagster_pd.DataFrame)] ) def df_solid(num_csv): return 'not a dataframe' @@ -62,7 +62,7 @@ def test_wrong_input_value(): @solid( name="test_wrong_input", inputs=[InputDefinition('foo', dagster_pd.DataFrame)], - output=OutputDefinition(), + outputs=[OutputDefinition()], ) def df_solid(foo): return foo From 0e02bba1ff51c9f211ea0cba37c7d05d03444ceb Mon Sep 17 00:00:00 2001 From: Mikhail Novikov Date: Fri, 24 Aug 2018 12:06:43 +0300 Subject: [PATCH 020/103] Add output and testS --- .../core/core_tests/test_decorators.py | 84 +++++++++++++++++-- .../dagster/dagster/core/decorators.py | 19 +++-- .../dagster/dagster/core/execution.py | 26 +++--- 3 files changed, 104 insertions(+), 25 deletions(-) diff --git a/python_modules/dagster/dagster/core/core_tests/test_decorators.py b/python_modules/dagster/dagster/core/core_tests/test_decorators.py index 5e6b4653ea828..d51ae9cb5ed69 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_decorators.py +++ b/python_modules/dagster/dagster/core/core_tests/test_decorators.py @@ -6,12 +6,14 @@ OutputDefinition, PipelineDefinition, config, + Result, execute_pipeline, ) from dagster.core.errors import DagsterInvalidDefinitionError from dagster.core.decorators import ( solid, with_context, + MultipleResults, ) from dagster.core.execution import ( execute_single_solid, @@ -45,8 +47,79 @@ def hello_world(): ) assert result.success + assert len(result.result_list) == 1 + assert result.result_list[0].transformed_value['foo'] == 'bar' + + +def test_solid_one_output(): + @solid(output=OutputDefinition()) + def hello_world(): + return {'foo': 'bar'} + + result = execute_single_solid( + create_test_context(), + hello_world, + environment=create_empty_test_env(), + ) + + assert result.success + assert len(result.result_list) == 1 + assert result.result_list[0].transformed_value['foo'] == 'bar' - assert result.transformed_value['foo'] == 'bar' + +def test_solid_yield(): + @solid(output=OutputDefinition()) + def hello_world(): + yield Result(value={'foo': 'bar'}) + + result = execute_single_solid( + create_test_context(), + hello_world, + environment=create_empty_test_env(), + ) + + assert result.success + assert len(result.result_list) == 1 + assert result.result_list[0].transformed_value['foo'] == 'bar' + + +def test_solid_result_return(): + @solid(output=OutputDefinition()) + def hello_world(): + return Result(value={'foo': 'bar'}) + + result = execute_single_solid( + create_test_context(), + hello_world, + environment=create_empty_test_env(), + ) + + assert result.success + assert len(result.result_list) == 1 + assert result.result_list[0].transformed_value['foo'] == 'bar' + + +def test_solid_multiple_outputs(): + @solid(outputs=[ + OutputDefinition(name="left"), + OutputDefinition(name="right"), + ]) + def hello_world(): + return MultipleResults( + Result(value={'foo': 'left'}, output_name='left'), + Result(value={'foo': 'right'}, output_name='right') + ) + + result = execute_single_solid( + create_test_context(), + hello_world, + environment=create_empty_test_env(), + ) + + assert result.success + assert len(result.result_list) == 2 + assert result.result_list[0].transformed_value['foo'] == 'left' + assert result.result_list[1].transformed_value['foo'] == 'right' def test_solid_with_name(): @@ -61,8 +134,8 @@ def hello_world(): ) assert result.success - - assert result.transformed_value['foo'] == 'bar' + assert len(result.result_list) == 1 + assert result.result_list[0].transformed_value['foo'] == 'bar' def test_solid_with_context(): @@ -78,8 +151,8 @@ def hello_world(_context): ) assert result.success - - assert result.transformed_value['foo'] == 'bar' + assert len(result.result_list) == 1 + assert result.result_list[0].transformed_value['foo'] == 'bar' def test_solid_with_input(): @@ -102,7 +175,6 @@ def hello_world(foo_to_foo): result = pipeline_result.result_named('hello_world') assert result.success - assert result.transformed_value['foo'] == 'bar' diff --git a/python_modules/dagster/dagster/core/decorators.py b/python_modules/dagster/dagster/core/decorators.py index 4bc1bfdc4ca47..59bc70276f82b 100644 --- a/python_modules/dagster/dagster/core/decorators.py +++ b/python_modules/dagster/dagster/core/decorators.py @@ -16,10 +16,11 @@ class MultipleResults(namedtuple('_MultipleResults', 'results')): - def __new__(cls, results): + def __new__(cls, *results): return super(MultipleResults, cls).__new__( cls, - check.list_param(results, 'results', Result), + # XXX(freiksenet): should check.list_param accept tuples from rest? + check.opt_list_param(list(results), 'results', Result), ) @@ -44,12 +45,18 @@ def __init__( name=None, inputs=None, outputs=None, + output=None, description=None, config_def=None, ): self.name = check.opt_str_param(name, 'name') self.inputs = check.opt_list_param(inputs, 'inputs', InputDefinition) - self.outputs = check.opt_list_param(outputs, 'outputs', OutputDefinition) + + if output is not None and outputs is None: + self.outputs = [check.opt_inst_param(output, 'output', OutputDefinition)] + else: + self.outputs = check.opt_list_param(outputs, 'outputs', OutputDefinition) + self.description = check.opt_str_param(description, 'description') if config_def: self.config_def = check_argument_def_dict(config_def) @@ -76,8 +83,8 @@ def __call__(self, fn): ) -def solid(*, name=None, inputs=None, outputs=None, description=None): - return _Solid(name=name, inputs=inputs, outputs=outputs, description=description) +def solid(*, name=None, inputs=None, output=None, outputs=None, description=None): + return _Solid(name=name, inputs=inputs, output=output, outputs=outputs, description=description) def _create_transform_wrapper(fn, inputs, outputs, include_context=False): @@ -99,7 +106,7 @@ def transform(context, args, config): if isinstance(result, Result): yield result elif isinstance(result, MultipleResults): - yield from MultipleResults.results + yield from result.results elif len(outputs) == 1: yield Result(value=result, output_name=outputs[0].name) elif result is not None: diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index b8a062ccd6020..f122908ffaf8e 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -49,6 +49,7 @@ from .graph import ExecutionGraph + class DagsterPipelineExecutionResult: def __init__( self, @@ -56,9 +57,7 @@ def __init__( result_list, ): self.context = check.inst_param(context, 'context', ExecutionContext) - self.result_list = check.list_param( - result_list, 'result_list', of_type=ExecutionStepResult - ) + self.result_list = check.list_param(result_list, 'result_list', of_type=ExecutionStepResult) @property def success(self): @@ -73,7 +72,10 @@ def result_named(self, name): class ExecutionStepResult: - def __init__(self, *, success, context, transformed_value, name, dagster_user_exception, solid, tag, output_name): + def __init__( + self, *, success, context, transformed_value, name, dagster_user_exception, solid, tag, + output_name + ): self.success = check.bool_param(success, 'success') self.context = context self.transformed_value = transformed_value @@ -121,8 +123,7 @@ def copy_result_dict(result_dict): def _create_passthrough_context_definition(context): check.inst_param(context, 'context', ExecutionContext) context_definition = PipelineContextDefinition( - argument_def_dict={}, - context_fn=lambda _pipeline, _args: context + argument_def_dict={}, context_fn=lambda _pipeline, _args: context ) return {'default': context_definition} @@ -146,9 +147,7 @@ def execute_single_solid(context, solid, environment, throw_on_error=True): environment=single_solid_environment, ) - results = pipeline_result.result_list - check.invariant(len(results) == 1, 'must be one result got ' + str(len(results))) - return results[0] + return pipeline_result def _do_throw_on_error(execution_result): @@ -161,8 +160,10 @@ def _do_throw_on_error(execution_result): raise execution_result.dagster_user_exception + def _wrap_in_yield(thing): if isinstance(thing, ExecutionContext): + def _wrap(): yield thing @@ -196,8 +197,7 @@ def yield_context(self): args_to_pass = validate_args( self.pipeline.context_definitions[context_name].argument_def_dict, - self.environment.context.args, - 'pipeline {pipeline_name} context {context_name}'.format( + self.environment.context.args, 'pipeline {pipeline_name} context {context_name}'.format( pipeline_name=self.pipeline.name, context_name=context_name, ) @@ -227,12 +227,12 @@ def execute_pipeline_iterator(pipeline, environment): with env.yield_context() as context: return _execute_graph_iterator(context, execution_graph, env) + def _execute_graph_iterator(context, execution_graph, env): check.inst_param(context, 'context', ExecutionContext) check.inst_param(execution_graph, 'execution_graph', ExecutionGraph) check.inst_param(env, 'env', DagsterEnv) - cn_graph = create_compute_node_graph_from_env(execution_graph, env) cn_nodes = list(cn_graph.topological_nodes()) @@ -267,7 +267,6 @@ def _execute_graph_iterator(context, execution_graph, env): ) - def execute_pipeline( pipeline, environment, @@ -289,6 +288,7 @@ def execute_pipeline( env = DagsterEnv(execution_graph, environment) return _execute_graph(execution_graph, env, throw_on_error) + def _execute_graph( execution_graph, env, From a53a3fc48cef61b00d95ab3e8d12327453265acd Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Fri, 24 Aug 2018 07:24:19 -0700 Subject: [PATCH 021/103] Add MultipleResults.from_dict This allows the construction of a MultipleResults object from a dictionary. I suspect this will actually end up being the most used form of constructing one of these things. --- python_modules/dagster/dagster/__init__.py | 1 + .../core/core_tests/test_decorators.py | 27 +++++++++++++++++++ .../dagster/dagster/core/decorators.py | 8 ++++++ 3 files changed, 36 insertions(+) diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index 4f98d26986193..9ba3eae4fea07 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -18,6 +18,7 @@ ) from dagster.core.decorators import ( + MultipleResults, solid, with_context, ) diff --git a/python_modules/dagster/dagster/core/core_tests/test_decorators.py b/python_modules/dagster/dagster/core/core_tests/test_decorators.py index d51ae9cb5ed69..f6c1f46591ff4 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_decorators.py +++ b/python_modules/dagster/dagster/core/core_tests/test_decorators.py @@ -122,6 +122,33 @@ def hello_world(): assert result.result_list[1].transformed_value['foo'] == 'right' +def test_dict_multiple_outputs(): + @solid(outputs=[ + OutputDefinition(name="left"), + OutputDefinition(name="right"), + ]) + def hello_world(): + return MultipleResults.from_dict({ + 'left': { + 'foo': 'left' + }, + 'right': { + 'foo': 'right' + }, + }) + + result = execute_single_solid( + create_test_context(), + hello_world, + environment=create_empty_test_env(), + ) + + assert result.success + assert len(result.result_list) == 2 + assert result.result_list[0].transformed_value['foo'] == 'left' + assert result.result_list[1].transformed_value['foo'] == 'right' + + def test_solid_with_name(): @solid(name="foobar", outputs=[OutputDefinition()]) def hello_world(): diff --git a/python_modules/dagster/dagster/core/decorators.py b/python_modules/dagster/dagster/core/decorators.py index 59bc70276f82b..e43bb77da5fc2 100644 --- a/python_modules/dagster/dagster/core/decorators.py +++ b/python_modules/dagster/dagster/core/decorators.py @@ -23,6 +23,14 @@ def __new__(cls, *results): check.opt_list_param(list(results), 'results', Result), ) + @staticmethod + def from_dict(result_dict): + check.dict_param(result_dict, 'result_dict', key_type=str) + results = [] + for name, value in result_dict.items(): + results.append(Result(value, name)) + return MultipleResults(*results) + def with_context(fn): """Pass context as a first argument to a transform. From 1af779b6898335aed7b5e9f22c014fcd0b9fec9b Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Fri, 24 Aug 2018 09:01:59 -0700 Subject: [PATCH 022/103] Do some renames and definition changes 1) Add ConfigDefinition This is to future proof this API a bit. Currently it only contains the argument definition dictionary, but I suspect that will change. This will make that easier to manage. 2) Add ArgumentDefinitionDictionary class for internal use 3) Consistent naming output_def, input_def in SolidDefinition and handle classes --- python_modules/dagit/dagit/schema.py | 8 +- python_modules/dagster/dagster/__init__.py | 1 + .../dagster/dagster/cli/pipeline.py | 8 +- .../dagster/dagster/core/argument_handling.py | 5 +- .../dagster/dagster/core/compute_nodes.py | 18 ++-- .../core/core_tests/test_argument_handling.py | 8 +- .../core/core_tests/test_definition_errors.py | 2 - .../core/core_tests/test_multiple_outputs.py | 4 - .../core/core_tests/test_naming_collisions.py | 5 +- .../core/core_tests/test_solid_with_config.py | 9 +- .../dagster/dagster/core/decorators.py | 22 +++-- .../dagster/dagster/core/definitions.py | 83 +++++++++++-------- python_modules/dagster/dagster/core/graph.py | 12 +-- .../dagster/dagster/core/utility_solids.py | 1 - python_modules/dagster/dagster/graphviz.py | 4 +- .../dagster/dagster/pandas_kernel/__init__.py | 13 ++- .../test_pandas_hello_world_library_slide.py | 2 +- ...est_pandas_hello_world_no_library_slide.py | 9 +- .../pandas_kernel_tests/test_pandas_solids.py | 4 +- .../subquery_builder_experimental.py | 5 +- .../dagster/sqlalchemy_kernel/templated.py | 7 +- 21 files changed, 128 insertions(+), 102 deletions(-) diff --git a/python_modules/dagit/dagit/schema.py b/python_modules/dagit/dagit/schema.py index 53f193edd5828..5db207e4e869c 100644 --- a/python_modules/dagit/dagit/schema.py +++ b/python_modules/dagit/dagit/schema.py @@ -81,7 +81,7 @@ def __init__(self, solid, depends_on=None, depended_by=None): if depended_by: self._depended_by = { - output_handle.output.name: input_handles + output_handle.output_def.name: input_handles for output_handle, input_handles in depended_by.items() } else: @@ -90,19 +90,19 @@ def __init__(self, solid, depends_on=None, depended_by=None): def resolve_inputs(self, info): return [ Input(input_definition, self, self._depends_on.get(input_definition.name)) - for input_definition in self._solid.inputs + for input_definition in self._solid.input_defs ] def resolve_outputs(self, info): return [ Output(output_definition, self, self._depended_by.get(output_definition.name, [])) - for output_definition in self._solid.outputs + for output_definition in self._solid.output_defs ] def resolve_config(self, info): return [ Argument(name=name, argument=argument) - for name, argument in self._solid.config_dict_def.items() + for name, argument in self._solid.config_def.argument_def_dict.items() ] diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index 9ba3eae4fea07..760c618dedc43 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -6,6 +6,7 @@ from dagster.core.definitions import ( ArgumentDefinition, + ConfigDefinition, DependencyDefinition, ExpectationDefinition, ExpectationResult, diff --git a/python_modules/dagster/dagster/cli/pipeline.py b/python_modules/dagster/dagster/cli/pipeline.py index 0748e320ea0e9..6a12986bc7696 100644 --- a/python_modules/dagster/dagster/cli/pipeline.py +++ b/python_modules/dagster/dagster/cli/pipeline.py @@ -143,13 +143,13 @@ def print_solid(printer, solid): printer.line('Outputs:') - for output in solid.outputs: - print(output.name) + for output_def in solid.output_defs: + print(output_def.name) def print_inputs(printer, solid): printer.line('Inputs:') - for input_def in solid.inputs: + for input_def in solid.input_defs: with printer.with_indent(): printer.line('Input: {name}'.format(name=input_def.name)) @@ -248,7 +248,7 @@ def print_metrics_to_console(results, printer): printer('Metrics for {name}'.format(name=result.name)) - for input_def in result.solid.inputs: + for input_def in result.solid.input_defs: metrics_for_input = list( context.metrics_covering_context({ 'solid': result.name, diff --git a/python_modules/dagster/dagster/core/argument_handling.py b/python_modules/dagster/dagster/core/argument_handling.py index 66976f036de35..8fb7d3d56db48 100644 --- a/python_modules/dagster/dagster/core/argument_handling.py +++ b/python_modules/dagster/dagster/core/argument_handling.py @@ -1,10 +1,11 @@ from dagster import check + +from .definitions import ArgumentDefinitionDictionary from .errors import DagsterTypeError -from .definitions import check_argument_def_dict def validate_args(argument_def_dict, arg_dict, error_context_str): - check_argument_def_dict(argument_def_dict) + check.inst_param(argument_def_dict, 'argument_def_dict', ArgumentDefinitionDictionary) check.dict_param(arg_dict, 'arg_dict', key_type=str) check.str_param(error_context_str, 'error_context_str') diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index f8e63b212f680..dcf1111d3638b 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -41,12 +41,6 @@ from .types import DagsterType -def get_single_solid_output(solid): - check.inst_param(solid, 'solid', SolidDefinition) - check.invariant(len(solid.outputs) == 1) - return solid.outputs[0] - - class ComputeNodeOutputHandle(namedtuple('_ComputeNodeOutputHandle', 'compute_node output_name')): def __new__(cls, compute_node, output_name): return super(ComputeNodeOutputHandle, cls).__new__( @@ -286,7 +280,7 @@ def execute(self, context, inputs): raise DagsterInvariantViolationError( f'''Core transform for {self.solid.name} returned an output {result.output_name} that does not exist. The available - outputs are {list([output.name for output in self.solid.outputs])}''' + outputs are {list([output_def.name for output_def in self.solid.output_defs])}''' ) if result.output_name in seen_outputs: @@ -508,7 +502,7 @@ def create_compute_node_graph_from_env(execution_graph, env): for topo_solid in execution_graph.topological_solids: cn_inputs = [] - for input_def in topo_solid.inputs: + for input_def in topo_solid.input_defs: prev_cn_output_handle = _prev_node_handle( dependency_structure, topo_solid, @@ -540,7 +534,7 @@ def create_compute_node_graph_from_env(execution_graph, env): ) validated_config_args = validate_args( - topo_solid.config_dict_def, + topo_solid.config_def.argument_def_dict, env.config_dict_for_solid(topo_solid.name), 'config for solid {solid_name}'.format(solid_name=topo_solid.name), ) @@ -551,7 +545,7 @@ def create_compute_node_graph_from_env(execution_graph, env): validated_config_args, ) - for output_def in topo_solid.outputs: + for output_def in topo_solid.output_defs: output_handle = topo_solid.output_handle(output_def.name) if env.evaluate_expectations and output_def.expectations: expectations_graph = create_expectations_cn_graph( @@ -652,8 +646,8 @@ def create_compute_node_from_solid_transform(solid, node_inputs, config_args): friendly_name=f'{solid.name}.transform', node_inputs=node_inputs, node_outputs=[ - ComputeNodeOutput(name=output.name, dagster_type=output.dagster_type) - for output in solid.outputs + ComputeNodeOutput(name=output_def.name, dagster_type=output_def.dagster_type) + for output_def in solid.output_defs ], arg_dict={}, compute_fn=lambda context, inputs: _execute_core_transform( diff --git a/python_modules/dagster/dagster/core/core_tests/test_argument_handling.py b/python_modules/dagster/dagster/core/core_tests/test_argument_handling.py index 992ee9bb6f576..447f3c92c3c65 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_argument_handling.py +++ b/python_modules/dagster/dagster/core/core_tests/test_argument_handling.py @@ -1,12 +1,16 @@ import pytest -from dagster import (ArgumentDefinition, types) +from dagster import ( + ArgumentDefinition, + types, +) from dagster.core.argument_handling import validate_args +from dagster.core.definitions import ArgumentDefinitionDictionary from dagster.core.errors import DagsterTypeError def _validate(argument_def_dict, arg_dict): - return validate_args(argument_def_dict, arg_dict, 'dummy') + return validate_args(ArgumentDefinitionDictionary(argument_def_dict), arg_dict, 'dummy') def _single_required_string_arg_def_dict(): diff --git a/python_modules/dagster/dagster/core/core_tests/test_definition_errors.py b/python_modules/dagster/dagster/core/core_tests/test_definition_errors.py index 1c71c397bebdb..55a9286fbca71 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_definition_errors.py +++ b/python_modules/dagster/dagster/core/core_tests/test_definition_errors.py @@ -17,14 +17,12 @@ def solid_a_b_list(): inputs=[], outputs=[OutputDefinition()], transform_fn=lambda context, inputs, config: None, - config_def={}, ), SolidDefinition( name='B', inputs=[InputDefinition('b_input')], outputs=[], transform_fn=lambda context, inputs, config: None, - config_def={}, ) ] diff --git a/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py b/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py index fe561bbda8439..81029c5248a25 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py +++ b/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py @@ -25,7 +25,6 @@ def _t_fn(_context, _inputs, _config_dict): OutputDefinition(name='output_one'), OutputDefinition(name='output_two'), ], - config_def={}, transform_fn=_t_fn, ) @@ -76,7 +75,6 @@ def _transform_fn(*_args, **_kwargs): ], ), ], - config_def={}, transform_fn=_transform_fn, ) @@ -99,7 +97,6 @@ def _t_fn(_context, _inputs, _config_dict): outputs=[ OutputDefinition(name='output_one'), ], - config_def={}, transform_fn=_t_fn, ) @@ -122,7 +119,6 @@ def _t_fn(_context, _inputs, _config_dict): outputs=[ OutputDefinition(name='output_one'), ], - config_def={}, transform_fn=_t_fn, ) diff --git a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py index 93b7608283ec7..c8f723bd69587 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py +++ b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py @@ -2,6 +2,7 @@ from dagster import ( ArgumentDefinition, + ConfigDefinition, DependencyDefinition, InputDefinition, OutputDefinition, @@ -27,7 +28,9 @@ def _value_t_fn(_context, _inputs, config_dict): description=description, inputs=[], outputs=[OutputDefinition(dagster_type=types.String)], - config_def={'value': ArgumentDefinition(types.String)}, + config_def=ConfigDefinition({ + 'value': ArgumentDefinition(types.String) + }), transform_fn=_value_t_fn, ) diff --git a/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py b/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py index 25fb00fb0f4d3..63a2da35b4fea 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py +++ b/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py @@ -2,6 +2,7 @@ from dagster import ( ArgumentDefinition, + ConfigDefinition, PipelineDefinition, SolidDefinition, config, @@ -22,7 +23,9 @@ def _t_fn(_context, _inputs, config_dict): name='with_context', inputs=[], outputs=[], - config_def={'some_config': ArgumentDefinition(types.String)}, + config_def=ConfigDefinition({ + 'some_config': ArgumentDefinition(types.String) + }), transform_fn=_t_fn, ) @@ -47,7 +50,9 @@ def _t_fn(_context, _inputs, _config_dict): name='with_context', inputs=[], outputs=[], - config_def={'some_config': ArgumentDefinition(types.String)}, + config_def=ConfigDefinition({ + 'some_config': ArgumentDefinition(types.String) + }), transform_fn=_t_fn, ) diff --git a/python_modules/dagster/dagster/core/decorators.py b/python_modules/dagster/dagster/core/decorators.py index e43bb77da5fc2..5acbbf56c6c34 100644 --- a/python_modules/dagster/dagster/core/decorators.py +++ b/python_modules/dagster/dagster/core/decorators.py @@ -1,14 +1,15 @@ import inspect from collections import namedtuple from functools import wraps -from dagster import check + from .definitions import ( - SolidDefinition, + ConfigDefinition, + DagsterInvalidDefinitionError, InputDefinition, OutputDefinition, - DagsterInvalidDefinitionError, Result, - check_argument_def_dict, + SolidDefinition, + check, ) # Error messages are long @@ -58,7 +59,7 @@ def __init__( config_def=None, ): self.name = check.opt_str_param(name, 'name') - self.inputs = check.opt_list_param(inputs, 'inputs', InputDefinition) + self.input_defs = check.opt_list_param(inputs, 'inputs', InputDefinition) if output is not None and outputs is None: self.outputs = [check.opt_inst_param(output, 'output', OutputDefinition)] @@ -66,10 +67,7 @@ def __init__( self.outputs = check.opt_list_param(outputs, 'outputs', OutputDefinition) self.description = check.opt_str_param(description, 'description') - if config_def: - self.config_def = check_argument_def_dict(config_def) - else: - self.config_def = {} + self.config_def = check.opt_inst_param(config_def, 'config_def', ConfigDefinition({})) def __call__(self, fn): expect_context = getattr(fn, 'has_context', False) @@ -79,11 +77,11 @@ def __call__(self, fn): if not self.name: self.name = fn.__name__ - _validate_transform_fn(self.name, fn, self.inputs, expect_context) - transform_fn = _create_transform_wrapper(fn, self.inputs, self.outputs, expect_context) + _validate_transform_fn(self.name, fn, self.input_defs, expect_context) + transform_fn = _create_transform_wrapper(fn, self.input_defs, self.outputs, expect_context) return SolidDefinition( name=self.name, - inputs=self.inputs, + inputs=self.input_defs, outputs=self.outputs, transform_fn=transform_fn, config_def=self.config_def, diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index 1023692b21879..e3a70da65b182 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -47,18 +47,23 @@ def check_valid_name(name): return name -def check_argument_def_dict(config_def_dict): - return check.dict_param( - config_def_dict, - 'config_def_dict', - key_type=str, - value_type=ArgumentDefinition, - ) +# We wrap the passed in dictionary of str : ArgumentDefinition to +# 1) enforce typing +# 2) enforce immutability +# 3) make type checks throughout execution cheaper +class ArgumentDefinitionDictionary(dict): + def __init__(self, ddict): + super().__init__( + check.dict_param(ddict, 'ddict', key_type=str, value_type=ArgumentDefinition) + ) + + def __setitem__(self, _key, _value): + check.failed('This dictionary is readonly') class PipelineContextDefinition: def __init__(self, *, argument_def_dict, context_fn, description=None): - self.argument_def_dict = check_argument_def_dict(argument_def_dict) + self.argument_def_dict = ArgumentDefinitionDictionary(argument_def_dict) self.context_fn = check.callable_param(context_fn, 'context_fn') self.description = description @@ -263,7 +268,7 @@ def __init__( if not self._solid_dict[from_solid].has_input(from_input): input_list = [ - input_def.name for input_def in self._solid_dict[from_solid].inputs + input_def.name for input_def in self._solid_dict[from_solid].input_defs ] raise DagsterInvalidDefinitionError( f'Solid {from_solid} does not have input {from_input}. ' + \ @@ -283,7 +288,7 @@ def __init__( self.dependency_structure = DependencyStructure.from_definitions(solids, dependencies) for solid in solids: - for input_def in solid.inputs: + for input_def in solid.input_defs: if not self.dependency_structure.has_dep(solid.input_handle(input_def.name)): if name: raise DagsterInvalidDefinitionError( @@ -405,25 +410,25 @@ def __eq__(self, other): return self.solid.name == other.solid.name and self.input_def.name == other.input_def.name -class SolidOutputHandle(namedtuple('_SolidOutputHandle', 'solid output')): - def __new__(cls, solid, output): +class SolidOutputHandle(namedtuple('_SolidOutputHandle', 'solid output_def')): + def __new__(cls, solid, output_def): return super(SolidOutputHandle, cls).__new__( cls, check.inst_param(solid, 'solid', SolidDefinition), - check.inst_param(output, 'output', OutputDefinition), + check.inst_param(output_def, 'output_def', OutputDefinition), ) def __str__(self): - return f'SolidOutputHandle(solid="{self.solid.name}", output.name="{self.output.name}")' + return f'SolidOutputHandle(solid="{self.solid.name}", output.name="{self.output_def.name}")' def __repr__(self): - return f'SolidOutputHandle(solid="{self.solid.name}", output.name="{self.output.name}")' + return f'SolidOutputHandle(solid="{self.solid.name}", output.name="{self.output_def.name}")' def __hash__(self): - return hash((self.solid.name, self.output.name)) + return hash((self.solid.name, self.output_def.name)) def __eq__(self, other): - return self.solid.name == other.solid.name and self.output.name == other.output.name + return self.solid.name == other.solid.name and self.output_def.name == other.output_def.name class Result(namedtuple('_Result', 'value output_name')): @@ -435,29 +440,43 @@ def __new__(cls, value, output_name=DEFAULT_OUTPUT): ) +class ConfigDefinition: + def __init__(self, argument_def_dict): + self.argument_def_dict = ArgumentDefinitionDictionary(argument_def_dict) + + class SolidDefinition: - def __init__(self, *, name, inputs, transform_fn, outputs, config_def, description=None): + def __init__(self, *, name, inputs, transform_fn, outputs, config_def=None, description=None): self.name = check_valid_name(name) - self.inputs = check.list_param(inputs, 'inputs', InputDefinition) + self.input_defs = check.list_param(inputs, 'inputs', InputDefinition) self.transform_fn = check.callable_param(transform_fn, 'transform_fn') - self.outputs = check.list_param(outputs, 'outputs', OutputDefinition) + self.output_defs = check.list_param(outputs, 'outputs', OutputDefinition) self.description = check.opt_str_param(description, 'description') - self.config_dict_def = check_argument_def_dict(config_def) + self.config_def = check.opt_inst_param( + config_def, + 'config_def', + ConfigDefinition, + ConfigDefinition({}), + ) input_handles = {} - for inp in self.inputs: - input_handles[inp.name] = SolidInputHandle(self, inp) + for input_def in self.input_defs: + input_handles[input_def.name] = SolidInputHandle(self, input_def) - self.input_handles = input_handles + self._input_handles = input_handles output_handles = {} - for output in outputs: - output_handles[output.name] = SolidOutputHandle(self, output) + for output_def in self.output_defs: + output_handles[output_def.name] = SolidOutputHandle(self, output_def) - self.output_handles = output_handles + self._output_handles = output_handles self._input_dict = _build_named_dict(inputs) self._output_dict = _build_named_dict(outputs) + @property + def outputs(self): + return self.output_defs + @staticmethod def single_output_transform( name, inputs, transform_fn, output, config_def=None, description=None @@ -471,21 +490,17 @@ def _new_transform_fn(context, inputs, _config_dict): inputs=inputs, transform_fn=_new_transform_fn, outputs=[output], - config_def=check.opt_dict_param(config_def, 'config_def'), + config_def=config_def, description=description, ) def input_handle(self, name): check.str_param(name, 'name') - return self.input_handles[name] + return self._input_handles[name] def output_handle(self, name): check.str_param(name, 'name') - return self.output_handles[name] - - @property - def input_names(self): - return [inp.name for inp in self.inputs] + return self._output_handles[name] def has_input(self, name): check.str_param(name, 'name') diff --git a/python_modules/dagster/dagster/core/graph.py b/python_modules/dagster/dagster/core/graph.py index b0c8bf2db6838..5bf92febf5110 100644 --- a/python_modules/dagster/dagster/core/graph.py +++ b/python_modules/dagster/dagster/core/graph.py @@ -47,7 +47,7 @@ def _dependency_structure_to_dep_dict(dependency_structure): for input_handle, output_handle in dependency_structure.items(): dep_dict[input_handle.solid.name][input_handle.input_def.name] = DependencyDefinition( solid=output_handle.solid.name, - output=output_handle.output.name, + output=output_handle.output_def.name, ) return dep_dict @@ -143,8 +143,8 @@ def transitive_dependencies_of(self, solid_name): trans_deps = set() solid = self._solid_dict[solid_name] - for inp in solid.inputs: - input_handle = solid.input_handle(inp.name) + for input_def in solid.input_defs: + input_handle = solid.input_handle(input_def.name) if self.dependency_structure.has_dep(input_handle): output_handle = self.dependency_structure.get_dep(input_handle) trans_deps.add(output_handle.solid.name) @@ -171,7 +171,7 @@ def create_execution_subgraph(self, from_solids, to_solids): handle_dict = InputToOutputHandleDict() for solid in involved_solids: - for input_def in solid.inputs: + for input_def in solid.input_defs: input_handle = solid.input_handle(input_def.name) if self.dependency_structure.has_dep(input_handle): handle_dict[input_handle] = self.dependency_structure.get_dep(input_handle) @@ -188,7 +188,7 @@ def visit(solid): involved_solid_set.add(solid.name) - for input_def in solid.inputs: + for input_def in solid.input_defs: input_handle = solid.input_handle(input_def.name) if not self.dependency_structure.has_dep(input_handle): continue @@ -220,7 +220,7 @@ def _all_depended_on_solids(execution_graph): dependency_structure = execution_graph.dependency_structure for solid in execution_graph.solids: - for input_def in solid.inputs: + for input_def in solid.input_defs: input_handle = solid.input_handle(input_def.name) if dependency_structure.has_dep(input_handle): output_handle = dependency_structure.get_dep(input_handle) diff --git a/python_modules/dagster/dagster/core/utility_solids.py b/python_modules/dagster/dagster/core/utility_solids.py index c05bff14582fc..9c0279b1df9bf 100644 --- a/python_modules/dagster/dagster/core/utility_solids.py +++ b/python_modules/dagster/dagster/core/utility_solids.py @@ -18,6 +18,5 @@ def _value_t_fn(_context, _inputs, _config_dict): name=name, inputs=[], outputs=[OutputDefinition()], - config_def={}, transform_fn=_value_t_fn, ) diff --git a/python_modules/dagster/dagster/graphviz.py b/python_modules/dagster/dagster/graphviz.py index 59db3b79d1514..be26fb7bae2b8 100644 --- a/python_modules/dagster/dagster/graphviz.py +++ b/python_modules/dagster/dagster/graphviz.py @@ -14,12 +14,12 @@ def build_graphviz_graph(pipeline): graphviz_graph.attr('node', color='grey', shape='box') for solid in pipeline.solids: - for input_def in solid.inputs: + for input_def in solid.input_defs: scoped_name = solid.name + '.' + input_def.name graphviz_graph.node(scoped_name) for solid in pipeline.solids: - for input_def in solid.inputs: + for input_def in solid.input_defs: scoped_name = solid.name + '.' + input_def.name graphviz_graph.edge(scoped_name, solid.name) diff --git a/python_modules/dagster/dagster/pandas_kernel/__init__.py b/python_modules/dagster/dagster/pandas_kernel/__init__.py index b5bcc000ad44d..d6117ca6cc87a 100644 --- a/python_modules/dagster/dagster/pandas_kernel/__init__.py +++ b/python_modules/dagster/dagster/pandas_kernel/__init__.py @@ -5,6 +5,7 @@ from dagster import ( ArgumentDefinition, + ConfigDefinition, ExecutionContext, InputDefinition, OutputDefinition, @@ -40,9 +41,9 @@ def _t_fn(_context, _inputs, config_dict): inputs=[], outputs=[OutputDefinition(dagster_type=DataFrame)], transform_fn=_t_fn, - config_def={ + config_def=ConfigDefinition({ 'path': ArgumentDefinition(types.Path), - } + }), ) @@ -54,7 +55,9 @@ def _t_fn(_context, inputs, config_dict): name=name, inputs=[InputDefinition('df', DataFrame)], outputs=[], - config_def={'path': ArgumentDefinition(types.Path)}, + config_def=ConfigDefinition({ + 'path': ArgumentDefinition(types.Path) + }), transform_fn=_t_fn, ) @@ -67,6 +70,8 @@ def _t_fn(_context, inputs, config_dict): name=name, inputs=[InputDefinition('df', DataFrame)], outputs=[], - config_def={'path': ArgumentDefinition(types.Path)}, + config_def=ConfigDefinition({ + 'path': ArgumentDefinition(types.Path) + }), transform_fn=_t_fn, ) diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py index 81862e5daba34..ebb67391c9e00 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py @@ -30,7 +30,7 @@ def test_hello_world_with_dataframe_fns(): def run_hello_world(hello_world): - assert len(hello_world.inputs) == 1 + assert len(hello_world.input_defs) == 1 pipeline = PipelineDefinition( solids=[ diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py index c9a86b3a12c3a..e79ba5f037304 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py @@ -2,6 +2,7 @@ from dagster import ( ArgumentDefinition, + ConfigDefinition, DependencyDefinition, InputDefinition, OutputDefinition, @@ -24,7 +25,9 @@ def _t_fn(_context, _inputs, config_dict): name=name, inputs=[], outputs=[OutputDefinition()], - config_def={'path': ArgumentDefinition(types.Path)}, + config_def=ConfigDefinition({ + 'path': ArgumentDefinition(types.Path) + }), transform_fn=_t_fn ) @@ -37,7 +40,9 @@ def _t_fn(_context, inputs, config_dict): name=name, inputs=[InputDefinition('df')], outputs=[], - config_def={'path': ArgumentDefinition(types.Path)}, + config_def=ConfigDefinition({ + 'path': ArgumentDefinition(types.Path) + }), transform_fn=_t_fn, ) diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py index abf7c25c9563e..92e57b518dd7c 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py @@ -36,7 +36,7 @@ def get_solid_transformed_value(_context, solid_inst, environment): solids=[dagster_pd.load_csv_solid('load_csv'), solid_inst], dependencies={ solid_inst.name: { - solid_inst.inputs[0].name: DependencyDefinition('load_csv'), + solid_inst.input_defs[0].name: DependencyDefinition('load_csv'), } } ) @@ -128,7 +128,7 @@ def execute_transform_in_temp_csv_files(solid_inst): solids=[load_csv_solid, solid_inst, to_csv_solid], dependencies={ solid_inst.name: { - solid_inst.inputs[0].name: DependencyDefinition('load_csv'), + solid_inst.input_defs[0].name: DependencyDefinition('load_csv'), }, 'to_csv': { 'df': DependencyDefinition(solid_inst.name), diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/subquery_builder_experimental.py b/python_modules/dagster/dagster/sqlalchemy_kernel/subquery_builder_experimental.py index a947852d31e79..fffa4a3a07170 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/subquery_builder_experimental.py +++ b/python_modules/dagster/dagster/sqlalchemy_kernel/subquery_builder_experimental.py @@ -2,6 +2,7 @@ from dagster import ( ArgumentDefinition, + ConfigDefinition, InputDefinition, OutputDefinition, SolidDefinition, @@ -61,9 +62,9 @@ def _materialization_fn(context, inputs, config_dict): inputs=[InputDefinition('expr')], outputs=[], transform_fn=_materialization_fn, - config_def={ + config_def=ConfigDefinition({ 'table_name': ArgumentDefinition(types.String), - } + }), ) diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/templated.py b/python_modules/dagster/dagster/sqlalchemy_kernel/templated.py index 6405433af94c8..619ffc030f1d0 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/templated.py +++ b/python_modules/dagster/dagster/sqlalchemy_kernel/templated.py @@ -2,6 +2,7 @@ from dagster import ( ArgumentDefinition, + ConfigDefinition, InputDefinition, OutputDefinition, Result, @@ -22,14 +23,14 @@ def create_templated_sql_transform_solid(name, sql, table_arguments, dependant_s dependant_solids, 'dependant_solids', of_type=SolidDefinition ) - config_def = {} + argument_def_dict = {} for table in table_arguments: - config_def[table] = ArgumentDefinition(types.String) + argument_def_dict[table] = ArgumentDefinition(types.String) return SolidDefinition( name=name, inputs=[InputDefinition(solid.name) for solid in dependant_solids], - config_def=config_def, + config_def=ConfigDefinition(argument_def_dict), transform_fn=_create_templated_sql_transform_with_output(sql), outputs=[OutputDefinition()], ) From 84b1f5dc6d41ce33070aa2aa7ab314b2e07eaf0d Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Fri, 24 Aug 2018 15:55:46 -0700 Subject: [PATCH 023/103] fix missed output --> output_def rename --- python_modules/dagit/dagit/schema.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python_modules/dagit/dagit/schema.py b/python_modules/dagit/dagit/schema.py index 5db207e4e869c..a0ecb8fb1967e 100644 --- a/python_modules/dagit/dagit/schema.py +++ b/python_modules/dagit/dagit/schema.py @@ -134,7 +134,7 @@ def resolve_expectations(self, info): def resolve_depends_on(self, info): return Output( - self._depends_on.output, + self._depends_on.output_def, Solid(self._depends_on.solid), # XXX(freiksenet): This is not right [] From 1dac8113399f51db6c6259cbf73f64df84e40b21 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Sat, 25 Aug 2018 16:54:28 -0700 Subject: [PATCH 024/103] Factor validation out of PipelineDefinition.__init__ --- .../dagster/dagster/core/definitions.py | 78 ++++++++++--------- 1 file changed, 42 insertions(+), 36 deletions(-) diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index e3a70da65b182..11c5913db64bc 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -222,45 +222,14 @@ def create_sub_pipeline(pipeline, from_solids, through_solids, injected_solids=N return subgraph.to_pipeline() - def __init__( - self, solids, name=None, description=None, context_definitions=None, dependencies=None - ): - self.description = check.opt_str_param(description, 'description') - self.name = check.opt_str_param(name, 'name') - - if context_definitions is None: - context_definitions = _default_pipeline_context_definitions() - - self.context_definitions = check.dict_param( - context_definitions, - 'context_definitions', - key_type=str, - value_type=PipelineContextDefinition, - ) - - for solid in solids: - if not isinstance(solid, SolidDefinition) and callable(solid): - raise DagsterInvalidDefinitionError( - '''You have passed a lambda or function {solid} into - a pipeline that is not a solid. You have likely forgetten to annotate this function - with an @solid decorator located in dagster.core.decorators - '''.format(solid=solid.__name__) - ) - - self._solid_dict = _build_named_dict(solids) - - dependencies = check_two_dim_str_dict( - dependencies, - 'dependencies', - DependencyDefinition, - ) if dependencies else {} - + def __validate_dependences(self, dependencies): for from_solid, dep_by_input in dependencies.items(): for from_input, dep in dep_by_input.items(): if from_solid == dep.solid: raise DagsterInvalidDefinitionError( f'Circular reference detected in solid {from_solid} input {from_input}.' ) + if not from_solid in self._solid_dict: raise DagsterInvalidDefinitionError( f'Solid {from_solid} in dependency dictionary not found in solid list', @@ -285,11 +254,10 @@ def __init__( f'Solid {dep.solid} does not have output {dep.output}', ) - self.dependency_structure = DependencyStructure.from_definitions(solids, dependencies) - + def __validate_dependency_structure(self, name, solids, dependency_structure): for solid in solids: for input_def in solid.input_defs: - if not self.dependency_structure.has_dep(solid.input_handle(input_def.name)): + if not dependency_structure.has_dep(solid.input_handle(input_def.name)): if name: raise DagsterInvalidDefinitionError( f'Dependency must be specified for solid {solid.name} input ' + \ @@ -301,6 +269,44 @@ def __init__( f'{input_def.name}' ) + def __init__( + self, solids, name=None, description=None, context_definitions=None, dependencies=None + ): + self.description = check.opt_str_param(description, 'description') + self.name = check.opt_str_param(name, 'name') + + if context_definitions is None: + context_definitions = _default_pipeline_context_definitions() + + self.context_definitions = check.dict_param( + context_definitions, + 'context_definitions', + key_type=str, + value_type=PipelineContextDefinition, + ) + + for solid in solids: + if not isinstance(solid, SolidDefinition) and callable(solid): + raise DagsterInvalidDefinitionError( + '''You have passed a lambda or function {func} into + a pipeline that is not a solid. You have likely forgetten to annotate this function + with an @solid decorator located in dagster.core.decorators + '''.format(func=solid.__name__) + ) + + self._solid_dict = _build_named_dict(solids) + + dependencies = check_two_dim_str_dict( + dependencies, + 'dependencies', + DependencyDefinition, + ) if dependencies else {} + + self.__validate_dependences(dependencies) + dependency_structure = DependencyStructure.from_definitions(solids, dependencies) + self.__validate_dependency_structure(name, solids, dependency_structure) + self.dependency_structure = dependency_structure + @property def solids(self): return list(self._solid_dict.values()) From 840be677be6a074c080220ae6d60a2e8ca9a335d Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Sat, 25 Aug 2018 11:55:14 -0700 Subject: [PATCH 025/103] Improve error messages for transform functions If someone returns the wrong thing from a transform function we should communicate reasonable error messages --- .../dagster/dagster/core/compute_nodes.py | 55 ++++++++++++------- .../core/core_tests/test_pipeline_errors.py | 38 +++++++++++++ 2 files changed, 74 insertions(+), 19 deletions(-) diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index dcf1111d3638b..7b86281a3037a 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -151,27 +151,45 @@ class ComputeNodeTag(Enum): EXPECTATION_INPUT = 'expectation_input' -def _execute_core_transform(context, solid_name, solid_transform_fn, values_dict, config_dict): +def _execute_core_transform(context, compute_node, values_dict, config_dict): ''' Execute the user-specified transform for the solid. Wrap in an error boundary and do all relevant logging and metrics tracking ''' check.inst_param(context, 'context', ExecutionContext) - check.str_param(solid_name, 'solid_name') - check.callable_param(solid_transform_fn, 'solid_transform_fn') + check.inst_param(compute_node, 'compute_node', ComputeNode) check.dict_param(values_dict, 'values_dict', key_type=str) check.dict_param(config_dict, 'config_dict', key_type=str) error_str = 'Error occured during core transform' - with _user_code_error_boundary(context, error_str): - with time_execution_scope() as timer_result: - with context.value('solid', solid_name): - gen = solid_transform_fn(context, values_dict, config_dict) - if gen is not None: - for result in gen: - yield result + with _user_code_error_boundary(context, error_str), \ + time_execution_scope() as timer_result, \ + context.value('solid', compute_node.solid.name): + + gen = compute_node.solid.transform_fn(context, values_dict, config_dict) + + if isinstance(gen, Result): + raise DagsterInvariantViolationError( + ('Transform for solid {solid_name} returned a Result rather than ' + + 'yielding it. The transform_fn of the core SolidDefinition must yield ' + + 'its results').format( + solid_name=compute_node.solid.name, + ) + ) + + if gen is not None: + for result in gen: + if not isinstance(result, Result): + raise DagsterInvariantViolationError( + ('Transform for solid {solid_name} yielded {result} rather an ' + + 'an instance of the Result class.').format( + result=repr(result), + solid_name=compute_node.solid.name, + ) + ) + yield result - context.metric('core_transform_time_ms', timer_result.millis) + context.metric('core_transform_time_ms', timer_result.millis) class ComputeNodeInput: @@ -267,7 +285,7 @@ def execute(self, context, inputs): try: with _user_code_error_boundary(context, error_str): - gen = self.compute_fn(context, inputs) + gen = self.compute_fn(context, self, inputs) if gen is None: check.invariant(not self.node_outputs) @@ -342,8 +360,8 @@ def execute_compute_nodes(context, compute_nodes): def _yieldify(sync_compute_fn): - def _wrap(context, inputs): - yield sync_compute_fn(context, inputs) + def _wrap(context, compute_node, inputs): + yield sync_compute_fn(context, compute_node, inputs) return _wrap @@ -614,7 +632,7 @@ def _create_join_node(solid, prev_nodes, prev_output_name): ExpectationExecutionInfo = namedtuple('ExpectationExecutionInfo', 'solid expectation_def') -def _create_join_lambda(_context, inputs): +def _create_join_lambda(_context, _compute_node, inputs): return Result(output_name=JOIN_OUTPUT, value=list(inputs.values())[0]) @@ -623,7 +641,7 @@ def _create_expectation_lambda(solid, expectation_def, output_name): check.inst_param(expectation_def, 'expectations_def', ExpectationDefinition) check.str_param(output_name, 'output_name') - def _do_expectation(context, inputs): + def _do_expectation(context, _compute_node, inputs): expt_result = expectation_def.expectation_fn( context, ExpectationExecutionInfo(solid, expectation_def), @@ -650,10 +668,9 @@ def create_compute_node_from_solid_transform(solid, node_inputs, config_args): for output_def in solid.output_defs ], arg_dict={}, - compute_fn=lambda context, inputs: _execute_core_transform( + compute_fn=lambda context, compute_node, inputs: _execute_core_transform( context, - solid.name, - solid.transform_fn, + compute_node, inputs, config_args, ), diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py index 522c700e367ff..8c3f1051b9e57 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py @@ -1,16 +1,21 @@ +import pytest + from dagster import ( + DagsterInvariantViolationError, DependencyDefinition, ExecutionContext, InputDefinition, OutputDefinition, PipelineContextDefinition, PipelineDefinition, + Result, SolidDefinition, check, config, execute_pipeline, ) +from dagster.core.execution import execute_single_solid from dagster.core.errors import DagsterUserCodeExecutionError @@ -109,3 +114,36 @@ def transform_fn(_context, args): assert result_list[1].success assert not result_list[2].success assert isinstance(result_list[2].dagster_user_exception, DagsterUserCodeExecutionError) + + +def test_do_not_yield_result(): + solid_inst = SolidDefinition( + name='do_not_yield_result', + inputs=[], + outputs=[OutputDefinition()], + transform_fn=lambda *_args, **_kwargs: Result('foo') + ) + + with pytest.raises( + DagsterInvariantViolationError, + message='Tranform for solid do_not_yield_result return a Result', + ): + execute_single_solid(ExecutionContext(), solid_inst, config.Environment()) + + +def test_yield_non_result(): + def _tn(*_args, **_kwargs): + yield 'foo' + + solid_inst = SolidDefinition( + name='yield_wrong_thing', + inputs=[], + outputs=[OutputDefinition()], + transform_fn=_tn, + ) + + with pytest.raises( + DagsterInvariantViolationError, + message="Tranform for solid yield_wrong_thing yielded 'foo'", + ): + execute_single_solid(ExecutionContext(), solid_inst, config.Environment()) From ec78a81bbc351dd711f73b435e1ff7d5211d63ca Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Sat, 25 Aug 2018 17:10:06 -0700 Subject: [PATCH 026/103] Various, sundry cleanup in compute nodes Mainly eliminating SingleSyncOutputComputeNode and then refactoring the core compute node building path to be decomposed a bit more --- .../dagster/dagster/core/compute_nodes.py | 174 +++++++++--------- 1 file changed, 83 insertions(+), 91 deletions(-) diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index 7b86281a3037a..af94c03815213 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -20,7 +20,6 @@ from .argument_handling import validate_args from .definitions import ( - DependencyStructure, ExpectationDefinition, InputDefinition, OutputDefinition, @@ -41,6 +40,9 @@ from .types import DagsterType +# TODO: remove circular +import dagster.core.execution + class ComputeNodeOutputHandle(namedtuple('_ComputeNodeOutputHandle', 'compute_node output_name')): def __new__(cls, compute_node, output_name): return super(ComputeNodeOutputHandle, cls).__new__( @@ -88,7 +90,7 @@ def __new__(cls, dagster_user_exception): class ComputeNodeResult( namedtuple( '_ComputeNodeResult', - 'success compute_node tag success_data failure_data ', + 'success compute_node tag success_data failure_data', ) ): @staticmethod @@ -355,22 +357,10 @@ def execute_compute_nodes(context, compute_nodes): for result in compute_node.execute(context, input_values): check.invariant(isinstance(result, ComputeNodeResult)) yield result - output_handle = create_cn_output_handle(compute_node, result.success_data.output_name) + output_handle = ComputeNodeOutputHandle(compute_node, result.success_data.output_name) intermediate_results[output_handle] = result -def _yieldify(sync_compute_fn): - def _wrap(context, compute_node, inputs): - yield sync_compute_fn(context, compute_node, inputs) - - return _wrap - - -class SingleSyncOutputComputeNode(ComputeNode): - def __init__(self, *, sync_compute_fn, **kwargs): - super().__init__(compute_fn=_yieldify(sync_compute_fn), **kwargs) - - def print_graph(graph, printer=print): check.inst_param(graph, 'graph', ComputeNodeGraph) printer = IndentingPrinter(printer=printer) @@ -414,7 +404,7 @@ def create_expectation_cn( value_type = inout_def.dagster_type - return SingleSyncOutputComputeNode( + return ComputeNode( friendly_name=friendly_name, node_inputs=[ ComputeNodeInput( @@ -427,7 +417,7 @@ def create_expectation_cn( ComputeNodeOutput(name=EXPECTATION_VALUE_OUTPUT, dagster_type=value_type), ], arg_dict={}, - sync_compute_fn=_create_expectation_lambda( + compute_fn=_create_expectation_lambda( solid, expectation_def, EXPECTATION_VALUE_OUTPUT, @@ -437,8 +427,8 @@ def create_expectation_cn( ) -ExpectationsComputeNodeGraph = namedtuple( - 'ExpectationsComputeNodeGraph', +ComputeNodeSubgraph = namedtuple( + 'ComputeNodeSubgraph', 'nodes terminal_cn_output_handle', ) @@ -466,35 +456,11 @@ def create_expectations_cn_graph(solid, inout_def, prev_node_output_handle, tag) join_cn = _create_join_node(solid, input_expect_nodes, EXPECTATION_VALUE_OUTPUT) output_name = join_cn.node_outputs[0].name - return ExpectationsComputeNodeGraph( + return ComputeNodeSubgraph( compute_nodes + [join_cn], - create_cn_output_handle(join_cn, output_name), - ) - - -def _prev_node_handle(dep_structure, solid, input_def, compute_node_output_map): - check.inst_param(dep_structure, 'dep_structure', DependencyStructure) - check.inst_param(solid, 'solid', SolidDefinition) - check.inst_param(input_def, 'input_def', InputDefinition) - check.inst_param(compute_node_output_map, 'compute_node_output_map', ComputeNodeOutputMap) - - input_handle = solid.input_handle(input_def.name) - - check.invariant( - dep_structure.has_dep(input_handle), - f'{input_handle} not found in dependency structure', + ComputeNodeOutputHandle(join_cn, output_name), ) - solid_output_handle = dep_structure.get_dep(input_handle) - return compute_node_output_map[solid_output_handle] - - -def create_cn_output_handle(compute_node, cn_output_name): - check.inst_param(compute_node, 'compute_node', ComputeNode) - check.str_param(cn_output_name, 'cn_output_name') - return ComputeNodeOutputHandle(compute_node, cn_output_name) - - class ComputeNodeOutputMap(dict): def __getitem__(self, key): check.inst_param(key, 'key', SolidOutputHandle) @@ -507,7 +473,6 @@ def __setitem__(self, key, val): def create_compute_node_graph_from_env(execution_graph, env): - import dagster.core.execution check.inst_param(execution_graph, 'execution_graph', ExecutionGraph) check.inst_param(env, 'env', dagster.core.execution.DagsterEnv) @@ -521,34 +486,25 @@ def create_compute_node_graph_from_env(execution_graph, env): cn_inputs = [] for input_def in topo_solid.input_defs: - prev_cn_output_handle = _prev_node_handle( - dependency_structure, - topo_solid, - input_def, - cn_output_node_map, - ) + input_handle = topo_solid.input_handle(input_def.name) - check.inst(prev_cn_output_handle, ComputeNodeOutputHandle) - - # jam in input expectations here - - if env.evaluate_expectations and input_def.expectations: - expectations_graph = create_expectations_cn_graph( - topo_solid, - input_def, - prev_cn_output_handle, - tag=ComputeNodeTag.INPUT_EXPECTATION, - ) - compute_nodes = compute_nodes + expectations_graph.nodes + check.invariant( + dependency_structure.has_dep(input_handle), + f'{input_handle} not found in dependency structure', + ) - check.inst(expectations_graph.terminal_cn_output_handle, ComputeNodeOutputHandle) + solid_output_handle = dependency_structure.get_dep(input_handle) + prev_cn_output_handle = cn_output_node_map[solid_output_handle] - cn_output_handle = expectations_graph.terminal_cn_output_handle - else: - cn_output_handle = prev_cn_output_handle + subgraph = create_subgraph_for_input(env, topo_solid, prev_cn_output_handle, input_def) + compute_nodes.extend(subgraph.nodes) cn_inputs.append( - ComputeNodeInput(input_def.name, input_def.dagster_type, cn_output_handle) + ComputeNodeInput( + input_def.name, + input_def.dagster_type, + subgraph.terminal_cn_output_handle, + ) ) validated_config_args = validate_args( @@ -563,25 +519,19 @@ def create_compute_node_graph_from_env(execution_graph, env): validated_config_args, ) + compute_nodes.append(solid_transform_cn) + for output_def in topo_solid.output_defs: + subgraph = create_subgraph_for_output(env, topo_solid, solid_transform_cn, output_def) + compute_nodes.extend(subgraph.nodes) + output_handle = topo_solid.output_handle(output_def.name) - if env.evaluate_expectations and output_def.expectations: - expectations_graph = create_expectations_cn_graph( - topo_solid, - output_def, - create_cn_output_handle(solid_transform_cn, output_def.name), - tag=ComputeNodeTag.OUTPUT_EXPECTATION - ) - compute_nodes = compute_nodes + expectations_graph.nodes - cn_output_node_map[output_handle] = expectations_graph.terminal_cn_output_handle - else: - cn_output_node_map[output_handle] = create_cn_output_handle( - solid_transform_cn, - output_def.name, - ) + cn_output_node_map[output_handle] = subgraph.terminal_cn_output_handle - compute_nodes.append(solid_transform_cn) + return _create_compute_node_graph(compute_nodes) + +def _create_compute_node_graph(compute_nodes): cn_dict = {} for cn in compute_nodes: cn_dict[cn.guid] = cn @@ -595,6 +545,48 @@ def create_compute_node_graph_from_env(execution_graph, env): return ComputeNodeGraph(cn_dict, deps) +def create_subgraph_for_input(env, solid, prev_cn_output_handle, input_def): + check.inst_param(env, 'env', dagster.core.execution.DagsterEnv) + check.inst_param(solid, 'solid', SolidDefinition) + check.inst_param(prev_cn_output_handle, 'prev_cn_output_handle', ComputeNodeOutputHandle) + check.inst_param(input_def, 'input_def', InputDefinition) + + if env.evaluate_expectations and input_def.expectations: + return create_expectations_cn_graph( + solid, + input_def, + prev_cn_output_handle, + tag=ComputeNodeTag.INPUT_EXPECTATION, + ) + else: + return ComputeNodeSubgraph( + nodes=[], + terminal_cn_output_handle=prev_cn_output_handle, + ) + + +def create_subgraph_for_output(env, solid, solid_transform_cn, output_def): + check.inst_param(env, 'env', dagster.core.execution.DagsterEnv) + check.inst_param(solid, 'solid', SolidDefinition) + check.inst_param(solid_transform_cn, 'solid_transform_cn', ComputeNode) + check.inst_param(output_def, 'output_def', OutputDefinition) + + if env.evaluate_expectations and output_def.expectations: + return create_expectations_cn_graph( + solid, + output_def, + ComputeNodeOutputHandle(solid_transform_cn, output_def.name), + tag=ComputeNodeTag.OUTPUT_EXPECTATION + ) + else: + return ComputeNodeSubgraph( + nodes=[], + terminal_cn_output_handle=ComputeNodeOutputHandle( + solid_transform_cn, + output_def.name, + ), + ) + def _create_join_node(solid, prev_nodes, prev_output_name): check.inst_param(solid, 'solid', SolidDefinition) @@ -612,18 +604,18 @@ def _create_join_node(solid, prev_nodes, prev_output_name): else: check.invariant(seen_dagster_type == prev_node_output.dagster_type) - output_handle = create_cn_output_handle(prev_node, prev_output_name) + output_handle = ComputeNodeOutputHandle(prev_node, prev_output_name) node_inputs.append( ComputeNodeInput(prev_node.guid, prev_node_output.dagster_type, output_handle) ) - return SingleSyncOutputComputeNode( + return ComputeNode( friendly_name='join', node_inputs=node_inputs, node_outputs=[ComputeNodeOutput(JOIN_OUTPUT, seen_dagster_type)], arg_dict={}, - sync_compute_fn=_create_join_lambda, + compute_fn=_create_join_lambda, tag=ComputeNodeTag.JOIN, solid=solid, ) @@ -633,7 +625,7 @@ def _create_join_node(solid, prev_nodes, prev_output_name): def _create_join_lambda(_context, _compute_node, inputs): - return Result(output_name=JOIN_OUTPUT, value=list(inputs.values())[0]) + yield Result(output_name=JOIN_OUTPUT, value=list(inputs.values())[0]) def _create_expectation_lambda(solid, expectation_def, output_name): @@ -648,9 +640,9 @@ def _do_expectation(context, _compute_node, inputs): inputs[EXPECTATION_INPUT], ) if expt_result.success: - return Result(output_name=output_name, value=inputs[EXPECTATION_INPUT]) - - raise DagsterExpectationFailedError(None) # for now + yield Result(output_name=output_name, value=inputs[EXPECTATION_INPUT]) + else: + raise DagsterExpectationFailedError(None) # for now return _do_expectation From 1bb4b9d83cd1f82d0efbb6b350106b5558dc4c9c Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Thu, 23 Aug 2018 18:03:51 -0700 Subject: [PATCH 027/103] Handle optional outputs With this you can optionally emit an output. This allows for branching behavior in dags. --- .../dagster/dagster/core/compute_nodes.py | 18 +++--- .../core/core_tests/test_multiple_outputs.py | 58 ++++++++++++++++++- 2 files changed, 66 insertions(+), 10 deletions(-) diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index af94c03815213..fb4a6df9973ee 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -332,25 +332,25 @@ def output_named(self, name): check.failed(f'output {name} not found') +def _all_inputs_covered(cn, results): + for node_input in cn.node_inputs: + if node_input.prev_output_handle not in results: + return False + return True def execute_compute_nodes(context, compute_nodes): check.inst_param(context, 'context', ExecutionContext) check.list_param(compute_nodes, 'compute_nodes', of_type=ComputeNode) + intermediate_results = {} for compute_node in compute_nodes: + if not _all_inputs_covered(compute_node, intermediate_results): + continue + input_values = {} for node_input in compute_node.node_inputs: prev_output_handle = node_input.prev_output_handle - if prev_output_handle not in intermediate_results: - - target_cn = prev_output_handle.compute_node - - check.failed( - f'Could not find handle {prev_output_handle} in results. ' + \ - f'current node: {compute_node.friendly_name}\n' + \ - f'target node: {target_cn.friendly_name}' - ) input_value = intermediate_results[prev_output_handle].success_data.value input_values[node_input.name] = input_value diff --git a/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py b/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py index 81029c5248a25..8f885b7b32006 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py +++ b/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py @@ -2,8 +2,10 @@ from dagster import ( DagsterInvariantViolationError, + DependencyDefinition, ExpectationDefinition, ExpectationResult, + InputDefinition, OutputDefinition, PipelineDefinition, Result, @@ -125,4 +127,58 @@ def _t_fn(_context, _inputs, _config_dict): pipeline = PipelineDefinition(solids=[solid]) with pytest.raises(DagsterInvariantViolationError): - execute_pipeline(pipeline, config.Environment()) \ No newline at end of file + execute_pipeline(pipeline, config.Environment()) + + +def test_multiple_outputs_only_emit_one(): + def _t_fn(_context, _inputs, _config_dict): + yield Result(output_name='output_one', value='foo') + + solid = SolidDefinition( + name='multiple_outputs', + inputs=[], + outputs=[ + OutputDefinition(name='output_one'), + OutputDefinition(name='output_two'), + ], + transform_fn=_t_fn, + ) + + called = {} + + def _transform_fn_one(*_args, **_kwargs): + called['one'] = True + + downstream_one = SolidDefinition( + name='downstream_one', + inputs=[InputDefinition('some_input')], + outputs=[], + transform_fn=_transform_fn_one, + ) + + def _transform_fn_two(*_args, **_kwargs): + raise Exception('do not call me') + + downstream_two = SolidDefinition( + name='downstream_two', + inputs=[InputDefinition('some_input')], + outputs=[], + transform_fn=_transform_fn_two, + ) + + pipeline = PipelineDefinition( + solids=[solid, downstream_one, downstream_two], + dependencies={ + 'downstream_one': { + 'some_input': DependencyDefinition(solid.name, output='output_one'), + }, + 'downstream_two': { + 'some_input': DependencyDefinition(solid.name, output='output_two'), + }, + }, + ) + + result = execute_pipeline(pipeline, config.Environment()) + assert result.success + + assert called['one'] From 450b802600927ee6be7da1502eb4e90b807973a8 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Mon, 27 Aug 2018 07:56:14 -0700 Subject: [PATCH 028/103] Use defaultdict in depended_by_of_solid (#109) Let's start using defaultdict whenever we have data structure like this --- python_modules/dagster/dagster/core/definitions.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index 11c5913db64bc..a038264d612be 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -167,11 +167,9 @@ def __gen_deps_of_solid(self, solid_name): def depended_by_of_solid(self, solid_name): check.str_param(solid_name, 'solid_name') - result = {} + result = defaultdict(list) for input_handle, output_handle in self._handle_dict.items(): if output_handle.solid.name == solid_name: - if output_handle not in result: - result[output_handle] = [] result[output_handle].extend(input_handle) return result From 61a28724481b2b9cac8bf590545fd34235f25908 Mon Sep 17 00:00:00 2001 From: Mikhail Novikov Date: Tue, 28 Aug 2018 12:58:00 +0200 Subject: [PATCH 029/103] Add better pan and zoom (#113) --- .../dagit/webapp/src/graph/PipelineGraph.tsx | 51 +------------------ .../webapp/src/graph/getFullSolidLayout.ts | 4 +- 2 files changed, 4 insertions(+), 51 deletions(-) diff --git a/python_modules/dagit/dagit/webapp/src/graph/PipelineGraph.tsx b/python_modules/dagit/dagit/webapp/src/graph/PipelineGraph.tsx index 2b81268cd3aee..d690e2b0a3577 100644 --- a/python_modules/dagit/dagit/webapp/src/graph/PipelineGraph.tsx +++ b/python_modules/dagit/dagit/webapp/src/graph/PipelineGraph.tsx @@ -17,14 +17,9 @@ interface IPipelineGraphProps { onClickSolid?: (solidName: string) => void; } -interface IPipelineGraphState { - graphWidth: number | null; - graphHeight: number | null; -} - export default class PipelineGraph extends React.Component< IPipelineGraphProps, - IPipelineGraphState + {} > { static fragments = { PipelineGraphFragment: gql` @@ -39,36 +34,6 @@ export default class PipelineGraph extends React.Component< ` }; - state = { - graphWidth: null, - graphHeight: null - }; - - graphWrapper: React.RefObject = React.createRef(); - - componentDidMount() { - if (this.graphWrapper.current) { - this.setState({ - graphWidth: this.graphWrapper.current.clientWidth, - graphHeight: this.graphWrapper.current.clientHeight - }); - } - } - - componentDidUpdate() { - if (this.graphWrapper.current) { - if ( - this.state.graphWidth !== this.graphWrapper.current.clientWidth || - this.state.graphHeight !== this.graphWrapper.current.clientHeight - ) { - this.setState({ - graphWidth: this.graphWrapper.current.clientWidth, - graphHeight: this.graphWrapper.current.clientHeight - }); - } - } - } - renderSolids(layout: IFullPipelineLayout) { return this.props.pipeline.solids.map(solid => { const solidLayout = layout.solids[solid.name]; @@ -133,20 +98,8 @@ export default class PipelineGraph extends React.Component< render() { const layout = getDagrePipelineLayout(this.props.pipeline); - let minScale; - const { graphWidth, graphHeight } = this.state; - if (graphWidth !== null && graphHeight !== null) { - if (graphWidth > graphHeight) { - minScale = graphWidth / (layout.width + 200); - } else { - minScale = graphHeight / (layout.height - 300); - } - } else { - minScale = 0.1; - } - return ( - + Date: Wed, 29 Aug 2018 13:44:33 -0700 Subject: [PATCH 030/103] readd logging improvements --- .../dagster/dagster/core/execution_context.py | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/python_modules/dagster/dagster/core/execution_context.py b/python_modules/dagster/dagster/core/execution_context.py index 8c4c5e189df7a..ebae13302bf53 100644 --- a/python_modules/dagster/dagster/core/execution_context.py +++ b/python_modules/dagster/dagster/core/execution_context.py @@ -1,6 +1,10 @@ import copy -from collections import OrderedDict, namedtuple +from collections import ( + OrderedDict, + namedtuple, +) from contextlib import contextmanager +import uuid from dagster import check from dagster.utils.logging import CompositeLogger @@ -50,6 +54,12 @@ def _log(self, method, msg, **kwargs): check.str_param(method, 'method') check.str_param(msg, 'msg') + check.invariant('extra' not in kwargs, 'do not allow until explicit support is handled') + check.invariant('exc_info' not in kwargs, 'do not allow until explicit support is handled') + + check.invariant('log_message' not in kwargs, 'log_message_id reserved value') + check.invariant('log_message_id' not in kwargs, 'log_message_id reserved value') + full_message = 'message="{message}" {kv_message}'.format( message=msg, kv_message=self._kv_message(kwargs) ) @@ -57,6 +67,9 @@ def _log(self, method, msg, **kwargs): log_props = copy.copy(self._context_dict) log_props['log_message'] = msg + log_props['log_message'] = msg + log_props['log_message_id'] = str(uuid.uuid4()) + getattr(self._logger, method)(full_message, extra={**log_props, **kwargs}) def debug(self, msg, **kwargs): From d3ba9c35bab3d8509cb04bb3706232c995eebd37 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Wed, 29 Aug 2018 13:47:59 -0700 Subject: [PATCH 031/103] delete file --- .../dagit/dagit/webapp/src/Argumented.tsx | 104 ------------------ 1 file changed, 104 deletions(-) delete mode 100644 python_modules/dagit/dagit/webapp/src/Argumented.tsx diff --git a/python_modules/dagit/dagit/webapp/src/Argumented.tsx b/python_modules/dagit/dagit/webapp/src/Argumented.tsx deleted file mode 100644 index 96f14e818b245..0000000000000 --- a/python_modules/dagit/dagit/webapp/src/Argumented.tsx +++ /dev/null @@ -1,104 +0,0 @@ -// import * as React from "react"; -// import gql from "graphql-tag"; -// import styled from "styled-components"; -// import { H5, H6, Text, UL, Code, Collapse } from "@blueprintjs/core"; -// import SpacedCard from "./SpacedCard"; -// import TypeWithTooltip from "./TypeWithTooltip"; -// import Description from "./Description"; -// import { SourceFragment } from "./types/SourceFragment"; -// import { MaterializationFragment } from "./types/MaterializationFragment"; -// import { PipelineContextFragment } from "./types/PipelineContextFragment"; - -// interface IArgumentedProps { -// // XXX(freiksenet): Fix -// item: SourceFragment & MaterializationFragment & PipelineContextFragment; -// renderCard?: (props: any) => React.ReactNode; -// } - -// export default class Argumented extends React.Component { -// static fragments = { -// SourceFragment: gql` -// fragment SourceFragment on Source { -// name: sourceType -// description -// arguments { -// name -// description -// type { -// ...TypeFragment -// } -// isOptional -// } -// } - -// ${TypeWithTooltip.fragments.TypeFragment} -// `, -// MaterializationFragment: gql` -// fragment MaterializationFragment on Materialization { -// name -// description -// arguments { -// name -// description -// type { -// ...TypeFragment -// } -// isOptional -// } -// } - -// ${TypeWithTooltip.fragments.TypeFragment} -// `, -// PipelineContextFragment: gql` -// fragment PipelineContextFragment on PipelineContext { -// name -// description -// arguments { -// name -// description -// type { -// ...TypeFragment -// } -// isOptional -// } -// } - -// ${TypeWithTooltip.fragments.TypeFragment} -// ` -// }; - -// public render() { -// const renderCard = -// this.props.renderCard || ((props: any) => ); -// return renderCard({ -// elevation: 2, -// children: ( -// <> -//
-// {this.props.item.name} -//
-// -// -// -//
Arguments
-//
    -// {this.props.item.arguments.map((argument: any, i: number) => ( -//
  • -// {argument.name} {argument.isOptional ? "(optional)" : null}{" "} -// -// -// -// -//
  • -// ))} -//
-// -// ) -// }); -// } -// } - -// const DescriptionWrapper = styled.div` -// max-width: 400px; -// margin-bottom: 10px; -// `; From 3f77b0b5026963b886415603c4c6d111819395c9 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Wed, 29 Aug 2018 19:52:15 -0700 Subject: [PATCH 032/103] Fix some dangling lint errors --- python_modules/dagit/dagit/cli.py | 3 ++- python_modules/dagster/dagster/config.py | 2 +- .../dagster/dagster/core/compute_nodes.py | 26 +++++++++++++------ .../test_pandas_hello_world.py | 1 - .../dagster/dagster/utils/logging.py | 12 +++++++-- python_modules/dagster/docs/conf.py | 2 +- python_modules/dagster/setup.py | 2 +- 7 files changed, 33 insertions(+), 15 deletions(-) diff --git a/python_modules/dagit/dagit/cli.py b/python_modules/dagit/dagit/cli.py index a0614c6b48a98..ca4980f5799d9 100644 --- a/python_modules/dagit/dagit/cli.py +++ b/python_modules/dagit/dagit/cli.py @@ -1,6 +1,7 @@ -import click import os import sys + +import click from waitress import serve from watchdog.observers import Observer from watchdog.events import FileSystemEventHandler diff --git a/python_modules/dagster/dagster/config.py b/python_modules/dagster/dagster/config.py index 527c7dc87b818..3cb13dd2afc60 100644 --- a/python_modules/dagster/dagster/config.py +++ b/python_modules/dagster/dagster/config.py @@ -1,4 +1,4 @@ -from collections import (namedtuple, defaultdict) +from collections import namedtuple from dagster import check diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index fb4a6df9973ee..b33e1af62e99f 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -40,9 +40,6 @@ from .types import DagsterType -# TODO: remove circular -import dagster.core.execution - class ComputeNodeOutputHandle(namedtuple('_ComputeNodeOutputHandle', 'compute_node output_name')): def __new__(cls, compute_node, output_name): return super(ComputeNodeOutputHandle, cls).__new__( @@ -54,7 +51,10 @@ def __new__(cls, compute_node, output_name): # Make this hashable so it be a key in a dictionary def __str__(self): - return f'ComputeNodeOutputHandle(guid="{self.compute_node.guid}", output_name="{self.output_name}")' + return ( + f'ComputeNodeOutputHandle' + f'(guid="{self.compute_node.guid}", output_name="{self.output_name}")' + ) def __hash__(self): return hash(self.compute_node.guid + self.output_name) @@ -297,10 +297,11 @@ def execute(self, context, inputs): for result in results: if not self.has_node(result.output_name): + output_names = list([output_def.name for output_def in self.solid.output_defs]) raise DagsterInvariantViolationError( f'''Core transform for {self.solid.name} returned an output {result.output_name} that does not exist. The available - outputs are {list([output_def.name for output_def in self.solid.output_defs])}''' + outputs are {output_names}''' ) if result.output_name in seen_outputs: @@ -461,6 +462,7 @@ def create_expectations_cn_graph(solid, inout_def, prev_node_output_handle, tag) ComputeNodeOutputHandle(join_cn, output_name), ) + class ComputeNodeOutputMap(dict): def __getitem__(self, key): check.inst_param(key, 'key', SolidOutputHandle) @@ -472,9 +474,15 @@ def __setitem__(self, key, val): return dict.__setitem__(self, key, val) +def check_dagster_env(env): + # TODO: remove circular + import dagster.core.execution + check.inst_param(env, 'env', dagster.core.execution.DagsterEnv) + + def create_compute_node_graph_from_env(execution_graph, env): check.inst_param(execution_graph, 'execution_graph', ExecutionGraph) - check.inst_param(env, 'env', dagster.core.execution.DagsterEnv) + check_dagster_env(env) dependency_structure = execution_graph.dependency_structure @@ -531,6 +539,7 @@ def create_compute_node_graph_from_env(execution_graph, env): return _create_compute_node_graph(compute_nodes) + def _create_compute_node_graph(compute_nodes): cn_dict = {} for cn in compute_nodes: @@ -545,8 +554,9 @@ def _create_compute_node_graph(compute_nodes): return ComputeNodeGraph(cn_dict, deps) + def create_subgraph_for_input(env, solid, prev_cn_output_handle, input_def): - check.inst_param(env, 'env', dagster.core.execution.DagsterEnv) + check_dagster_env(env) check.inst_param(solid, 'solid', SolidDefinition) check.inst_param(prev_cn_output_handle, 'prev_cn_output_handle', ComputeNodeOutputHandle) check.inst_param(input_def, 'input_def', InputDefinition) @@ -566,7 +576,7 @@ def create_subgraph_for_input(env, solid, prev_cn_output_handle, input_def): def create_subgraph_for_output(env, solid, solid_transform_cn, output_def): - check.inst_param(env, 'env', dagster.core.execution.DagsterEnv) + check_dagster_env(env) check.inst_param(solid, 'solid', SolidDefinition) check.inst_param(solid_transform_cn, 'solid_transform_cn', ComputeNode) check.inst_param(output_def, 'output_def', OutputDefinition) diff --git a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/pandas_hello_world/test_pandas_hello_world.py b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/pandas_hello_world/test_pandas_hello_world.py index c32f09c97bca9..52441a1e4fc2c 100644 --- a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/pandas_hello_world/test_pandas_hello_world.py +++ b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/pandas_hello_world/test_pandas_hello_world.py @@ -6,7 +6,6 @@ from dagster.cli.pipeline import ( do_execute_command, print_pipeline, - print_solids, ) from dagster.dagster_examples.pandas_hello_world.pipeline import define_success_pipeline diff --git a/python_modules/dagster/dagster/utils/logging.py b/python_modules/dagster/dagster/utils/logging.py index c8868c8c33bad..b6e29d4dbd55c 100644 --- a/python_modules/dagster/dagster/utils/logging.py +++ b/python_modules/dagster/dagster/utils/logging.py @@ -54,7 +54,11 @@ def emit(self, record): def define_json_file_logger(name, json_path, level): check.str_param(name, 'name') check.str_param(json_path, 'json_path') - check.param_invariant(level in VALID_LEVELS, 'level', f'Must be valid python logging level. Got {level}') + check.param_invariant( + level in VALID_LEVELS, + 'level', + f'Must be valid python logging level. Got {level}', + ) klass = logging.getLoggerClass() logger = klass(name, level=level) @@ -66,7 +70,11 @@ def define_json_file_logger(name, json_path, level): def define_colored_console_logger(name, level=INFO): check.str_param(name, 'name') - check.param_invariant(level in VALID_LEVELS, 'level', f'Must be valid python logging level. Got {level}') + check.param_invariant( + level in VALID_LEVELS, + 'level', + f'Must be valid python logging level. Got {level}', + ) klass = logging.getLoggerClass() logger = klass(name, level=level) diff --git a/python_modules/dagster/docs/conf.py b/python_modules/dagster/docs/conf.py index 48001e191473f..25e2157f109ed 100644 --- a/python_modules/dagster/docs/conf.py +++ b/python_modules/dagster/docs/conf.py @@ -21,7 +21,7 @@ from recommonmark.parser import CommonMarkParser project = 'Dagster' -copyright = '2018, Nicholas Schrock' +copyright = '2018, Nicholas Schrock' # pylint: disable=W0622 author = 'Nicholas Schrock' # The short X.Y version diff --git a/python_modules/dagster/setup.py b/python_modules/dagster/setup.py index 991b1bcc98c0b..b4165207f40dd 100644 --- a/python_modules/dagster/setup.py +++ b/python_modules/dagster/setup.py @@ -18,7 +18,7 @@ def long_description(): setup( name='dagster', - version='0.2.0', + version='0.2.0.dev1', author='Elementl', author_email='schrockn@elementl.com', license='Apache-2.0', From 1b801e1c65a1cf1edb57166a5499c8d85cd813fb Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Wed, 29 Aug 2018 20:07:58 -0700 Subject: [PATCH 033/103] Eliminating nearly all lint errors --- .../core/core_tests/test_compute_nodes.py | 7 +++-- .../core_tests/test_pipeline_execution.py | 12 ++++---- .../test_transform_only_pipeline.py | 1 - .../dagster/dagster/core/decorators.py | 2 +- python_modules/dagster/dagster/core/types.py | 2 -- .../dagster/dagster/core/utility_solids.py | 2 -- .../dagster/dagster/pandas_kernel/__init__.py | 6 ++-- .../pandas_kernel_tests/test_pandas_solids.py | 29 +++++++------------ .../test_pandas_user_error.py | 3 -- .../test_isolated_sql_tests.py | 15 ++++++---- .../test_isolated_templated_sql_tests.py | 7 ----- .../test_mocked_context.py | 2 +- 12 files changed, 35 insertions(+), 53 deletions(-) diff --git a/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py b/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py index 67cc0c17fef4e..3433d292cfb14 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py +++ b/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py @@ -7,9 +7,12 @@ OutputDefinition, ) -from dagster.core.execution import (DagsterEnv, ExecutionGraph) +from dagster.core.execution import ( + DagsterEnv, + ExecutionGraph, +) + from dagster.core.compute_nodes import create_compute_node_graph_from_env -from dagster.core.graph import ExecutionGraph def silencing_default_context(): diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py index bdf754fb46b84..b99fc5c7c9b51 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py @@ -10,13 +10,18 @@ from dagster.core.definitions import DependencyStructure -from dagster.core.graph import (_create_adjacency_lists, ExecutionGraph) +from dagster.core.graph import ( + _create_adjacency_lists, + ExecutionGraph, +) + from dagster.core.execution import ( execute_pipeline_iterator, - ExecutionContext, ExecutionStepResult, ) +from dagster.core.utility_solids import define_stub_solid + # protected members # pylint: disable=W0212 @@ -156,9 +161,6 @@ def test_disconnected_graphs_adjaceny_lists(): assert backwards_edges == {'B': {'A'}, 'A': set(), 'D': {'C'}, 'C': set()} -from dagster.core.utility_solids import define_stub_solid - - def create_diamond_solids(): a_source = define_stub_solid('A_source', [input_set('A_input')]) node_a = create_root_solid('A') diff --git a/python_modules/dagster/dagster/core/core_tests/test_transform_only_pipeline.py b/python_modules/dagster/dagster/core/core_tests/test_transform_only_pipeline.py index fcd7277006698..12b246ba1d69b 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_transform_only_pipeline.py +++ b/python_modules/dagster/dagster/core/core_tests/test_transform_only_pipeline.py @@ -4,7 +4,6 @@ OutputDefinition, PipelineDefinition, SolidDefinition, - check, config, execute_pipeline, ) diff --git a/python_modules/dagster/dagster/core/decorators.py b/python_modules/dagster/dagster/core/decorators.py index 5acbbf56c6c34..5d58a2b4fe0ee 100644 --- a/python_modules/dagster/dagster/core/decorators.py +++ b/python_modules/dagster/dagster/core/decorators.py @@ -97,7 +97,7 @@ def _create_transform_wrapper(fn, inputs, outputs, include_context=False): input_names = [input.name for input in inputs] @wraps(fn) - def transform(context, args, config): + def transform(context, args, _config): kwargs = {} for input_name in input_names: kwargs[input_name] = args[input_name] diff --git a/python_modules/dagster/dagster/core/types.py b/python_modules/dagster/dagster/core/types.py index d790f3dd3f540..8097d2abfa57a 100644 --- a/python_modules/dagster/dagster/core/types.py +++ b/python_modules/dagster/dagster/core/types.py @@ -2,8 +2,6 @@ from dagster import check -import dagster - class DagsterType: def __init__(self, name, description=None): diff --git a/python_modules/dagster/dagster/core/utility_solids.py b/python_modules/dagster/dagster/core/utility_solids.py index 9c0279b1df9bf..de652b4fb178b 100644 --- a/python_modules/dagster/dagster/core/utility_solids.py +++ b/python_modules/dagster/dagster/core/utility_solids.py @@ -1,10 +1,8 @@ from dagster import ( - ArgumentDefinition, OutputDefinition, Result, SolidDefinition, check, - types, ) diff --git a/python_modules/dagster/dagster/pandas_kernel/__init__.py b/python_modules/dagster/dagster/pandas_kernel/__init__.py index d6117ca6cc87a..6922c8f9a1885 100644 --- a/python_modules/dagster/dagster/pandas_kernel/__init__.py +++ b/python_modules/dagster/dagster/pandas_kernel/__init__.py @@ -20,10 +20,8 @@ def _create_dataframe_type(): return types.PythonObjectType( name='PandasDataFrame', python_type=pd.DataFrame, - description= - '''Two-dimensional size-mutable, potentially heterogeneous tabular data structure with labeled axes (rows and columns). - See http://pandas.pydata.org/ - ''', + description='''Two-dimensional size-mutable, potentially heterogeneous +tabular data structure with labeled axes (rows and columns). See http://pandas.pydata.org/''', ) diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py index 92e57b518dd7c..09c248c5b0922 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py @@ -3,23 +3,29 @@ import pandas as pd from dagster import ( - DependencyDefinition, ExecutionContext, + DependencyDefinition, InputDefinition, OutputDefinition, PipelineDefinition, SolidDefinition, check, config, + solid, ) -from dagster.core.decorators import solid + from dagster.core.execution import ( - ExecutionGraph, execute_pipeline_iterator, execute_pipeline, ) + import dagster.pandas_kernel as dagster_pd -from dagster.utils.test import (get_temp_file_name, get_temp_file_names, script_relative_path) + +from dagster.utils.test import ( + get_temp_file_name, + get_temp_file_names, + script_relative_path, +) def _dataframe_solid(name, inputs, transform_fn): @@ -68,8 +74,6 @@ def transform(_context, args): num_csv['sum'] = num_csv['num1'] + num_csv['num2'] return num_csv - test_output = {} - single_solid = SolidDefinition.single_output_transform( name='sum_table', inputs=[csv_input], @@ -199,19 +203,6 @@ def sum_sq_table_renamed_input(sum_table_renamed): return sum_table_renamed -def create_mult_table(sum_table_solid): - def transform(_context, args): - sum_df = args['sum_table'] - sum_df['sum_squared'] = sum_df['sum'] * sum_df['sum'] - return sum_df - - return _dataframe_solid( - name='mult_table', - inputs=[InputDefinition('sum_table', dagster_pd.DataFrame)], - transform_fn=transform - ) - - def test_pandas_csv_to_csv_better_api(): output_df = execute_transform_in_temp_csv_files(create_sum_table()) assert output_df.to_dict('list') == {'num1': [1, 3], 'num2': [2, 4], 'sum': [3, 7]} diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_user_error.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_user_error.py index 02dc821855a66..d386235191ccf 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_user_error.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_user_error.py @@ -4,7 +4,6 @@ import pandas as pd -import dagster import dagster.pandas_kernel as dagster_pd from dagster import ( DependencyDefinition, @@ -17,8 +16,6 @@ solid, ) from dagster.core.errors import DagsterInvariantViolationError -from dagster.core.execution import execute_single_solid -from dagster.utils.test import script_relative_path from dagster.core.utility_solids import define_stub_solid diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_sql_tests.py b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_sql_tests.py index 78441b44d0110..b777314d0cc7e 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_sql_tests.py +++ b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_sql_tests.py @@ -1,12 +1,15 @@ -import dagster from dagster import ( DependencyDefinition, InputDefinition, + PipelineContextDefinition, + PipelineDefinition, config, + execute_pipeline, ) from dagster.core.execution import execute_single_solid from dagster.sqlalchemy_kernel.subquery_builder_experimental import ( - create_sql_statement_solid, sql_file_solid + create_sql_statement_solid, + sql_file_solid, ) from dagster.utils.test import script_relative_path @@ -14,11 +17,11 @@ def pipeline_test_def(solids, context, dependencies=None): - return dagster.PipelineDefinition( + return PipelineDefinition( solids=solids, context_definitions={ 'default': - dagster.PipelineContextDefinition( + PipelineContextDefinition( argument_def_dict={}, context_fn=lambda _pipeline, _args: context, ), @@ -69,7 +72,7 @@ def test_basic_pipeline(): }, ) - pipeline_result = dagster.execute_pipeline(pipeline, environment=config.Environment.empty()) + pipeline_result = execute_pipeline(pipeline, environment=config.Environment.empty()) assert pipeline_result.success @@ -107,7 +110,7 @@ def test_pipeline_from_files(): }, ) - pipeline_result = dagster.execute_pipeline(pipeline, environment=config.Environment.empty()) + pipeline_result = execute_pipeline(pipeline, environment=config.Environment.empty()) assert pipeline_result.success diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py index 442106719a564..0a431d5dc3273 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py +++ b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py @@ -1,15 +1,10 @@ from dagster import ( - ArgumentDefinition, DependencyDefinition, - InputDefinition, PipelineContextDefinition, PipelineDefinition, - OutputDefinition, - SolidDefinition, check, config, execute_pipeline, - types, ) from dagster.sqlalchemy_kernel.templated import ( @@ -263,8 +258,6 @@ def test_with_from_through_specifying_all_solids(): first_mult_table = 'first_mult_table' first_sum_mult_table = 'first_sum_mult_table' - all_solid_names = [solid.name for solid in pipeline.solids] - environment = config.Environment( solids={ 'sum_table': diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_mocked_context.py b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_mocked_context.py index 8fd0220248938..b2699287da2f4 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_mocked_context.py +++ b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_mocked_context.py @@ -18,4 +18,4 @@ def raw_connection(self): def test_mock(): sa_resource = dagster_sa.SqlAlchemyResource(engine=MockEngine(), mock_sql=True) context = dagster_sa.common.create_sql_alchemy_context_from_sa_resource(sa_resource) - dagster_sa.common.execute_sql_text_on_context(context, 'NOPE') \ No newline at end of file + dagster_sa.common.execute_sql_text_on_context(context, 'NOPE') From ed82b852ccd29c39904ecb96ebed9c73f3ebdc43 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Wed, 29 Aug 2018 21:01:55 -0700 Subject: [PATCH 034/103] Eliminate input and output callbacks Now that we have the type system, we no longer need the output and input callbacks to do typechecking. Eliminating. --- .../dagster/dagster/core/definitions.py | 18 ++---------------- 1 file changed, 2 insertions(+), 16 deletions(-) diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index a038264d612be..404cb220426d5 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -349,14 +349,9 @@ class InputDefinition: - depends_on: (Optional). This input depends on another solid in the context of a a pipeline. - - - input_callback: (Optional) Called on the source result. Gets execution context and result. - Can be used to validate the result, log stats etc. ''' - def __init__( - self, name, dagster_type=None, expectations=None, input_callback=None, description=None - ): + def __init__(self, name, dagster_type=None, expectations=None, description=None): self.name = check_valid_name(name) self.dagster_type = check.opt_inst_param( @@ -366,20 +361,12 @@ def __init__( self.expectations = check.opt_list_param( expectations, 'expectations', of_type=ExpectationDefinition ) - self.input_callback = check.opt_callable_param(input_callback, 'input_callback') self.description = check.opt_str_param(description, 'description') class OutputDefinition: # runtime type info - def __init__( - self, - name=None, - dagster_type=None, - expectations=None, - output_callback=None, - description=None - ): + def __init__(self, name=None, dagster_type=None, expectations=None, description=None): self.name = check.opt_str_param(name, 'name', DEFAULT_OUTPUT) self.dagster_type = check.opt_inst_param( @@ -389,7 +376,6 @@ def __init__( self.expectations = check.opt_list_param( expectations, 'expectations', of_type=ExpectationDefinition ) - self.output_callback = check.opt_callable_param(output_callback, 'output_callback') self.description = check.opt_str_param(description, 'description') From f24fb1aeddc7875568a4d051c883ba820fc1f15a Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Wed, 29 Aug 2018 21:42:42 -0700 Subject: [PATCH 035/103] Various and sundry cleanup tasks Delete unused things. Move some things out of core files and into modules that indicate that they are test only. Move script_relative_path to utils rather than test because it useful outside of tests. --- .../dagster_ge_tests/test_pandas_ge.py | 29 +++++++------- python_modules/dagster/dagster/config.py | 9 ----- .../core/core_tests/test_decorators.py | 15 +++---- .../core/core_tests/test_pipeline_errors.py | 2 +- .../dagster/dagster/core/definitions.py | 13 ++++-- .../dagster/dagster/core/execution.py | 37 +---------------- .../dagster/dagster/core/test_utils.py | 31 ++++++++++++++ .../dagster/dagster/core/validation.py | 0 .../test_pandas_hello_world.py | 2 +- .../test_sql_project_pipeline.py | 2 +- .../test_pandas_hello_world_library_slide.py | 3 +- ...est_pandas_hello_world_no_library_slide.py | 2 +- .../pandas_kernel_tests/test_pandas_solids.py | 3 +- .../test_isolated_sql_tests.py | 4 +- .../dagster/dagster/utils/__init__.py | 40 ++++++------------- python_modules/dagster/dagster/utils/test.py | 13 ------ .../utils/utils_tests/test_contextify.py | 18 --------- .../dagster/dagster_tests/test_config.py | 1 - 18 files changed, 84 insertions(+), 140 deletions(-) create mode 100644 python_modules/dagster/dagster/core/test_utils.py delete mode 100644 python_modules/dagster/dagster/core/validation.py delete mode 100644 python_modules/dagster/dagster/utils/utils_tests/test_contextify.py diff --git a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py index 412d354314cb6..6791aca8d707b 100644 --- a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py +++ b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py @@ -2,19 +2,21 @@ import pandas as pd -import dagster -import dagster.pandas_kernel as dagster_pd -import dagster_ge from dagster import ( - config, - solid, - execute_pipeline, + DependencyDefinition, InputDefinition, OutputDefinition, - DependencyDefinition, + PipelineDefinition, + config, + execute_pipeline, + solid, ) -from dagster.core.errors import (DagsterExecutionFailureReason, DagsterExpectationFailedError) -from dagster.utils.test import script_relative_path +from dagster.core.errors import DagsterExpectationFailedError +from dagster.core.utility_solids import define_stub_solid +import dagster.pandas_kernel as dagster_pd +from dagster.utils import script_relative_path + +import dagster_ge def _sum_solid_impl(num_df): @@ -71,12 +73,9 @@ def sum_solid_expectations_config(num_df): return _sum_solid_impl(num_df) -from dagster.core.utility_solids import define_stub_solid - - def test_single_node_passing_expectation(): in_df = pd.DataFrame.from_dict({'num1': [1, 3], 'num2': [2, 4]}) - pipeline = dagster.PipelineDefinition( + pipeline = PipelineDefinition( solids=[define_stub_solid('value', in_df), sum_solid], dependencies={'sum_solid': { 'num_df': DependencyDefinition('value') @@ -96,7 +95,7 @@ def test_single_node_passing_expectation(): def test_single_node_passing_json_config_expectations(): in_df = pd.DataFrame.from_dict({'num1': [1, 3], 'num2': [2, 4]}) - pipeline = dagster.PipelineDefinition( + pipeline = PipelineDefinition( solids=[define_stub_solid('value', in_df), sum_solid_expectations_config], dependencies={ sum_solid_expectations_config.name: { @@ -118,7 +117,7 @@ def test_single_node_passing_json_config_expectations(): def test_single_node_failing_expectation(): in_df = pd.DataFrame.from_dict({'num1': [1, 3], 'num2': [2, 4]}) - pipeline = dagster.PipelineDefinition( + pipeline = PipelineDefinition( solids=[define_stub_solid('value', in_df), sum_solid_fails_input_expectation], dependencies={ sum_solid_fails_input_expectation.name: { diff --git a/python_modules/dagster/dagster/config.py b/python_modules/dagster/dagster/config.py index 3cb13dd2afc60..08e4e3fa8ec4d 100644 --- a/python_modules/dagster/dagster/config.py +++ b/python_modules/dagster/dagster/config.py @@ -39,15 +39,6 @@ def empty(): return Environment() -class Source(namedtuple('SourceData', 'name args')): - def __new__(cls, name, args): - return super(Source, cls).__new__( - cls, - name=check.str_param(name, 'name'), - args=check.dict_param(args, 'args', key_type=str), - ) - - class Expectations(namedtuple('ExpectationsData', 'evaluate')): def __new__(cls, evaluate): return super(Expectations, cls).__new__( diff --git a/python_modules/dagster/dagster/core/core_tests/test_decorators.py b/python_modules/dagster/dagster/core/core_tests/test_decorators.py index f6c1f46591ff4..003c88cc4d480 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_decorators.py +++ b/python_modules/dagster/dagster/core/core_tests/test_decorators.py @@ -1,24 +1,21 @@ import pytest import dagster from dagster import ( + DagsterInvalidDefinitionError, DependencyDefinition, + ExecutionContext, InputDefinition, + MultipleResults, OutputDefinition, PipelineDefinition, - config, Result, + config, execute_pipeline, -) -from dagster.core.errors import DagsterInvalidDefinitionError -from dagster.core.decorators import ( solid, with_context, - MultipleResults, -) -from dagster.core.execution import ( - execute_single_solid, - ExecutionContext, ) + +from dagster.core.test_utils import execute_single_solid from dagster.core.utility_solids import define_stub_solid # This file tests a lot of parameter name stuff diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py index 8c3f1051b9e57..8f74978bfc528 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py @@ -15,7 +15,7 @@ execute_pipeline, ) -from dagster.core.execution import execute_single_solid +from dagster.core.test_utils import execute_single_solid from dagster.core.errors import DagsterUserCodeExecutionError diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index 404cb220426d5..0e3e52d84acc2 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -11,6 +11,7 @@ ) from .errors import DagsterInvalidDefinitionError +from .execution_context import ExecutionContext DEFAULT_OUTPUT = 'result' @@ -62,6 +63,14 @@ def __setitem__(self, _key, _value): class PipelineContextDefinition: + @staticmethod + def passthrough_context_definition(context): + check.inst_param(context, 'context', ExecutionContext) + context_definition = PipelineContextDefinition( + argument_def_dict={}, context_fn=lambda _pipeline, _args: context + ) + return {'default': context_definition} + def __init__(self, *, argument_def_dict, context_fn, description=None): self.argument_def_dict = ArgumentDefinitionDictionary(argument_def_dict) self.context_fn = check.callable_param(context_fn, 'context_fn') @@ -463,10 +472,6 @@ def __init__(self, *, name, inputs, transform_fn, outputs, config_def=None, desc self._input_dict = _build_named_dict(inputs) self._output_dict = _build_named_dict(outputs) - @property - def outputs(self): - return self.output_defs - @staticmethod def single_output_transform( name, inputs, transform_fn, output, config_def=None, description=None diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index f122908ffaf8e..8c1e8238abbab 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -25,12 +25,7 @@ config, ) - -from .definitions import ( - SolidDefinition, - PipelineDefinition, - PipelineContextDefinition, -) +from .definitions import PipelineDefinition from .errors import ( DagsterUserCodeExecutionError, @@ -120,36 +115,6 @@ def copy_result_dict(result_dict): return new_dict -def _create_passthrough_context_definition(context): - check.inst_param(context, 'context', ExecutionContext) - context_definition = PipelineContextDefinition( - argument_def_dict={}, context_fn=lambda _pipeline, _args: context - ) - return {'default': context_definition} - - -def execute_single_solid(context, solid, environment, throw_on_error=True): - check.inst_param(context, 'context', ExecutionContext) - check.inst_param(solid, 'solid', SolidDefinition) - check.inst_param(environment, 'environment', config.Environment) - check.bool_param(throw_on_error, 'throw_on_error') - - single_solid_environment = config.Environment( - expectations=environment.expectations, - context=environment.context, - ) - - pipeline_result = execute_pipeline( - PipelineDefinition( - solids=[solid], - context_definitions=_create_passthrough_context_definition(context), - ), - environment=single_solid_environment, - ) - - return pipeline_result - - def _do_throw_on_error(execution_result): check.inst_param(execution_result, 'execution_result', ExecutionStepResult) if execution_result.success: diff --git a/python_modules/dagster/dagster/core/test_utils.py b/python_modules/dagster/dagster/core/test_utils.py new file mode 100644 index 0000000000000..3d51ad6d1cad3 --- /dev/null +++ b/python_modules/dagster/dagster/core/test_utils.py @@ -0,0 +1,31 @@ +from dagster import ( + ExecutionContext, + PipelineDefinition, + PipelineContextDefinition, + SolidDefinition, + check, + config, + execute_pipeline, +) + + +def execute_single_solid(context, solid, environment, throw_on_error=True): + check.inst_param(context, 'context', ExecutionContext) + check.inst_param(solid, 'solid', SolidDefinition) + check.inst_param(environment, 'environment', config.Environment) + check.bool_param(throw_on_error, 'throw_on_error') + + single_solid_environment = config.Environment( + expectations=environment.expectations, + context=environment.context, + ) + + pipeline_result = execute_pipeline( + PipelineDefinition( + solids=[solid], + context_definitions=PipelineContextDefinition.passthrough_context_definition(context), + ), + environment=single_solid_environment, + ) + + return pipeline_result diff --git a/python_modules/dagster/dagster/core/validation.py b/python_modules/dagster/dagster/core/validation.py deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/pandas_hello_world/test_pandas_hello_world.py b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/pandas_hello_world/test_pandas_hello_world.py index 52441a1e4fc2c..3c845eec2dc3f 100644 --- a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/pandas_hello_world/test_pandas_hello_world.py +++ b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/pandas_hello_world/test_pandas_hello_world.py @@ -2,7 +2,7 @@ from dagster import config from dagster.core.execution import execute_pipeline -from dagster.utils.test import script_relative_path +from dagster.utils import script_relative_path from dagster.cli.pipeline import ( do_execute_command, print_pipeline, diff --git a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py index 43413bba1b7a9..2587edc135081 100644 --- a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py +++ b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py @@ -2,7 +2,7 @@ import dagster from dagster import (config, InputDefinition, DependencyDefinition) import dagster.sqlalchemy_kernel as dagster_sa -from dagster.utils.test import script_relative_path +from dagster.utils import script_relative_path from dagster.sqlalchemy_kernel.subquery_builder_experimental import sql_file_solid diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py index ebb67391c9e00..4205e7e9b4df2 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py @@ -11,7 +11,8 @@ ) from dagster.core.decorators import solid -from dagster.utils.test import (script_relative_path, get_temp_file_name) +from dagster.utils import script_relative_path +from dagster.utils.test import get_temp_file_name import dagster.pandas_kernel as dagster_pd diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py index e79ba5f037304..f2c4a236c230b 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py @@ -14,7 +14,7 @@ types, ) -from dagster.utils.test import script_relative_path +from dagster.utils import script_relative_path def define_read_csv_solid(name): diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py index 09c248c5b0922..44c3bd3f8b39d 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py @@ -21,10 +21,11 @@ import dagster.pandas_kernel as dagster_pd +from dagster.utils import script_relative_path + from dagster.utils.test import ( get_temp_file_name, get_temp_file_names, - script_relative_path, ) diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_sql_tests.py b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_sql_tests.py index b777314d0cc7e..94b8bccf6a70b 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_sql_tests.py +++ b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_sql_tests.py @@ -6,12 +6,12 @@ config, execute_pipeline, ) -from dagster.core.execution import execute_single_solid +from dagster.core.test_utils import execute_single_solid from dagster.sqlalchemy_kernel.subquery_builder_experimental import ( create_sql_statement_solid, sql_file_solid, ) -from dagster.utils.test import script_relative_path +from dagster.utils import script_relative_path from .math_test_db import in_mem_context diff --git a/python_modules/dagster/dagster/utils/__init__.py b/python_modules/dagster/dagster/utils/__init__.py index 3e1c858e5de11..95bd47a77e2e8 100644 --- a/python_modules/dagster/dagster/utils/__init__.py +++ b/python_modules/dagster/dagster/utils/__init__.py @@ -1,30 +1,16 @@ -from __future__ import (absolute_import, division, print_function, unicode_literals) -from builtins import * # pylint: disable=W0622,W0401 import inspect +import os from dagster import check - - -def has_context_argument(fn): - check.callable_param(fn, 'fn') - - argspec = inspect.getfullargspec(fn) - return 'context' in argspec[0] - - -def make_context_arg_optional(fn): - check.callable_param(fn, 'fn') - # make this idempotent during the transition - if fn.__name__ == '__wrapper_with_context': - return fn - check.invariant(fn.__name__ != '__wrapper_with_context') - - if not has_context_argument(fn): - - def __wrapper_with_context(*args, context, **kwargs): - check.not_none_param(context, 'context') - return fn(*args, **kwargs) - - return __wrapper_with_context - else: - return fn +def script_relative_path(file_path): + ''' + Useful for testing with local files. Use a path relative to where the + test resides and this function will return the absolute path + of that file. Otherwise it will be relative to script that + ran the test + ''' + # from http://bit.ly/2snyC6s + + check.str_param(file_path, 'file_path') + scriptdir = inspect.stack()[1][1] + return os.path.join(os.path.dirname(os.path.abspath(scriptdir)), file_path) diff --git a/python_modules/dagster/dagster/utils/test.py b/python_modules/dagster/dagster/utils/test.py index 59deb657cb27c..b2bf36301c0b9 100644 --- a/python_modules/dagster/dagster/utils/test.py +++ b/python_modules/dagster/dagster/utils/test.py @@ -1,4 +1,3 @@ -import inspect import itertools import os import tempfile @@ -7,18 +6,6 @@ from dagster import check -def script_relative_path(file_path): - ''' - Useful for testing with local files. Use a path relative to where the - test resides and this function will return the absolute path - of that file. Otherwise it will be relative to script that - ran the test - ''' - # from http://bit.ly/2snyC6s - - check.str_param(file_path, 'file_path') - scriptdir = inspect.stack()[1][1] - return os.path.join(os.path.dirname(os.path.abspath(scriptdir)), file_path) def _unlink_swallow_errors(path): diff --git a/python_modules/dagster/dagster/utils/utils_tests/test_contextify.py b/python_modules/dagster/dagster/utils/utils_tests/test_contextify.py deleted file mode 100644 index 2c50ca3393542..0000000000000 --- a/python_modules/dagster/dagster/utils/utils_tests/test_contextify.py +++ /dev/null @@ -1,18 +0,0 @@ -from dagster.utils import has_context_argument - - -def test_has_context_variable(): - # pylint: disable=W0613 - - def nope(_foo): - pass - - def yup(context, _bar): - pass - - assert not has_context_argument(nope) - assert has_context_argument(yup) - assert not has_context_argument(lambda: None) - assert not has_context_argument(lambda bar: None) - assert has_context_argument(lambda context: None) - assert has_context_argument(lambda bar, context: None) diff --git a/python_modules/dagster/dagster_tests/test_config.py b/python_modules/dagster/dagster_tests/test_config.py index 60929a58acd8c..c0fe57d596ecf 100644 --- a/python_modules/dagster/dagster_tests/test_config.py +++ b/python_modules/dagster/dagster_tests/test_config.py @@ -1,4 +1,3 @@ -import pytest import yaml from dagster import config From ce7a0225d16a19e98e08796d567ffe3c5e3327fe Mon Sep 17 00:00:00 2001 From: Mikhail Novikov Date: Wed, 29 Aug 2018 16:51:11 +0300 Subject: [PATCH 036/103] New execution result model * Moved execution results to the following hierarchy PipelineExecutionResult -> SolidExecutionResult -> ComputeNodeResult * Adjusted tests and usage for it --- .../dagster_ge_tests/test_pandas_ge.py | 4 +- python_modules/dagster/README.rst | 2 +- .../dagster/dagster/cli/pipeline.py | 14 +- .../dagster/dagster/core/compute_nodes.py | 21 +- .../core/core_tests/test_decorators.py | 31 +-- .../core/core_tests/test_multiple_outputs.py | 11 +- .../core/core_tests/test_naming_collisions.py | 12 +- .../core/core_tests/test_pipeline_errors.py | 1 + .../core_tests/test_pipeline_execution.py | 72 ++++--- python_modules/dagster/dagster/core/errors.py | 2 +- .../dagster/dagster/core/execution.py | 188 +++++++++++------- .../test_pandas_hello_world.py | 4 +- .../test_pandas_hello_world_library_slide.py | 6 +- ...est_pandas_hello_world_no_library_slide.py | 10 +- .../pandas_kernel_tests/test_pandas_solids.py | 37 ++-- .../test_basic_solid.py | 6 +- .../test_isolated_templated_sql_tests.py | 4 +- python_modules/dagster/tox.ini | 2 +- 18 files changed, 234 insertions(+), 193 deletions(-) diff --git a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py index 6791aca8d707b..a8b8ad82b0a22 100644 --- a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py +++ b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py @@ -86,7 +86,7 @@ def test_single_node_passing_expectation(): assert result.success assert len(result.result_list) == 2 assert result.result_list[1].success - assert result.result_list[1].transformed_value.to_dict('list') == { + assert result.result_list[1].transformed_value().to_dict('list') == { 'num1': [1, 3], 'num2': [2, 4], 'sum': [3, 7], @@ -108,7 +108,7 @@ def test_single_node_passing_json_config_expectations(): assert result.success assert len(result.result_list) == 2 assert result.result_list[1].success - assert result.result_list[1].transformed_value.to_dict('list') == { + assert result.result_list[1].transformed_value().to_dict('list') == { 'num1': [1, 3], 'num2': [2, 4], 'sum': [3, 7], diff --git a/python_modules/dagster/README.rst b/python_modules/dagster/README.rst index 781b12beb3308..e019f6e44d6c7 100644 --- a/python_modules/dagster/README.rst +++ b/python_modules/dagster/README.rst @@ -390,7 +390,7 @@ enough source data to create all the inputs necessary for the pipeline. environment ) - print(pipeline_result.result_named('sum').transformed_value) + print(pipeline_result.result_for_solid('sum').transformed_value) Execute pipeline does a purely in-memory transform, materializing nothing. This is useful in testing and CI/CD contexts. diff --git a/python_modules/dagster/dagster/cli/pipeline.py b/python_modules/dagster/dagster/cli/pipeline.py index 6a12986bc7696..01a0812b97b87 100644 --- a/python_modules/dagster/dagster/cli/pipeline.py +++ b/python_modules/dagster/dagster/cli/pipeline.py @@ -244,16 +244,18 @@ def process_results_for_console(pipeline_iter, printer): def print_metrics_to_console(results, printer): for result in results: context = result.context - metrics_of_solid = list(context.metrics_matching_context({'solid': result.name})) + metrics_of_solid = list(context.metrics_matching_context({'solid': result.solid.name})) - printer('Metrics for {name}'.format(name=result.name)) + printer('Metrics for {name}'.format(name=result.solid.name)) for input_def in result.solid.input_defs: metrics_for_input = list( - context.metrics_covering_context({ - 'solid': result.name, - 'input': input_def.name, - }) + context.metrics_covering_context( + { + 'solid': result.solid.name, + 'input': input_def.name, + } + ) ) if metrics_for_input: printer(' Input {input_name}'.format(input_name=input_def.name)) diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index b33e1af62e99f..c3f93c410c022 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -40,6 +40,7 @@ from .types import DagsterType + class ComputeNodeOutputHandle(namedtuple('_ComputeNodeOutputHandle', 'compute_node output_name')): def __new__(cls, compute_node, output_name): return super(ComputeNodeOutputHandle, cls).__new__( @@ -172,19 +173,21 @@ def _execute_core_transform(context, compute_node, values_dict, config_dict): if isinstance(gen, Result): raise DagsterInvariantViolationError( - ('Transform for solid {solid_name} returned a Result rather than ' + - 'yielding it. The transform_fn of the core SolidDefinition must yield ' + - 'its results').format( - solid_name=compute_node.solid.name, - ) + ( + 'Transform for solid {solid_name} returned a Result rather than ' + + 'yielding it. The transform_fn of the core SolidDefinition must yield ' + + 'its results' + ).format(solid_name=compute_node.solid.name, ) ) if gen is not None: for result in gen: if not isinstance(result, Result): raise DagsterInvariantViolationError( - ('Transform for solid {solid_name} yielded {result} rather an ' + - 'an instance of the Result class.').format( + ( + 'Transform for solid {solid_name} yielded {result} rather an ' + + 'an instance of the Result class.' + ).format( result=repr(result), solid_name=compute_node.solid.name, ) @@ -333,17 +336,18 @@ def output_named(self, name): check.failed(f'output {name} not found') + def _all_inputs_covered(cn, results): for node_input in cn.node_inputs: if node_input.prev_output_handle not in results: return False return True + def execute_compute_nodes(context, compute_nodes): check.inst_param(context, 'context', ExecutionContext) check.list_param(compute_nodes, 'compute_nodes', of_type=ComputeNode) - intermediate_results = {} for compute_node in compute_nodes: if not _all_inputs_covered(compute_node, intermediate_results): @@ -536,7 +540,6 @@ def create_compute_node_graph_from_env(execution_graph, env): output_handle = topo_solid.output_handle(output_def.name) cn_output_node_map[output_handle] = subgraph.terminal_cn_output_handle - return _create_compute_node_graph(compute_nodes) diff --git a/python_modules/dagster/dagster/core/core_tests/test_decorators.py b/python_modules/dagster/dagster/core/core_tests/test_decorators.py index 003c88cc4d480..29b5b98414ae5 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_decorators.py +++ b/python_modules/dagster/dagster/core/core_tests/test_decorators.py @@ -45,7 +45,7 @@ def hello_world(): assert result.success assert len(result.result_list) == 1 - assert result.result_list[0].transformed_value['foo'] == 'bar' + assert result.result_list[0].transformed_value()['foo'] == 'bar' def test_solid_one_output(): @@ -61,7 +61,7 @@ def hello_world(): assert result.success assert len(result.result_list) == 1 - assert result.result_list[0].transformed_value['foo'] == 'bar' + assert result.result_list[0].transformed_value()['foo'] == 'bar' def test_solid_yield(): @@ -77,7 +77,7 @@ def hello_world(): assert result.success assert len(result.result_list) == 1 - assert result.result_list[0].transformed_value['foo'] == 'bar' + assert result.result_list[0].transformed_value()['foo'] == 'bar' def test_solid_result_return(): @@ -93,7 +93,7 @@ def hello_world(): assert result.success assert len(result.result_list) == 1 - assert result.result_list[0].transformed_value['foo'] == 'bar' + assert result.result_list[0].transformed_value()['foo'] == 'bar' def test_solid_multiple_outputs(): @@ -114,9 +114,10 @@ def hello_world(): ) assert result.success - assert len(result.result_list) == 2 - assert result.result_list[0].transformed_value['foo'] == 'left' - assert result.result_list[1].transformed_value['foo'] == 'right' + assert len(result.result_list) == 1 + solid_result = result.result_list[0] + assert solid_result.transformed_value('left')['foo'] == 'left' + assert solid_result.transformed_value('right')['foo'] == 'right' def test_dict_multiple_outputs(): @@ -141,9 +142,10 @@ def hello_world(): ) assert result.success - assert len(result.result_list) == 2 - assert result.result_list[0].transformed_value['foo'] == 'left' - assert result.result_list[1].transformed_value['foo'] == 'right' + assert len(result.result_list) == 1 + solid_result = result.result_list[0] + assert solid_result.transformed_value('left')['foo'] == 'left' + assert solid_result.transformed_value('right')['foo'] == 'right' def test_solid_with_name(): @@ -159,7 +161,7 @@ def hello_world(): assert result.success assert len(result.result_list) == 1 - assert result.result_list[0].transformed_value['foo'] == 'bar' + assert result.result_list[0].transformed_value()['foo'] == 'bar' def test_solid_with_context(): @@ -176,7 +178,7 @@ def hello_world(_context): assert result.success assert len(result.result_list) == 1 - assert result.result_list[0].transformed_value['foo'] == 'bar' + assert result.result_list[0].transformed_value()['foo'] == 'bar' def test_solid_with_input(): @@ -195,11 +197,12 @@ def hello_world(foo_to_foo): pipeline, environment=config.Environment(), ) + print([p.solid.name for p in pipeline_result.result_list]) - result = pipeline_result.result_named('hello_world') + result = pipeline_result.result_for_solid('hello_world') assert result.success - assert result.transformed_value['foo'] == 'bar' + assert result.transformed_value()['foo'] == 'bar' def test_solid_definition_errors(): diff --git a/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py b/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py index 8f885b7b32006..2271893b2c988 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py +++ b/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py @@ -33,14 +33,11 @@ def _t_fn(_context, _inputs, _config_dict): pipeline = PipelineDefinition(solids=[solid]) result = execute_pipeline(pipeline, config.Environment()) + solid_result = result.result_list[0] - assert result.result_list[0].name == 'multiple_outputs' - assert result.result_list[0].output_name == 'output_one' - assert result.result_list[0].transformed_value == 'foo' - - assert result.result_list[1].name == 'multiple_outputs' - assert result.result_list[1].output_name == 'output_two' - assert result.result_list[1].transformed_value == 'bar' + assert solid_result.solid.name == 'multiple_outputs' + assert solid_result.transformed_value('output_one') == 'foo' + assert solid_result.transformed_value('output_two') == 'bar' def test_multiple_outputs_expectations(): diff --git a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py index c8f723bd69587..1aaa6f24eaea6 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py +++ b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py @@ -55,7 +55,7 @@ def test_execute_solid_with_input_same_name(): config.Environment(solids={'pass_value': config.Solid(config_dict={'value': 'foo'})}), ) - assert result.result_named('a_thing').transformed_value == 'foofoo' + assert result.result_for_solid('a_thing').transformed_value() == 'foofoo' def test_execute_two_solids_with_same_input_name(): @@ -108,8 +108,8 @@ def test_execute_two_solids_with_same_input_name(): assert result.success - assert result.result_named('solid_one').transformed_value == 'foofoo' - assert result.result_named('solid_two').transformed_value == 'barbar' + assert result.result_for_solid('solid_one').transformed_value() == 'foofoo' + assert result.result_for_solid('solid_two').transformed_value() == 'barbar' def test_execute_dep_solid_different_input_name(): @@ -152,6 +152,6 @@ def test_execute_dep_solid_different_input_name(): assert result.success assert len(result.result_list) == 3 - assert result.result_list[0].transformed_value == 'bar' - assert result.result_list[1].transformed_value == 'barbar' - assert result.result_list[2].transformed_value == 'barbarbarbar' + assert result.result_list[0].transformed_value() == 'bar' + assert result.result_list[1].transformed_value() == 'barbar' + assert result.result_list[2].transformed_value() == 'barbarbarbar' diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py index 8f74978bfc528..92354490a3b87 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py @@ -109,6 +109,7 @@ def transform_fn(_context, args): ) result_list = pipeline_result.result_list + print([r.solid.name for r in result_list]) assert result_list[0].success assert result_list[1].success diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py index b99fc5c7c9b51..7586675c01f1b 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py @@ -16,8 +16,7 @@ ) from dagster.core.execution import ( - execute_pipeline_iterator, - ExecutionStepResult, + execute_pipeline_iterator, ExecutionContext, SolidExecutionResult, PipelineExecutionResult ) from dagster.core.utility_solids import define_stub_solid @@ -35,18 +34,29 @@ def create_dep_input_fn(name): return lambda context, arg_dict: {name: 'input_set'} +def make_transform(name): + def transform(_context, args): + passed_rows = [] + seen = set() + for row in args.values(): + for item in row: + key = list(item.keys())[0] + if key not in seen: + seen.add(key) + passed_rows.append(item) + + return [*passed_rows, {name: 'transform_called'}] + + return transform + + def create_solid_with_deps(name, *solid_deps): inputs = [InputDefinition(solid_dep.name) for solid_dep in solid_deps] - def dep_transform(_context, args): - passed_rows = list(args.values())[0] - passed_rows.append({name: 'transform_called'}) - return passed_rows - return SolidDefinition.single_output_transform( name=name, inputs=inputs, - transform_fn=dep_transform, + transform_fn=make_transform(name), output=OutputDefinition(), ) @@ -55,16 +65,10 @@ def create_root_solid(name): input_name = name + '_input' inp = InputDefinition(input_name) - def root_transform(_context, args): - passed_rows = list(args.values())[0] - - passed_rows.append({name: 'transform_called'}) - return passed_rows - return SolidDefinition.single_output_transform( name=name, inputs=[inp], - transform_fn=root_transform, + transform_fn=make_transform(name), output=OutputDefinition(), ) @@ -213,8 +217,8 @@ def transform_called(name): def assert_equivalent_results(left, right): - check.inst_param(left, 'left', ExecutionStepResult) - check.inst_param(right, 'right', ExecutionStepResult) + check.inst_param(left, 'left', SolidExecutionResult) + check.inst_param(right, 'right', SolidExecutionResult) assert left.success == right.success assert left.name == right.name @@ -223,8 +227,8 @@ def assert_equivalent_results(left, right): def assert_all_results_equivalent(expected_results, result_results): - check.list_param(expected_results, 'expected_results', of_type=ExecutionStepResult) - check.list_param(result_results, 'result_results', of_type=ExecutionStepResult) + check.list_param(expected_results, 'expected_results', of_type=SolidExecutionResult) + check.list_param(result_results, 'result_results', of_type=SolidExecutionResult) assert len(expected_results) == len(result_results) for expected, result in zip(expected_results, result_results): assert_equivalent_results(expected, result) @@ -255,42 +259,36 @@ def _do_test(do_execute_pipeline_iter): results = list() for result in do_execute_pipeline_iter(): - results.append(result.copy()) + print(result.solid.name) + print(result.transformed_value()) - assert results[1].transformed_value[0] == input_set('A_input') - assert results[1].transformed_value[1] == transform_called('A') + results.append(result) - assert results[1].transformed_value == [input_set('A_input'), transform_called('A')] + result = PipelineExecutionResult(ExecutionContext(), results) - assert results[2].transformed_value == [ - input_set('A_input'), - transform_called('A'), - transform_called('C'), - ] or results[2].transformed_value == [ - input_set('A_input'), - transform_called('A'), - transform_called('B'), + assert result.result_for_solid('A').transformed_value() == [ + input_set('A_input'), transform_called('A') ] - assert results[3].transformed_value == [ + assert result.result_for_solid('B').transformed_value() == [ input_set('A_input'), transform_called('A'), - transform_called('C'), transform_called('B'), - ] or results[3].transformed_value == [ + ] + + assert result.result_for_solid('C').transformed_value() == [ input_set('A_input'), transform_called('A'), - transform_called('B'), transform_called('C'), ] - assert results[4].transformed_value == [ + assert result.result_for_solid('D').transformed_value() == [ input_set('A_input'), transform_called('A'), transform_called('C'), transform_called('B'), transform_called('D'), - ] or results[4].transformed_value == [ + ] or result.result_for_solid('D').transformed_value() == [ input_set('A_input'), transform_called('A'), transform_called('B'), diff --git a/python_modules/dagster/dagster/core/errors.py b/python_modules/dagster/dagster/core/errors.py index 0c9fc63a13d51..874888bb80656 100644 --- a/python_modules/dagster/dagster/core/errors.py +++ b/python_modules/dagster/dagster/core/errors.py @@ -60,5 +60,5 @@ def __init__(self, execution_result, *args, **kwargs): self.execution_result = check.opt_inst_param( execution_result, 'execution_result', - dagster.core.execution.ExecutionStepResult, + dagster.core.execution.SolidExecutionResult, ) diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index 8c1e8238abbab..c336c38593330 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -16,6 +16,7 @@ # pylint: disable=C0302 from contextlib import contextmanager +import itertools import copy import six @@ -25,7 +26,9 @@ config, ) -from .definitions import PipelineDefinition +from .definitions import ( + SolidDefinition, PipelineDefinition, PipelineContextDefinition, DEFAULT_OUTPUT +) from .errors import ( DagsterUserCodeExecutionError, @@ -36,6 +39,7 @@ from .compute_nodes import ( ComputeNodeTag, + ComputeNodeResult, create_compute_node_graph_from_env, execute_compute_nodes, ) @@ -45,86 +49,124 @@ from .graph import ExecutionGraph -class DagsterPipelineExecutionResult: +class PipelineExecutionResult: def __init__( self, context, result_list, ): self.context = check.inst_param(context, 'context', ExecutionContext) - self.result_list = check.list_param(result_list, 'result_list', of_type=ExecutionStepResult) + self.result_list = check.list_param( + result_list, 'result_list', of_type=SolidExecutionResult + ) @property def success(self): return all([result.success for result in self.result_list]) - def result_named(self, name): + def result_for_solid(self, name): check.str_param(name, 'name') for result in self.result_list: - if result.name == name: + if result.solid.name == name: return result - check.failed('Did not find result {name} in pipeline execution result'.format(name=name)) - - -class ExecutionStepResult: - def __init__( - self, *, success, context, transformed_value, name, dagster_user_exception, solid, tag, - output_name - ): - self.success = check.bool_param(success, 'success') - self.context = context - self.transformed_value = transformed_value - self.name = name - self.dagster_user_exception = dagster_user_exception - self.solid = solid - self.tag = tag - self.output_name = output_name - - def copy(self): - ''' This must be used instead of copy.deepcopy() because exceptions cannot - be deepcopied''' - return ExecutionStepResult( - name=self.name, - solid=self.solid, - success=self.success, - transformed_value=copy.deepcopy(self.transformed_value), - context=self.context, - dagster_user_exception=self.dagster_user_exception, - tag=self.tag, - output_name=self.output_name + check.failed( + 'Did not find result for solid {name} in pipeline execution result'.format(name=name) ) - def reraise_user_error(self): - check.inst(self.dagster_user_exception, DagsterUserCodeExecutionError) - six.reraise(*self.dagster_user_exception.original_exc_info) +class SolidExecutionResult: + def __init__(self, *, context, solid, input_expectations, transforms, output_expectations): + self.context = check.inst_param(context, 'context', ExecutionContext) + self.solid = check.inst_param(solid, 'solid', SolidDefinition) + self.input_expectations = check.list_param( + input_expectations, 'input_expectations', ComputeNodeResult + ) + self.output_expectations = check.list_param( + output_expectations, 'output_expectations', ComputeNodeResult + ) + self.transforms = check.list_param(transforms, 'transforms', ComputeNodeResult) + + @staticmethod + def from_results(context, results): + results = check.list_param(results, 'results', ComputeNodeResult) + if results: + input_expectations = [] + output_expectations = [] + transforms = [] + + for result in results: + if result.tag == ComputeNodeTag.INPUT_EXPECTATION: + input_expectations.append(result) + elif result.tag == ComputeNodeTag.OUTPUT_EXPECTATION: + output_expectations.append(result) + elif result.tag == ComputeNodeTag.TRANSFORM: + transforms.append(result) + + return SolidExecutionResult( + context=context, + solid=results[0].compute_node.solid, + input_expectations=input_expectations, + output_expectations=output_expectations, + transforms=transforms, + ) + else: + check.failed("Cannot create SolidExecutionResult from empty list") -def copy_result_list(result_list): - if result_list is None: - return result_list + @property + def success(self): + return all( + [ + result.success for result in + itertools.chain(self.input_expectations, self.output_expectations, self.transforms) + ] + ) - return [result.copy() for result in result_list] + @property + def transformed_values(self): + if self.success and self.transforms: + return { + result.success_data.output_name: result.success_data.value + for result in self.transforms + } + else: + return None + + def transformed_value(self, output_name=DEFAULT_OUTPUT): + check.str_param(output_name, 'output_name') + if self.success: + for result in self.transforms: + if result.success_data.output_name == output_name: + return result.success_data.value + check.failed( + f'Did not find result {output_name} in solid {self.solid.name} execution result' + ) + else: + return None + def reraise_user_error(self): + if not self.success: + for result in itertools.chain( + self.input_expectations, self.output_expectations, self.transforms + ): + if not result.success: + six.reraise(*result.failure_data.dagster_user_exception.original_exc_info) -def copy_result_dict(result_dict): - if result_dict is None: - return None - new_dict = {} - for input_name, result in result_dict.items(): - new_dict[input_name] = result.copy() - return new_dict + @property + def dagster_user_exception(self): + for result in itertools.chain( + self.input_expectations, self.output_expectations, self.transforms + ): + if not result.success: + return result.failure_data.dagster_user_exception def _do_throw_on_error(execution_result): - check.inst_param(execution_result, 'execution_result', ExecutionStepResult) + check.inst_param(execution_result, 'execution_result', SolidExecutionResult) if execution_result.success: return - - if isinstance(execution_result.dagster_user_exception, DagsterUserCodeExecutionError): + else: execution_result.reraise_user_error() - raise execution_result.dagster_user_exception - def _wrap_in_yield(thing): if isinstance(thing, ExecutionContext): @@ -204,32 +246,26 @@ def _execute_graph_iterator(context, execution_graph, env): check.invariant(len(cn_nodes[0].node_inputs) == 0) + solid = None + solid_results = [] for cn_result in execute_compute_nodes(context, cn_nodes): cn_node = cn_result.compute_node + + if solid and solid is not cn_node.solid: + yield SolidExecutionResult.from_results(context, solid_results) + solid_results = [] + if not cn_result.success: - yield ExecutionStepResult( - success=False, - context=context, - transformed_value=None, - name=cn_node.solid.name, - dagster_user_exception=cn_result.failure_data.dagster_user_exception, - solid=cn_node.solid, - tag=cn_result.tag, - output_name=None - ) + solid_results.append(cn_result) + yield SolidExecutionResult.from_results(context, solid_results) + solid_results = [] return - if cn_node.tag == ComputeNodeTag.TRANSFORM: - yield ExecutionStepResult( - success=True, - context=context, - transformed_value=cn_result.success_data.value, - name=cn_node.solid.name, - dagster_user_exception=None, - solid=cn_node.solid, - tag=cn_node.tag, - output_name=cn_result.success_data.output_name, - ) + solid = cn_node.solid + solid_results.append(cn_result) + + if solid and solid_results: + yield SolidExecutionResult.from_results(context, solid_results) def execute_pipeline( @@ -271,5 +307,5 @@ def _execute_graph( if not result.success: _do_throw_on_error(result) - results.append(result.copy()) - return DagsterPipelineExecutionResult(context, results) + results.append(result) + return PipelineExecutionResult(context, results) diff --git a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/pandas_hello_world/test_pandas_hello_world.py b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/pandas_hello_world/test_pandas_hello_world.py index 3c845eec2dc3f..a190a4fee465c 100644 --- a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/pandas_hello_world/test_pandas_hello_world.py +++ b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/pandas_hello_world/test_pandas_hello_world.py @@ -27,13 +27,13 @@ def test_execute_pipeline(): assert result.success - assert result.result_named('sum_solid').transformed_value.to_dict('list') == { + assert result.result_for_solid('sum_solid').transformed_value().to_dict('list') == { 'num1': [1, 3], 'num2': [2, 4], 'sum': [3, 7], } - assert result.result_named('sum_sq_solid').transformed_value.to_dict('list') == { + assert result.result_for_solid('sum_sq_solid').transformed_value().to_dict('list') == { 'num1': [1, 3], 'num2': [2, 4], 'sum': [3, 7], diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py index 4205e7e9b4df2..583e4a12697d4 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py @@ -50,11 +50,11 @@ def run_hello_world(hello_world): environment=create_num_csv_environment(), ) - result = pipeline_result.result_named('hello_world') + result = pipeline_result.result_for_solid('hello_world') assert result.success - assert result.transformed_value.to_dict('list') == { + assert result.transformed_value().to_dict('list') == { 'num1': [1, 3], 'num2': [2, 4], 'sum': [3, 7], @@ -92,7 +92,7 @@ def run_hello_world(hello_world): environment, ) - output_result = pipeline_result.result_named('hello_world') + output_result = pipeline_result.result_for_solid('hello_world') assert output_result.success diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py index f2c4a236c230b..59bacd118f288 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py @@ -84,9 +84,9 @@ def hello_world_transform_fn(_context, args): assert pipeline_result.success - result = pipeline_result.result_named('hello_world') + result = pipeline_result.result_for_solid('hello_world') - assert result.transformed_value.to_dict('list') == { + assert result.transformed_value().to_dict('list') == { 'num1': [1, 3], 'num2': [2, 4], 'sum': [3, 7], @@ -130,11 +130,11 @@ def test_hello_world_composed(): assert pipeline_result.success - result = pipeline_result.result_named('hello_world') + result = pipeline_result.result_for_solid('hello_world') assert result.success - assert result.transformed_value.to_dict('list') == { + assert result.transformed_value().to_dict('list') == { 'num1': [1, 3], 'num2': [2, 4], 'sum': [3, 7], @@ -191,7 +191,7 @@ def solid_two_transform(_context, args): environment=environment, ) - assert execute_pipeline_result.result_named('solid_two').transformed_value.to_dict('list') == { + assert execute_pipeline_result.result_for_solid('solid_two').transformed_value().to_dict('list') == { 'num1': [1, 3], 'num2': [2, 4], 'sum': [3, 7], diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py index 44c3bd3f8b39d..d04271cf9c90b 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py @@ -50,9 +50,9 @@ def get_solid_transformed_value(_context, solid_inst, environment): pipeline_result = execute_pipeline(pipeline, environment) - execution_result = pipeline_result.result_named(solid_inst.name) + execution_result = pipeline_result.result_for_solid(solid_inst.name) - return execution_result.transformed_value + return execution_result.transformed_value() def get_load_only_solids_config(load_csv_solid_name): @@ -96,7 +96,7 @@ def transform(_context, args): assert pipeline_result.success - assert pipeline_result.result_named('sum_table').transformed_value.to_dict('list') == { + assert pipeline_result.result_for_solid('sum_table').transformed_value().to_dict('list') == { 'num1': [1, 3], 'num2': [2, 4], 'sum': [3, 7] @@ -276,7 +276,7 @@ def transform(_context, args): pipeline_result = execute_pipeline(pipeline, environment) assert pipeline_result.success - df = pipeline_result.result_named('two_input_solid').transformed_value + df = pipeline_result.result_for_solid('two_input_solid').transformed_value() # df = get_solid_transformed_value(create_test_context(), two_input_solid, environment) assert isinstance(df, pd.DataFrame) @@ -381,7 +381,7 @@ def test_pandas_in_memory_diamond_pipeline(): pipeline, environment=get_num_csv_environment(get_load_only_solids_config('load_csv')) ) - assert result.result_named('sum_mult_table').transformed_value.to_dict('list') == { + assert result.result_for_solid('sum_mult_table').transformed_value().to_dict('list') == { 'num1': [1, 3], 'num2': [2, 4], 'sum': [3, 7], @@ -424,7 +424,7 @@ def test_pandas_output_csv_pipeline(): } -def _result_named(results, name): +def _result_for_solid(results, name): for result in results: if result.name == name: return result @@ -477,8 +477,8 @@ def test_pandas_output_intermediate_csv_files(): } assert pd.read_csv(sum_file).to_dict('list') == expected_sum - sum_table_result = subgraph_one_result.result_named('sum_table') - assert sum_table_result.transformed_value.to_dict('list') == expected_sum + sum_table_result = subgraph_one_result.result_for_solid('sum_table') + assert sum_table_result.transformed_value().to_dict('list') == expected_sum expected_mult = { 'num1': [1, 3], @@ -486,8 +486,8 @@ def test_pandas_output_intermediate_csv_files(): 'mult': [2, 12], } assert pd.read_csv(mult_file).to_dict('list') == expected_mult - mult_table_result = subgraph_one_result.result_named('mult_table') - assert mult_table_result.transformed_value.to_dict('list') == expected_mult + mult_table_result = subgraph_one_result.result_for_solid('mult_table') + assert mult_table_result.transformed_value().to_dict('list') == expected_mult injected_solids = { 'sum_mult_table': { @@ -520,7 +520,7 @@ def test_pandas_output_intermediate_csv_files(): subgraph_two_result_list = pipeline_result.result_list assert len(subgraph_two_result_list) == 3 - output_df = pipeline_result.result_named('sum_mult_table').transformed_value + output_df = pipeline_result.result_for_solid('sum_mult_table').transformed_value() assert output_df.to_dict('list') == { 'num1': [1, 3], 'num2': [2, 4], @@ -622,7 +622,7 @@ def transform_fn(_context, args): output_df = execute_pipeline( pipeline, environment=environment, - ).result_named('double_sum').transformed_value + ).result_for_solid('double_sum').transformed_value() assert not output_df.empty @@ -708,9 +708,10 @@ def test_rename_input(): assert result.success - assert result.result_named('sum_sq_table_renamed_input').transformed_value.to_dict('list') == { - 'num1': [1, 3], - 'num2': [2, 4], - 'sum': [3, 7], - 'sum_squared': [9, 49], - } + assert result.result_for_solid('sum_sq_table_renamed_input' + ).transformed_value().to_dict('list') == { + 'num1': [1, 3], + 'num2': [2, 4], + 'sum': [3, 7], + 'sum_squared': [9, 49], + } diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_basic_solid.py b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_basic_solid.py index 68bef85bc435f..ae5bbbba3c61b 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_basic_solid.py +++ b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_basic_solid.py @@ -81,7 +81,7 @@ def test_sql_sum_solid(): pipeline_result = execute_pipeline(pipeline, environment) assert pipeline_result.success - result = pipeline_result.result_named(sum_table_solid.name) + result = pipeline_result.result_for_solid(sum_table_solid.name) assert result.success @@ -142,10 +142,10 @@ def test_execute_sql_sum_sq_solid(): result_list = pipeline_result.result_list - sum_table_sql_text = result_list[1].transformed_value.query_text + sum_table_sql_text = result_list[1].transformed_value().query_text assert sum_table_sql_text == 'SELECT num1, num2, num1 + num2 as sum FROM num_table' - sum_sq_table_sql_text = result_list[2].transformed_value.query_text + sum_sq_table_sql_text = result_list[2].transformed_value().query_text assert sum_sq_table_sql_text == 'SELECT num1, num2, sum, sum * sum as sum_sq from ' + \ '(SELECT num1, num2, num1 + num2 as sum FROM num_table)' diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py index 0a431d5dc3273..46769ea30642c 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py +++ b/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py @@ -184,8 +184,8 @@ def test_templated_sql_solid_pipeline(): assert first_result.success assert len(first_result.result_list) == 2 - assert first_result.result_list[0].transformed_value == {'sum_table': first_sum_table} - assert first_result.result_list[1].transformed_value == { + assert first_result.result_list[0].transformed_value() == {'sum_table': first_sum_table} + assert first_result.result_list[1].transformed_value() == { 'sum_table': first_sum_table, 'sum_sq_table': first_sum_sq_table, } diff --git a/python_modules/dagster/tox.ini b/python_modules/dagster/tox.ini index 306a6bcb5f4c3..9fb2fd067d2e5 100644 --- a/python_modules/dagster/tox.ini +++ b/python_modules/dagster/tox.ini @@ -5,4 +5,4 @@ envlist = py36 deps = pytest pytest-runner -commands = pytest +commands = pytest -v From 31b46aa462da4c3f1f5acb526c864e7309694a37 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Thu, 30 Aug 2018 06:55:15 -0700 Subject: [PATCH 037/103] Minor cleanup post execution result PR Trivial stuff --- .../core/core_tests/test_pipeline_execution.py | 3 --- python_modules/dagster/dagster/core/execution.py | 10 ++++------ .../pandas_kernel_tests/test_pandas_solids.py | 15 ++++++++------- 3 files changed, 12 insertions(+), 16 deletions(-) diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py index 7586675c01f1b..c978fbfb39e78 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py @@ -259,9 +259,6 @@ def _do_test(do_execute_pipeline_iter): results = list() for result in do_execute_pipeline_iter(): - print(result.solid.name) - print(result.transformed_value()) - results.append(result) result = PipelineExecutionResult(ExecutionContext(), results) diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index c336c38593330..f035a262b4dd5 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -17,7 +17,6 @@ from contextlib import contextmanager import itertools -import copy import six @@ -27,13 +26,12 @@ ) from .definitions import ( - SolidDefinition, PipelineDefinition, PipelineContextDefinition, DEFAULT_OUTPUT + DEFAULT_OUTPUT, + PipelineDefinition, + SolidDefinition, ) -from .errors import ( - DagsterUserCodeExecutionError, - DagsterInvariantViolationError, -) +from .errors import DagsterInvariantViolationError from .argument_handling import validate_args diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py index d04271cf9c90b..dd95787aebe81 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py +++ b/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py @@ -708,10 +708,11 @@ def test_rename_input(): assert result.success - assert result.result_for_solid('sum_sq_table_renamed_input' - ).transformed_value().to_dict('list') == { - 'num1': [1, 3], - 'num2': [2, 4], - 'sum': [3, 7], - 'sum_squared': [9, 49], - } + expected = { + 'num1': [1, 3], + 'num2': [2, 4], + 'sum': [3, 7], + 'sum_squared': [9, 49], + } + solid_result = result.result_for_solid('sum_sq_table_renamed_input') + assert solid_result.transformed_value().to_dict('list') == expected From df1e33b1b3aa1ec3ae2cb089390f195b975f35bc Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Thu, 30 Aug 2018 07:00:04 -0700 Subject: [PATCH 038/103] {pandas, sqlalchemy}_kernel -> {pandas, sqlalchemy} Renaming these because the kernel nomenclature no longer makes sense --- python_modules/.vscode/settings.json | 6 +++--- python_modules/Makefile | 4 ++-- .../dagster-ge/dagster_ge_tests/test_pandas_ge.py | 2 +- python_modules/dagster/README.rst | 4 ++-- .../dagster/core/core_tests/test_pipeline_execution.py | 1 - .../sql_project_example/test_sql_project_pipeline.py | 4 ++-- .../dagster/dagster_examples/pandas_hello_world/pipeline.py | 2 +- .../dagster/dagster_examples/sql_hello_world/pipeline.py | 2 +- .../dagster_examples/sql_project_example/pipelines.py | 2 +- .../dagster/dagster/{pandas_kernel => pandas}/__init__.py | 0 .../pandas_kernel_tests => pandas/pandas_tests}/__init__.py | 0 .../pandas_kernel_tests => pandas/pandas_tests}/num.csv | 0 .../pandas_tests}/num_table.csv | 0 .../pandas_tests}/test_pandas_hello_world_library_slide.py | 2 +- .../test_pandas_hello_world_no_library_slide.py | 0 .../pandas_tests}/test_pandas_solids.py | 2 +- .../pandas_tests}/test_pandas_user_error.py | 2 +- .../dagster/{sqlalchemy_kernel => sqlalchemy}/__init__.py | 0 .../dagster/{sqlalchemy_kernel => sqlalchemy}/common.py | 0 .../sqlalchemy_tests}/__init__.py | 0 .../sqlalchemy_tests}/math_test_db.py | 2 +- .../sqlalchemy_tests}/sql_files/create_sum_sq_table.sql | 0 .../sqlalchemy_tests}/sql_files/create_sum_table.sql | 0 .../sqlalchemy_tests}/test_basic_solid.py | 2 +- .../sqlalchemy_tests}/test_isolated_sql_tests.py | 2 +- .../sqlalchemy_tests}/test_isolated_templated_sql_tests.py | 2 +- .../sqlalchemy_tests}/test_mocked_context.py | 2 +- .../sqlalchemy_tests}/test_sql_libraries.py | 0 .../subquery_builder_experimental.py | 2 +- .../dagster/{sqlalchemy_kernel => sqlalchemy}/templated.py | 0 python_modules/dagster/setup.py | 2 +- 31 files changed, 23 insertions(+), 24 deletions(-) rename python_modules/dagster/dagster/{pandas_kernel => pandas}/__init__.py (100%) rename python_modules/dagster/dagster/{pandas_kernel/pandas_kernel_tests => pandas/pandas_tests}/__init__.py (100%) rename python_modules/dagster/dagster/{pandas_kernel/pandas_kernel_tests => pandas/pandas_tests}/num.csv (100%) rename python_modules/dagster/dagster/{pandas_kernel/pandas_kernel_tests => pandas/pandas_tests}/num_table.csv (100%) rename python_modules/dagster/dagster/{pandas_kernel/pandas_kernel_tests => pandas/pandas_tests}/test_pandas_hello_world_library_slide.py (98%) rename python_modules/dagster/dagster/{pandas_kernel/pandas_kernel_tests => pandas/pandas_tests}/test_pandas_hello_world_no_library_slide.py (100%) rename python_modules/dagster/dagster/{pandas_kernel/pandas_kernel_tests => pandas/pandas_tests}/test_pandas_solids.py (99%) rename python_modules/dagster/dagster/{pandas_kernel/pandas_kernel_tests => pandas/pandas_tests}/test_pandas_user_error.py (97%) rename python_modules/dagster/dagster/{sqlalchemy_kernel => sqlalchemy}/__init__.py (100%) rename python_modules/dagster/dagster/{sqlalchemy_kernel => sqlalchemy}/common.py (100%) rename python_modules/dagster/dagster/{sqlalchemy_kernel/sqlalchemy_kernel_tests => sqlalchemy/sqlalchemy_tests}/__init__.py (100%) rename python_modules/dagster/dagster/{sqlalchemy_kernel/sqlalchemy_kernel_tests => sqlalchemy/sqlalchemy_tests}/math_test_db.py (94%) rename python_modules/dagster/dagster/{sqlalchemy_kernel/sqlalchemy_kernel_tests => sqlalchemy/sqlalchemy_tests}/sql_files/create_sum_sq_table.sql (100%) rename python_modules/dagster/dagster/{sqlalchemy_kernel/sqlalchemy_kernel_tests => sqlalchemy/sqlalchemy_tests}/sql_files/create_sum_table.sql (100%) rename python_modules/dagster/dagster/{sqlalchemy_kernel/sqlalchemy_kernel_tests => sqlalchemy/sqlalchemy_tests}/test_basic_solid.py (98%) rename python_modules/dagster/dagster/{sqlalchemy_kernel/sqlalchemy_kernel_tests => sqlalchemy/sqlalchemy_tests}/test_isolated_sql_tests.py (98%) rename python_modules/dagster/dagster/{sqlalchemy_kernel/sqlalchemy_kernel_tests => sqlalchemy/sqlalchemy_tests}/test_isolated_templated_sql_tests.py (99%) rename python_modules/dagster/dagster/{sqlalchemy_kernel/sqlalchemy_kernel_tests => sqlalchemy/sqlalchemy_tests}/test_mocked_context.py (92%) rename python_modules/dagster/dagster/{sqlalchemy_kernel/sqlalchemy_kernel_tests => sqlalchemy/sqlalchemy_tests}/test_sql_libraries.py (100%) rename python_modules/dagster/dagster/{sqlalchemy_kernel => sqlalchemy}/subquery_builder_experimental.py (98%) rename python_modules/dagster/dagster/{sqlalchemy_kernel => sqlalchemy}/templated.py (100%) diff --git a/python_modules/.vscode/settings.json b/python_modules/.vscode/settings.json index 2be7fbfa0b1a0..f3b097dd7d55c 100644 --- a/python_modules/.vscode/settings.json +++ b/python_modules/.vscode/settings.json @@ -5,10 +5,10 @@ "dagster/dagster_tests", "dagster/dagster/core/core_tests", "dagster/dagster/utils/utils_tests", - "dagster/dagster/pandas_kernel/pandas_kernel_tests", - "dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests", + "dagster/dagster/pandas/pandas_tests", + "dagster/dagster/sqlalchemy/sqlalchemy_tests", "dagster/dagster/dagster_examples/dagster_examples_tests", - "dagster-ge/dagster_ge_tests", + "dagster-ge/dagster_ge_tests" ], "python.unitTest.pyTestEnabled": true, "python.pythonPath": "/Users/schrockn/code/venvs/dagster/bin/python", diff --git a/python_modules/Makefile b/python_modules/Makefile index e336be44bdc7d..428c890212ab5 100644 --- a/python_modules/Makefile +++ b/python_modules/Makefile @@ -4,8 +4,8 @@ watch: "dagster/dagster/check/check_tests "\ "dagster/dagster/core/core_tests "\ "dagster/dagster/dagster_examples/dagster_examples_tests "\ - "dagster/dagster/pandas_kernel/pandas_kernel_tests "\ - "dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests "\ + "dagster/dagster/pandas/pandas_tests "\ + "dagster/dagster/sqlalchemy/sqlalchemy_tests "\ "dagster/dagster/utils/utils_tests "\ "dagster/dagster_tests "\ "dagster-ge/dagster_ge_tests " \ diff --git a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py index a8b8ad82b0a22..2ede5fc2f0172 100644 --- a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py +++ b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py @@ -13,7 +13,7 @@ ) from dagster.core.errors import DagsterExpectationFailedError from dagster.core.utility_solids import define_stub_solid -import dagster.pandas_kernel as dagster_pd +import dagster.pandas as dagster_pd from dagster.utils import script_relative_path import dagster_ge diff --git a/python_modules/dagster/README.rst b/python_modules/dagster/README.rst index e019f6e44d6c7..74ae42b59594c 100644 --- a/python_modules/dagster/README.rst +++ b/python_modules/dagster/README.rst @@ -134,7 +134,7 @@ We can simplify the above example by using built-in dagster pandas inputs and ou import dagster.core from dagster import config from dagster.core.decorators import solid, with_context - import dagster.pandas_kernel as dagster_pd + import dagster.pandas as dagster_pd @solid( inputs=[ @@ -347,7 +347,7 @@ pandas kernel. (Note: the "kernel" terminology is not settled) .. code-block:: python import dagster - import dagster.pandas_kernel as dagster_pd + import dagster.pandas as dagster_pd def sum_transform(num_df): num_df['sum'] = num_df['num1'] + num_df['num2'] diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py index c978fbfb39e78..7ef46455c21a3 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py @@ -253,7 +253,6 @@ def test_pipeline_execution_graph_diamond(): # # input_values={'A': input_values}, # )) - def _do_test(do_execute_pipeline_iter): results = list() diff --git a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py index 2587edc135081..5640abe977914 100644 --- a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py +++ b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py @@ -1,10 +1,10 @@ import sqlalchemy as sa import dagster from dagster import (config, InputDefinition, DependencyDefinition) -import dagster.sqlalchemy_kernel as dagster_sa +import dagster.sqlalchemy as dagster_sa from dagster.utils import script_relative_path -from dagster.sqlalchemy_kernel.subquery_builder_experimental import sql_file_solid +from dagster.sqlalchemy.subquery_builder_experimental import sql_file_solid def in_mem_engine(): diff --git a/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py b/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py index 1f51895ea21ed..854ad62ac2092 100644 --- a/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py +++ b/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py @@ -6,7 +6,7 @@ PipelineDefinition, solid, ) -import dagster.pandas_kernel as dagster_pd +import dagster.pandas as dagster_pd @solid( diff --git a/python_modules/dagster/dagster/dagster_examples/sql_hello_world/pipeline.py b/python_modules/dagster/dagster/dagster_examples/sql_hello_world/pipeline.py index b1cb93242b8c9..5af7682f4f17c 100644 --- a/python_modules/dagster/dagster/dagster_examples/sql_hello_world/pipeline.py +++ b/python_modules/dagster/dagster/dagster_examples/sql_hello_world/pipeline.py @@ -5,7 +5,7 @@ PipelineDefinition, ) -from dagster.sqlalchemy_kernel.subquery_builder_experimental import ( +from dagster.sqlalchemy.subquery_builder_experimental import ( create_sql_solid, create_table_expression_input, ) diff --git a/python_modules/dagster/dagster/dagster_examples/sql_project_example/pipelines.py b/python_modules/dagster/dagster/dagster_examples/sql_project_example/pipelines.py index a16f5fb6c2b1f..c0bee78c5eda7 100644 --- a/python_modules/dagster/dagster/dagster_examples/sql_project_example/pipelines.py +++ b/python_modules/dagster/dagster/dagster_examples/sql_project_example/pipelines.py @@ -1,6 +1,6 @@ import os import dagster -import dagster.sqlalchemy_kernel as dagster_sa +import dagster.sqlalchemy as dagster_sa from dagster import ( DependencyDefinition, diff --git a/python_modules/dagster/dagster/pandas_kernel/__init__.py b/python_modules/dagster/dagster/pandas/__init__.py similarity index 100% rename from python_modules/dagster/dagster/pandas_kernel/__init__.py rename to python_modules/dagster/dagster/pandas/__init__.py diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/__init__.py b/python_modules/dagster/dagster/pandas/pandas_tests/__init__.py similarity index 100% rename from python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/__init__.py rename to python_modules/dagster/dagster/pandas/pandas_tests/__init__.py diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/num.csv b/python_modules/dagster/dagster/pandas/pandas_tests/num.csv similarity index 100% rename from python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/num.csv rename to python_modules/dagster/dagster/pandas/pandas_tests/num.csv diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/num_table.csv b/python_modules/dagster/dagster/pandas/pandas_tests/num_table.csv similarity index 100% rename from python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/num_table.csv rename to python_modules/dagster/dagster/pandas/pandas_tests/num_table.csv diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_library_slide.py similarity index 98% rename from python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py rename to python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_library_slide.py index 583e4a12697d4..dccd24ab14f9e 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_library_slide.py +++ b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_library_slide.py @@ -14,7 +14,7 @@ from dagster.utils import script_relative_path from dagster.utils.test import get_temp_file_name -import dagster.pandas_kernel as dagster_pd +import dagster.pandas as dagster_pd def create_num_csv_environment(): diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_no_library_slide.py similarity index 100% rename from python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_hello_world_no_library_slide.py rename to python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_no_library_slide.py diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py similarity index 99% rename from python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py rename to python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py index dd95787aebe81..78746a97ae016 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_solids.py +++ b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py @@ -19,7 +19,7 @@ execute_pipeline, ) -import dagster.pandas_kernel as dagster_pd +import dagster.pandas as dagster_pd from dagster.utils import script_relative_path diff --git a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_user_error.py b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_user_error.py similarity index 97% rename from python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_user_error.py rename to python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_user_error.py index d386235191ccf..afd1cee9b8639 100644 --- a/python_modules/dagster/dagster/pandas_kernel/pandas_kernel_tests/test_pandas_user_error.py +++ b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_user_error.py @@ -4,7 +4,7 @@ import pandas as pd -import dagster.pandas_kernel as dagster_pd +import dagster.pandas as dagster_pd from dagster import ( DependencyDefinition, InputDefinition, diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/__init__.py b/python_modules/dagster/dagster/sqlalchemy/__init__.py similarity index 100% rename from python_modules/dagster/dagster/sqlalchemy_kernel/__init__.py rename to python_modules/dagster/dagster/sqlalchemy/__init__.py diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/common.py b/python_modules/dagster/dagster/sqlalchemy/common.py similarity index 100% rename from python_modules/dagster/dagster/sqlalchemy_kernel/common.py rename to python_modules/dagster/dagster/sqlalchemy/common.py diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/__init__.py b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/__init__.py similarity index 100% rename from python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/__init__.py rename to python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/__init__.py diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/math_test_db.py b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/math_test_db.py similarity index 94% rename from python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/math_test_db.py rename to python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/math_test_db.py index cba3d7cffec85..5d7e5cad85d98 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/math_test_db.py +++ b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/math_test_db.py @@ -1,5 +1,5 @@ import sqlalchemy as sa -import dagster.sqlalchemy_kernel as dagster_sa +import dagster.sqlalchemy as dagster_sa def create_num_table(engine, num_table_name='num_table'): diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/sql_files/create_sum_sq_table.sql b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/sql_files/create_sum_sq_table.sql similarity index 100% rename from python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/sql_files/create_sum_sq_table.sql rename to python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/sql_files/create_sum_sq_table.sql diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/sql_files/create_sum_table.sql b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/sql_files/create_sum_table.sql similarity index 100% rename from python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/sql_files/create_sum_table.sql rename to python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/sql_files/create_sum_table.sql diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_basic_solid.py b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_basic_solid.py similarity index 98% rename from python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_basic_solid.py rename to python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_basic_solid.py index ae5bbbba3c61b..9e562d909aad1 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_basic_solid.py +++ b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_basic_solid.py @@ -12,7 +12,7 @@ from dagster.core.utility_solids import define_stub_solid -from dagster.sqlalchemy_kernel.subquery_builder_experimental import ( +from dagster.sqlalchemy.subquery_builder_experimental import ( create_sql_solid, DagsterSqlTableExpression, define_create_table_solid, diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_sql_tests.py b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_sql_tests.py similarity index 98% rename from python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_sql_tests.py rename to python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_sql_tests.py index 94b8bccf6a70b..cad0cc066c9c3 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_sql_tests.py +++ b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_sql_tests.py @@ -7,7 +7,7 @@ execute_pipeline, ) from dagster.core.test_utils import execute_single_solid -from dagster.sqlalchemy_kernel.subquery_builder_experimental import ( +from dagster.sqlalchemy.subquery_builder_experimental import ( create_sql_statement_solid, sql_file_solid, ) diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_templated_sql_tests.py similarity index 99% rename from python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py rename to python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_templated_sql_tests.py index 46769ea30642c..39105362c8c21 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_isolated_templated_sql_tests.py +++ b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_templated_sql_tests.py @@ -7,7 +7,7 @@ execute_pipeline, ) -from dagster.sqlalchemy_kernel.templated import ( +from dagster.sqlalchemy.templated import ( _render_template_string, create_templated_sql_transform_solid, ) diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_mocked_context.py b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_mocked_context.py similarity index 92% rename from python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_mocked_context.py rename to python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_mocked_context.py index b2699287da2f4..f4ab7d7a11442 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_mocked_context.py +++ b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_mocked_context.py @@ -1,5 +1,5 @@ import sqlalchemy -import dagster.sqlalchemy_kernel as dagster_sa +import dagster.sqlalchemy as dagster_sa # disable warnings about malformed inheritance diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_sql_libraries.py b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_sql_libraries.py similarity index 100% rename from python_modules/dagster/dagster/sqlalchemy_kernel/sqlalchemy_kernel_tests/test_sql_libraries.py rename to python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_sql_libraries.py diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/subquery_builder_experimental.py b/python_modules/dagster/dagster/sqlalchemy/subquery_builder_experimental.py similarity index 98% rename from python_modules/dagster/dagster/sqlalchemy_kernel/subquery_builder_experimental.py rename to python_modules/dagster/dagster/sqlalchemy/subquery_builder_experimental.py index fffa4a3a07170..e279e9285fc35 100644 --- a/python_modules/dagster/dagster/sqlalchemy_kernel/subquery_builder_experimental.py +++ b/python_modules/dagster/dagster/sqlalchemy/subquery_builder_experimental.py @@ -10,7 +10,7 @@ types, ) -from dagster.sqlalchemy_kernel import execute_sql_text_on_context +from dagster.sqlalchemy import execute_sql_text_on_context class DagsterSqlExpression: diff --git a/python_modules/dagster/dagster/sqlalchemy_kernel/templated.py b/python_modules/dagster/dagster/sqlalchemy/templated.py similarity index 100% rename from python_modules/dagster/dagster/sqlalchemy_kernel/templated.py rename to python_modules/dagster/dagster/sqlalchemy/templated.py diff --git a/python_modules/dagster/setup.py b/python_modules/dagster/setup.py index b4165207f40dd..92d3089ca9a01 100644 --- a/python_modules/dagster/setup.py +++ b/python_modules/dagster/setup.py @@ -18,7 +18,7 @@ def long_description(): setup( name='dagster', - version='0.2.0.dev1', + version='0.2.0.dev2', author='Elementl', author_email='schrockn@elementl.com', license='Apache-2.0', From 403e300e3fcdf55916fb9ec152f627d54d9259a1 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Wed, 29 Aug 2018 20:58:55 -0700 Subject: [PATCH 039/103] Add docstrings for public classes in definitions.py --- .../core_tests/test_pipeline_execution.py | 18 ++ .../dagster/dagster/core/definitions.py | 226 ++++++++++++++++-- 2 files changed, 227 insertions(+), 17 deletions(-) diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py index 7ef46455c21a3..0a7bf0d890e64 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py @@ -6,6 +6,7 @@ SolidDefinition, check, config, + execute_pipeline, ) from dagster.core.definitions import DependencyStructure @@ -243,6 +244,22 @@ def test_pipeline_execution_graph_diamond(): )) +def test_create_single_solid_pipeline(): + stub_solid = define_stub_solid('stub', [{'a key': 'a value'}]) + single_solid_pipeline = PipelineDefinition.create_single_solid_pipeline( + PipelineDefinition(solids=create_diamond_solids(), dependencies=diamond_deps()), + 'A', + { + 'A': { + 'A_input': stub_solid + }, + }, + ) + + result = execute_pipeline(single_solid_pipeline, config.Environment()) + assert result.success + + # def test_pipeline_execution_graph_diamond_in_memory(): # pipeline = PipelineDefinition(solids=create_diamond_solids(), dependencies=diamond_deps()) # # input_values = {'A_input': [{'A_input': 'input_set'}]} @@ -253,6 +270,7 @@ def test_pipeline_execution_graph_diamond(): # # input_values={'A': input_values}, # )) + def _do_test(do_execute_pipeline_iter): results = list() diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index 0e3e52d84acc2..646f6a5788f74 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -11,6 +11,7 @@ ) from .errors import DagsterInvalidDefinitionError + from .execution_context import ExecutionContext DEFAULT_OUTPUT = 'result' @@ -63,6 +64,16 @@ def __setitem__(self, _key, _value): class PipelineContextDefinition: + '''Pipelines declare the different context types they support, in the form + of PipelineContextDefinitions. For example a pipeline could declare a context + definition for different operating environments: unittest, integration tests, + production and so forth. The use provides context function that returns an + ExecutionContext that is passed to every solid. One can hang resources + (such as db connections) off of that context. Thus the pipeline author + has complete control over how the author of each individual solid within + the pipeline interacts with its operating environment. + ''' + @staticmethod def passthrough_context_definition(context): check.inst_param(context, 'context', ExecutionContext) @@ -72,6 +83,26 @@ def passthrough_context_definition(context): return {'default': context_definition} def __init__(self, *, argument_def_dict, context_fn, description=None): + ''' + Parameters + ---------- + argument_def_dict: str => ArgumentDefinition + Define the arguments expected by the context configuration + + context_fn: callable (pipeline: PipelineDefinition, args: dict str => Any + Returns *or* yields an ExecutionContext. + + If it yields a context, the code after the yield executes after pipeline + completion, just like a python context manager. + + Environment-specific resources should be placed in the "resources" argument + to an execution context. This argument can be *anything* and it is made + avaiable to every solid in the pipeline. A typical pattern is to have this + resources object be a namedtuple, where each property is an object that + manages a particular resource, e.g. aws, a local filesystem manager, etc. + + description: str (optional) + ''' self.argument_def_dict = ArgumentDefinitionDictionary(argument_def_dict) self.context_fn = check.callable_param(context_fn, 'context_fn') self.description = description @@ -98,7 +129,22 @@ def _default_context_fn(_pipeline, args): class DependencyDefinition(namedtuple('_DependencyDefinition', 'solid output description')): + '''Dependency definitions represent an edge in the DAG of solids. This object is + used with a dictionary structure (whose keys represent solid/input where the dependency + comes from) so this object only contains the target dependency information. + ''' + def __new__(cls, solid, output=DEFAULT_OUTPUT, description=None): + ''' + Parameters: + solid: str + The name of the solid that is the target of the dependency. + This is the solid where the value passed between the solids + comes from. + output: str (optional) defaults to 'result' + The name of the output that is the target of the dependency. + description: str (optional) + ''' return super(DependencyDefinition, cls).__new__( cls, check.str_param(solid, 'solid'), @@ -201,8 +247,57 @@ def _build_named_dict(things): class PipelineDefinition: + ''' A instance of a PipelineDefinition represents a pipeline in dagster. + + A pipeline is comprised of: + + - Solids. Each solid represents a functional unit of data computation. + - Context Definitions. Pipelines can be designed to execute in a number of + different operating environments (e.g. prod, dev, unittest) that require + different configuration and setup. A context definition defines how a context + (of type ExecutionContext) is created and what configuration is necessary to create it. + - Dependencies. Solids within a pipeline are arranged as a DAG (directed, acyclic graph). + Dependencies determine how the values produced by solids flow through the DAG. + ''' + @staticmethod def create_single_solid_pipeline(pipeline, solid_name, injected_solids=None): + ''' + Return a new pipeline which is a single solid of the passed-in pipeline. + + Frequently (especially in test contexts) one wants to isolate a single solid for + independent testing. + + See Also + -------- + PipelineDefinition.create_sub_pipeline + + Parameters + ---------- + pipeline: PipelineDefinition + solid_name: str + injected_solids: + Two dimensional dictionary. (optional) + solid_name (str) => index_name (str) => SolidDefinition + + When you create a subpipeline, you possible left with solids within that pipeline + who have unmet dependencies. To fulfill these dependencies new solids must be + provided. + + returns a PipelineDefinition + + Examples + -------- + new_pipeline = PipelineDefinition.create_single_solid_pipeline( + existing_pipeline, + 'A', # existing_pipeline + { + 'A': { + 'A_input': new_solid_instance, + }, + }, + ) + ''' return PipelineDefinition.create_sub_pipeline( pipeline, [solid_name], @@ -212,6 +307,30 @@ def create_single_solid_pipeline(pipeline, solid_name, injected_solids=None): @staticmethod def create_sub_pipeline(pipeline, from_solids, through_solids, injected_solids=None): + ''' + Return a new pipeline which is a subset of the passed-in pipeline. + + In addition to making sub-pipelines out of a single solid, one can also create a + pipeline using an arbitrary subset of the dag. In this case, we start "from" a set + of solids and then proceed forward through the dependency graph until all the + "through" solids are reached. + + See Also + -------- + PipelineDefinition.create_single_solid_pipeline + + Parameters + ---------- + pipeline: PipelineDefinintion + from_solids: list of strs + through_solids: list of strs + injected_solids: + Two dimensional dictionary. (optional) + solid_name (str) => index_name (str) => SolidDefinition + + returns a PipelineDefinition + ''' + # FIXME: fix circular reference from .graph import ExecutionGraph check.inst_param(pipeline, 'pipeline', PipelineDefinition) check.list_param(from_solids, 'from_solids', of_type=str) @@ -279,6 +398,15 @@ def __validate_dependency_structure(self, name, solids, dependency_structure): def __init__( self, solids, name=None, description=None, context_definitions=None, dependencies=None ): + ''' + Parameters + ---------- + solids: list of SolidDefinition + name: string (optional) + description: string (optional) + context_definitions: dictionary str => PipelineContextDefinition (optional) + dependencies: 2D dictionary str => str => DependencyDefinition (optional) + ''' self.description = check.opt_str_param(description, 'description') self.name = check.opt_str_param(name, 'name') @@ -316,6 +444,7 @@ def __init__( @property def solids(self): + '''Return the solids in the pipeline.''' return list(self._solid_dict.values()) def has_solid(self, name): @@ -323,6 +452,7 @@ def has_solid(self, name): return name in self._solid_dict def solid_named(self, name): + '''Return the solid named "name". Throws if it does not exist.''' check.str_param(name, 'name') return self._solid_dict[name] @@ -346,18 +476,15 @@ def __init__(self, name, expectation_fn, description=None): class InputDefinition: - ''' - An InputDefinition instances represents an argument to a transform defined within a solid. - - - name: Name of input - - - sources: A list of possible sources for the input. For example, an input which is passed - to the transform as a pandas dataframe could have any number of different source types - (CSV, Parquet etc). Some inputs have zero sources, and can only be created by - execute a dependant solid. - - - depends_on: (Optional). This input depends on another solid in the context of a - a pipeline. + '''An InputDefinition instance represents an argument to a transform defined within a solid. + Inputs are values within the dagster type system that are created from previous solids. + + Parameters + ---------- + name: str + dagster_type: DagsterType (optional) defaults to types.Any + expectations: list of ExpectationDefinition (optional) + description: str (optional) ''' def __init__(self, name, dagster_type=None, expectations=None, description=None): @@ -374,7 +501,19 @@ def __init__(self, name, dagster_type=None, expectations=None, description=None) class OutputDefinition: - # runtime type info + '''An OutputDefinition represents an output from a solid. Solids can have multiple + outputs. In those cases the outputs must be named. Frequently solids have only one + output, and so the user can construct a single OutputDefinition that will have + the default name of "result". + + Parameters + ---------- + name: str (optional) defaults to "result" + dagster_type: DagsterType (optional) defaults to types.Any + expectations: list of ExpectationDefinition (optional) + description: str (optional) + ''' + def __init__(self, name=None, dagster_type=None, expectations=None, description=None): self.name = check.opt_str_param(name, 'name', DEFAULT_OUTPUT) @@ -431,6 +570,15 @@ def __eq__(self, other): class Result(namedtuple('_Result', 'value output_name')): + '''A solid transform function return a stream of Result objects. + An implementator of a SolidDefinition must provide a transform that + yields objects of this type. + + Parameters + ---------- + value: Any + output_name: str (optional) defaults to "result"''' + def __new__(cls, value, output_name=DEFAULT_OUTPUT): return super(Result, cls).__new__( cls, @@ -440,11 +588,33 @@ def __new__(cls, value, output_name=DEFAULT_OUTPUT): class ConfigDefinition: + '''Solids have config, which determine how they interact with the external world. + Example configs would be file paths, database table names, and so forth. + + Parameters: + ---------- + argument_def_dict: str => ArgumentDefinition''' + def __init__(self, argument_def_dict): self.argument_def_dict = ArgumentDefinitionDictionary(argument_def_dict) class SolidDefinition: + '''A solid is a node of computation within a pipeline. + + Parameters: + ---------- + name: str + inputs: list of InputDefinitions + transform_fn: callable with sig ( + context: ExecutionContext, + inputs: str => Any, + config_dict: str => Any) : Iterable + outputs: list of OutputDefinitions + config_def: ConfigDefinition (optional) + description: str (optional) + ''' + def __init__(self, *, name, inputs, transform_fn, outputs, config_def=None, description=None): self.name = check_valid_name(name) self.input_defs = check.list_param(inputs, 'inputs', InputDefinition) @@ -473,9 +643,11 @@ def __init__(self, *, name, inputs, transform_fn, outputs, config_def=None, desc self._output_dict = _build_named_dict(outputs) @staticmethod - def single_output_transform( - name, inputs, transform_fn, output, config_def=None, description=None - ): + def single_output_transform(name, inputs, transform_fn, output, description=None): + '''It is commmon to want a Solid that has only inputs, a single output (with the default + name), and no config. So this is a helper function to do that. This transform function + must return the naked return value (as opposed to a Result object)''' + def _new_transform_fn(context, inputs, _config_dict): value = transform_fn(context, inputs) yield Result(output_name=DEFAULT_OUTPUT, value=value) @@ -485,7 +657,6 @@ def _new_transform_fn(context, inputs, _config_dict): inputs=inputs, transform_fn=_new_transform_fn, outputs=[output], - config_def=config_def, description=description, ) @@ -522,9 +693,30 @@ class __ArgumentValueSentinel: class ArgumentDefinition: + '''Definition of an argument passed through the config system. Used in a few different + contexts: to configure a context, to configure a solid, and more to come. + + We have decided to allow arguments to be explictly made *optional* and separate that + concept from the nullability of the type. That means one could have a *required* argument + that is nullable, because sometimes an argument set to null has a distinct semantic meaning + from the lack of an argument. Optional arguments can have default values. Required arguments + cannot. + ''' + def __init__( self, dagster_type, default_value=NO_DEFAULT_PROVIDED, is_optional=False, description=None ): + ''' + Parameters + --------- + dagster_type: DagsterType + The type of the argument. + default_value: Any (optional, defaults to a sentinel value) + Default value of argument. Can only be provided if argument is optional + is_optional: bool (optional, defaults to false) + Optional. Can the solid execute with the argument even set. + description: str (optional) + ''' if not is_optional: check.param_invariant( default_value == NO_DEFAULT_PROVIDED, From 8f86d5c781f361d15b2b10b18cc7de38618eca6e Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Thu, 30 Aug 2018 10:37:15 -0700 Subject: [PATCH 040/103] Eliminate circular deps Eliminatin circular deps almost always yields some nice architectural wins and this is no exception. DagsterEnv (which began as InputManager) is gone! All rejoice. Things are much cleaner now. --- .../dagster/dagster/cli/pipeline.py | 2 +- .../dagster/dagster/core/compute_nodes.py | 77 ++++-- .../core/core_tests/test_compute_nodes.py | 20 +- .../core_tests/test_pipeline_execution.py | 7 +- .../dagster/dagster/core/definitions.py | 244 ++++++++++++++++- .../dagster/dagster/core/execution.py | 84 +++--- .../dagster/dagster/core/execution_context.py | 2 +- python_modules/dagster/dagster/core/graph.py | 252 ------------------ 8 files changed, 348 insertions(+), 340 deletions(-) delete mode 100644 python_modules/dagster/dagster/core/graph.py diff --git a/python_modules/dagster/dagster/cli/pipeline.py b/python_modules/dagster/dagster/cli/pipeline.py index 01a0812b97b87..cd3fd95f8e83c 100644 --- a/python_modules/dagster/dagster/cli/pipeline.py +++ b/python_modules/dagster/dagster/cli/pipeline.py @@ -8,9 +8,9 @@ import dagster from dagster import check +from dagster.core.definitions import ExecutionGraph from dagster.core.execution import execute_pipeline_iterator from dagster.core.errors import DagsterExecutionFailureReason -from dagster.core.graph import ExecutionGraph from dagster.graphviz import build_graphviz_graph from dagster.utils.indenting_printer import IndentingPrinter diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index c3f93c410c022..d7d6589222397 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -6,7 +6,10 @@ import toposort -from dagster import check +from dagster import ( + check, + config, +) from dagster.utils.indenting_printer import IndentingPrinter from dagster.utils.logging import ( @@ -20,6 +23,7 @@ from .argument_handling import validate_args from .definitions import ( + ExecutionGraph, ExpectationDefinition, InputDefinition, OutputDefinition, @@ -36,11 +40,23 @@ DagsterUserCodeExecutionError, ) -from .graph import ExecutionGraph - from .types import DagsterType +class ComputeNodeExecutionInfo(namedtuple('_ComputeNodeExecutionInfo', 'context execution_graph environment')): + def __new__(cls, context, execution_graph, environment): + return super(ComputeNodeExecutionInfo, cls).__new__( + cls, + check.inst_param(context, 'context', ExecutionContext), + check.inst_param(execution_graph, 'execution_graph', ExecutionGraph), + check.inst_param(environment, 'environment', config.Environment), + ) + + @property + def pipeline(self): + return self.execution_graph.pipeline + + class ComputeNodeOutputHandle(namedtuple('_ComputeNodeOutputHandle', 'compute_node output_name')): def __new__(cls, compute_node, output_name): return super(ComputeNodeOutputHandle, cls).__new__( @@ -477,16 +493,25 @@ def __setitem__(self, key, val): check.inst_param(val, 'val', ComputeNodeOutputHandle) return dict.__setitem__(self, key, val) +def validate_config_dict(execution_info, solid): + check.inst_param(execution_info, 'execution_info', ComputeNodeExecutionInfo) + check.inst_param(solid, 'solid', SolidDefinition) -def check_dagster_env(env): - # TODO: remove circular - import dagster.core.execution - check.inst_param(env, 'env', dagster.core.execution.DagsterEnv) + name = solid.name + solid_configs = execution_info.environment.solids + config_dict = solid_configs[name].config_dict if name in solid_configs else {} + + return validate_args( + solid.config_def.argument_def_dict, + config_dict, + 'config for solid {solid_name}'.format(solid_name=name), + ) -def create_compute_node_graph_from_env(execution_graph, env): - check.inst_param(execution_graph, 'execution_graph', ExecutionGraph) - check_dagster_env(env) +def create_compute_node_graph(execution_info): + check.inst_param(execution_info, 'execution_info', ComputeNodeExecutionInfo) + + execution_graph = execution_info.execution_graph dependency_structure = execution_graph.dependency_structure @@ -508,7 +533,12 @@ def create_compute_node_graph_from_env(execution_graph, env): solid_output_handle = dependency_structure.get_dep(input_handle) prev_cn_output_handle = cn_output_node_map[solid_output_handle] - subgraph = create_subgraph_for_input(env, topo_solid, prev_cn_output_handle, input_def) + subgraph = create_subgraph_for_input( + execution_info, + topo_solid, + prev_cn_output_handle, + input_def, + ) compute_nodes.extend(subgraph.nodes) cn_inputs.append( @@ -519,11 +549,7 @@ def create_compute_node_graph_from_env(execution_graph, env): ) ) - validated_config_args = validate_args( - topo_solid.config_def.argument_def_dict, - env.config_dict_for_solid(topo_solid.name), - 'config for solid {solid_name}'.format(solid_name=topo_solid.name), - ) + validated_config_args = validate_config_dict(execution_info, topo_solid) solid_transform_cn = create_compute_node_from_solid_transform( topo_solid, @@ -534,7 +560,12 @@ def create_compute_node_graph_from_env(execution_graph, env): compute_nodes.append(solid_transform_cn) for output_def in topo_solid.output_defs: - subgraph = create_subgraph_for_output(env, topo_solid, solid_transform_cn, output_def) + subgraph = create_subgraph_for_output( + execution_info, + topo_solid, + solid_transform_cn, + output_def, + ) compute_nodes.extend(subgraph.nodes) output_handle = topo_solid.output_handle(output_def.name) @@ -558,13 +589,13 @@ def _create_compute_node_graph(compute_nodes): return ComputeNodeGraph(cn_dict, deps) -def create_subgraph_for_input(env, solid, prev_cn_output_handle, input_def): - check_dagster_env(env) +def create_subgraph_for_input(execution_info, solid, prev_cn_output_handle, input_def): + check.inst_param(execution_info, 'execution_info', ComputeNodeExecutionInfo) check.inst_param(solid, 'solid', SolidDefinition) check.inst_param(prev_cn_output_handle, 'prev_cn_output_handle', ComputeNodeOutputHandle) check.inst_param(input_def, 'input_def', InputDefinition) - if env.evaluate_expectations and input_def.expectations: + if execution_info.environment.expectations.evaluate and input_def.expectations: return create_expectations_cn_graph( solid, input_def, @@ -578,13 +609,13 @@ def create_subgraph_for_input(env, solid, prev_cn_output_handle, input_def): ) -def create_subgraph_for_output(env, solid, solid_transform_cn, output_def): - check_dagster_env(env) +def create_subgraph_for_output(execution_info, solid, solid_transform_cn, output_def): + check.inst_param(execution_info, 'execution_info', ComputeNodeExecutionInfo) check.inst_param(solid, 'solid', SolidDefinition) check.inst_param(solid_transform_cn, 'solid_transform_cn', ComputeNode) check.inst_param(output_def, 'output_def', OutputDefinition) - if env.evaluate_expectations and output_def.expectations: + if execution_info.environment.expectations.evaluate and output_def.expectations: return create_expectations_cn_graph( solid, output_def, diff --git a/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py b/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py index 3433d292cfb14..4b4e7dccdc6c1 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py +++ b/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py @@ -1,18 +1,19 @@ from dagster import ( ExecutionContext, + OutputDefinition, PipelineContextDefinition, PipelineDefinition, config, solid, - OutputDefinition, ) from dagster.core.execution import ( - DagsterEnv, - ExecutionGraph, + ComputeNodeExecutionInfo, ) -from dagster.core.compute_nodes import create_compute_node_graph_from_env +from dagster.core.definitions import ExecutionGraph + +from dagster.core.compute_nodes import create_compute_node_graph def silencing_default_context(): @@ -41,10 +42,13 @@ def test_compute_noop_node(): environment = config.Environment() - env = DagsterEnv(ExecutionGraph.from_pipeline(pipeline), environment) - compute_node_graph = create_compute_node_graph_from_env( - ExecutionGraph.from_pipeline(pipeline), - env, + execution_graph = ExecutionGraph.from_pipeline(pipeline) + compute_node_graph = create_compute_node_graph( + ComputeNodeExecutionInfo( + ExecutionContext(), + execution_graph, + environment, + ), ) assert len(compute_node_graph.nodes) == 1 diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py index 0a7bf0d890e64..e89b783252388 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py @@ -9,11 +9,10 @@ execute_pipeline, ) -from dagster.core.definitions import DependencyStructure - -from dagster.core.graph import ( - _create_adjacency_lists, +from dagster.core.definitions import ( + DependencyStructure, ExecutionGraph, + _create_adjacency_lists, ) from dagster.core.execution import ( diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index 646f6a5788f74..144da41dab767 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -2,6 +2,7 @@ import copy import keyword import re +from toposort import toposort_flatten from dagster import check from dagster.core import types @@ -110,10 +111,8 @@ def __init__(self, *, argument_def_dict, context_fn, description=None): def _default_pipeline_context_definitions(): def _default_context_fn(_pipeline, args): - import dagster.core.execution - log_level = level_from_string(args['log_level']) - context = dagster.core.execution.ExecutionContext( + context = ExecutionContext( loggers=[define_colored_console_logger('dagster', level=log_level)] ) return context @@ -331,7 +330,6 @@ def create_sub_pipeline(pipeline, from_solids, through_solids, injected_solids=N returns a PipelineDefinition ''' # FIXME: fix circular reference - from .graph import ExecutionGraph check.inst_param(pipeline, 'pipeline', PipelineDefinition) check.list_param(from_solids, 'from_solids', of_type=str) check.list_param(through_solids, 'through_solids', of_type=str) @@ -732,3 +730,241 @@ def __init__( @property def default_provided(self): return self.default_value != NO_DEFAULT_PROVIDED + + +def _create_adjacency_lists(solids, dep_structure): + check.list_param(solids, 'solids', of_type=SolidDefinition) + check.inst_param(dep_structure, 'dep_structure', DependencyStructure) + + visit_dict = {s.name: False for s in solids} + forward_edges = {s.name: set() for s in solids} + backward_edges = {s.name: set() for s in solids} + + def visit(solid_name): + if visit_dict[solid_name]: + return + + visit_dict[solid_name] = True + + for output_handle in dep_structure.deps_of_solid(solid_name): + forward_node = output_handle.solid.name + backward_node = solid_name + if forward_node in forward_edges: + forward_edges[forward_node].add(backward_node) + backward_edges[backward_node].add(forward_node) + visit(forward_node) + + for s in solids: + visit(s.name) + + return (forward_edges, backward_edges) + + +def _dependency_structure_to_dep_dict(dependency_structure): + dep_dict = defaultdict(dict) + for input_handle, output_handle in dependency_structure.items(): + dep_dict[input_handle.solid.name][input_handle.input_def.name] = DependencyDefinition( + solid=output_handle.solid.name, + output=output_handle.output_def.name, + ) + return dep_dict + + +class ExecutionGraph: + @staticmethod + def from_pipeline(pipeline): + check.inst_param(pipeline, 'pipeline', PipelineDefinition) + return ExecutionGraph(pipeline, pipeline.solids, pipeline.dependency_structure) + + @staticmethod + def from_pipeline_subset(pipeline, from_solids, through_solids, injected_solids): + check.inst_param(pipeline, 'pipeline', PipelineDefinition) + check.list_param(from_solids, 'from_solids', of_type=str) + check.list_param(through_solids, 'through_solids', of_type=str) + graph = ExecutionGraph.from_pipeline(pipeline) + return _create_subgraph(graph, from_solids, through_solids).augment(injected_solids) + + def to_pipeline(self): + return PipelineDefinition( + solids=self.solids, + dependencies=_dependency_structure_to_dep_dict(self.dependency_structure), + context_definitions=self.pipeline.context_definitions + ) + + def augment(self, injected_solids): + + new_deps = defaultdict(dict) + new_solids = [] + + for from_solid_name, targets_by_input in injected_solids.items(): + for from_input_name, target_solid in targets_by_input.items(): + new_solids.append(target_solid) + new_deps[from_solid_name][from_input_name] = DependencyDefinition( + solid=target_solid.name + ) + + check.list_param(new_solids, 'new_solids', of_type=SolidDefinition) + + solids = self.solids + new_solids + + solid_dict = _build_named_dict(solids) + + handle_dict = InputToOutputHandleDict() + for input_handle, output_handle in self.dependency_structure.items(): + handle_dict[input_handle] = output_handle + + for input_handle, output_handle in create_handle_dict(solid_dict, new_deps).items(): + handle_dict[input_handle] = output_handle + + return ExecutionGraph(self.pipeline, solids, DependencyStructure(handle_dict)) + + def __init__(self, pipeline, solids, dependency_structure): + self.pipeline = pipeline + solids = check.list_param(solids, 'solids', of_type=SolidDefinition) + + self.dependency_structure = check.inst_param( + dependency_structure, 'dependency_structure', DependencyStructure + ) + + self._solid_dict = {solid.name: solid for solid in solids} + + for input_handle in dependency_structure.input_handles(): + check.invariant(input_handle.solid.name in self._solid_dict) + + solid_names = set([solid.name for solid in solids]) + check.invariant(len(solid_names) == len(solids), 'must have unique names') + + self.forward_edges, self.backward_edges = _create_adjacency_lists( + solids, self.dependency_structure + ) + self.topological_order = toposort_flatten(self.backward_edges, sort=True) + + self._transitive_deps = {} + + @property + def topological_solids(self): + return [self._solid_dict[name] for name in self.topological_order] + + @property + def solids(self): + return list(self._solid_dict.values()) + + def solid_named(self, name): + check.str_param(name, 'name') + return self._solid_dict[name] + + def transitive_dependencies_of(self, solid_name): + check.str_param(solid_name, 'solid_name') + + if solid_name in self._transitive_deps: + return self._transitive_deps[solid_name] + + trans_deps = set() + solid = self._solid_dict[solid_name] + for input_def in solid.input_defs: + input_handle = solid.input_handle(input_def.name) + if self.dependency_structure.has_dep(input_handle): + output_handle = self.dependency_structure.get_dep(input_handle) + trans_deps.add(output_handle.solid.name) + trans_deps.union(self.transitive_dependencies_of(output_handle.solid.name)) + + self._transitive_deps[solid_name] = trans_deps + return self._transitive_deps[solid_name] + + def _check_solid_name(self, solid_name): + check.str_param(solid_name, 'output_name') + check.param_invariant( + solid_name in self._solid_dict, 'output_name', + f'Solid {solid_name} must exist in {list(self._solid_dict.keys())}' + ) + + def create_execution_subgraph(self, from_solids, to_solids): + check.list_param(from_solids, 'from_solids', of_type=str) + check.list_param(to_solids, 'to_solids', of_type=str) + + involved_solid_set = self._compute_involved_solid_set(from_solids, to_solids) + + involved_solids = [self._solid_dict[name] for name in involved_solid_set] + + handle_dict = InputToOutputHandleDict() + + for solid in involved_solids: + for input_def in solid.input_defs: + input_handle = solid.input_handle(input_def.name) + if self.dependency_structure.has_dep(input_handle): + handle_dict[input_handle] = self.dependency_structure.get_dep(input_handle) + + return ExecutionGraph(self.pipeline, involved_solids, DependencyStructure(handle_dict)) + + def _compute_involved_solid_set(self, from_solids, to_solids): + from_solid_set = set(from_solids) + involved_solid_set = from_solid_set + + def visit(solid): + if solid.name in involved_solid_set: + return + + involved_solid_set.add(solid.name) + + for input_def in solid.input_defs: + input_handle = solid.input_handle(input_def.name) + if not self.dependency_structure.has_dep(input_handle): + continue + + output_handle = self.dependency_structure.get_dep(input_handle) + + next_solid = output_handle.solid.name + if next_solid in from_solid_set: + continue + + visit(self._solid_dict[next_solid]) + + for to_solid in to_solids: + visit(self._solid_dict[to_solid]) + + return involved_solid_set + + +def _build_named_dict(things): + ddict = {} + for thing in things: + ddict[thing.name] = thing + return ddict + + +def _all_depended_on_solids(execution_graph): + check.inst_param(execution_graph, 'execution_graph', ExecutionGraph) + + dependency_structure = execution_graph.dependency_structure + + for solid in execution_graph.solids: + for input_def in solid.input_defs: + input_handle = solid.input_handle(input_def.name) + if dependency_structure.has_dep(input_handle): + output_handle = dependency_structure.get_dep(input_handle) + yield execution_graph.solid_named(output_handle.solid.name) + + +def _all_sink_solids(execution_graph): + check.inst_param(execution_graph, 'execution_graph', ExecutionGraph) + all_names = set([solid.name for solid in execution_graph.solids]) + all_depended_on_names = set([solid.name for solid in _all_depended_on_solids(execution_graph)]) + return all_names.difference(all_depended_on_names) + + +def _create_subgraph(execution_graph, from_solids, through_solids): + check.inst_param(execution_graph, 'execution_graph', ExecutionGraph) + check.opt_list_param(from_solids, 'from_solids', of_type=str) + check.opt_list_param(through_solids, 'through_solids', of_type=str) + + if not through_solids: + through_solids = list(_all_sink_solids(execution_graph)) + + if not from_solids: + all_deps = set() + for through_solid in through_solids: + all_deps.union(execution_graph.transitive_dependencies_of(through_solid)) + + from_solids = list(all_deps) + + return execution_graph.create_execution_subgraph(from_solids, through_solids) diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index f035a262b4dd5..f919fd30b7bc2 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -27,6 +27,7 @@ from .definitions import ( DEFAULT_OUTPUT, + ExecutionGraph, PipelineDefinition, SolidDefinition, ) @@ -36,16 +37,15 @@ from .argument_handling import validate_args from .compute_nodes import ( - ComputeNodeTag, + ComputeNodeExecutionInfo, ComputeNodeResult, - create_compute_node_graph_from_env, + ComputeNodeTag, + create_compute_node_graph, execute_compute_nodes, ) from .execution_context import ExecutionContext -from .graph import ExecutionGraph - class PipelineExecutionResult: def __init__( @@ -187,40 +187,26 @@ def _validate_environment(environment, pipeline): ) -class DagsterEnv: - def __init__(self, execution_graph, environment): - # This is not necessarily the best spot for these calls - pipeline = execution_graph.pipeline - _validate_environment(environment, pipeline) - self.pipeline = check.inst_param(pipeline, 'pipeline', PipelineDefinition) - self.environment = check.inst_param(environment, 'environment', config.Environment) - - @contextmanager - def yield_context(self): - context_name = self.environment.context.name - context_definition = self.pipeline.context_definitions[context_name] - - args_to_pass = validate_args( - self.pipeline.context_definitions[context_name].argument_def_dict, - self.environment.context.args, 'pipeline {pipeline_name} context {context_name}'.format( - pipeline_name=self.pipeline.name, - context_name=context_name, - ) - ) +@contextmanager +def yield_context(pipeline, environment): + check.inst_param(pipeline, 'pipeline', PipelineDefinition) + check.inst_param(environment, 'environment', config.Environment) - thing = context_definition.context_fn(self.pipeline, args_to_pass) - return _wrap_in_yield(thing) + _validate_environment(environment, pipeline) - @property - def evaluate_expectations(self): - return self.environment.expectations.evaluate + context_name = environment.context.name - def config_dict_for_solid(self, name): - check.str_param(name, 'name') - if name not in self.environment.solids: - return {} - else: - return self.environment.solids[name].config_dict + context_definition = pipeline.context_definitions[context_name] + + args_to_pass = validate_args( + pipeline.context_definitions[context_name].argument_def_dict, + environment.context.args, 'pipeline {pipeline_name} context {context_name}'.format( + pipeline_name=pipeline.name, + context_name=context_name, + ) + ) + thing = context_definition.context_fn(pipeline, args_to_pass) + return _wrap_in_yield(thing) def execute_pipeline_iterator(pipeline, environment): @@ -228,17 +214,22 @@ def execute_pipeline_iterator(pipeline, environment): check.inst_param(environment, 'enviroment', config.Environment) execution_graph = ExecutionGraph.from_pipeline(pipeline) - env = DagsterEnv(execution_graph, environment) - with env.yield_context() as context: - return _execute_graph_iterator(context, execution_graph, env) + with yield_context(pipeline, environment) as context: + return _execute_graph_iterator(context, execution_graph, environment) -def _execute_graph_iterator(context, execution_graph, env): +def _execute_graph_iterator(context, execution_graph, environment): check.inst_param(context, 'context', ExecutionContext) check.inst_param(execution_graph, 'execution_graph', ExecutionGraph) - check.inst_param(env, 'env', DagsterEnv) + check.inst_param(environment, 'environent', config.Environment) - cn_graph = create_compute_node_graph_from_env(execution_graph, env) + cn_graph = create_compute_node_graph( + ComputeNodeExecutionInfo( + context, + execution_graph, + environment, + ), + ) cn_nodes = list(cn_graph.topological_nodes()) @@ -284,23 +275,22 @@ def execute_pipeline( check.inst_param(pipeline, 'pipeline', PipelineDefinition) check.inst_param(environment, 'environment', config.Environment) execution_graph = ExecutionGraph.from_pipeline(pipeline) - env = DagsterEnv(execution_graph, environment) - return _execute_graph(execution_graph, env, throw_on_error) + return _execute_graph(execution_graph, environment, throw_on_error) def _execute_graph( execution_graph, - env, + environment, throw_on_error=True, ): check.inst_param(execution_graph, 'execution_graph', ExecutionGraph) - check.inst_param(env, 'env', DagsterEnv) + check.inst_param(environment, 'environment', config.Environment) check.bool_param(throw_on_error, 'throw_on_error') results = [] - with env.yield_context() as context: + with yield_context(execution_graph.pipeline, environment) as context: with context.value('pipeline', execution_graph.pipeline.name or '<>'): - for result in _execute_graph_iterator(context, execution_graph, env): + for result in _execute_graph_iterator(context, execution_graph, environment): if throw_on_error: if not result.success: _do_throw_on_error(result) diff --git a/python_modules/dagster/dagster/core/execution_context.py b/python_modules/dagster/dagster/core/execution_context.py index ebae13302bf53..3c57d9bf2cf04 100644 --- a/python_modules/dagster/dagster/core/execution_context.py +++ b/python_modules/dagster/dagster/core/execution_context.py @@ -147,4 +147,4 @@ def metrics_covering_context(self, needle_dict): def metrics_matching_context(self, needle_dict): for metric in self._metrics: if needle_dict == metric.context_dict: - yield metric + yield metric \ No newline at end of file diff --git a/python_modules/dagster/dagster/core/graph.py b/python_modules/dagster/dagster/core/graph.py deleted file mode 100644 index 5bf92febf5110..0000000000000 --- a/python_modules/dagster/dagster/core/graph.py +++ /dev/null @@ -1,252 +0,0 @@ -from collections import defaultdict -from toposort import toposort_flatten - -from dagster import check - -from .definitions import ( - DependencyDefinition, - DependencyStructure, - InputToOutputHandleDict, - PipelineDefinition, - SolidDefinition, - _build_named_dict, - create_handle_dict, -) - - -def _create_adjacency_lists(solids, dep_structure): - check.list_param(solids, 'solids', of_type=SolidDefinition) - check.inst_param(dep_structure, 'dep_structure', DependencyStructure) - - visit_dict = {s.name: False for s in solids} - forward_edges = {s.name: set() for s in solids} - backward_edges = {s.name: set() for s in solids} - - def visit(solid_name): - if visit_dict[solid_name]: - return - - visit_dict[solid_name] = True - - for output_handle in dep_structure.deps_of_solid(solid_name): - forward_node = output_handle.solid.name - backward_node = solid_name - if forward_node in forward_edges: - forward_edges[forward_node].add(backward_node) - backward_edges[backward_node].add(forward_node) - visit(forward_node) - - for s in solids: - visit(s.name) - - return (forward_edges, backward_edges) - - -def _dependency_structure_to_dep_dict(dependency_structure): - dep_dict = defaultdict(dict) - for input_handle, output_handle in dependency_structure.items(): - dep_dict[input_handle.solid.name][input_handle.input_def.name] = DependencyDefinition( - solid=output_handle.solid.name, - output=output_handle.output_def.name, - ) - return dep_dict - - -class ExecutionGraph: - @staticmethod - def from_pipeline(pipeline): - check.inst_param(pipeline, 'pipeline', PipelineDefinition) - return ExecutionGraph(pipeline, pipeline.solids, pipeline.dependency_structure) - - @staticmethod - def from_pipeline_subset(pipeline, from_solids, through_solids, injected_solids): - check.inst_param(pipeline, 'pipeline', PipelineDefinition) - check.list_param(from_solids, 'from_solids', of_type=str) - check.list_param(through_solids, 'through_solids', of_type=str) - graph = ExecutionGraph.from_pipeline(pipeline) - return _create_subgraph(graph, from_solids, through_solids).augment(injected_solids) - - def to_pipeline(self): - return PipelineDefinition( - solids=self.solids, - dependencies=_dependency_structure_to_dep_dict(self.dependency_structure), - context_definitions=self.pipeline.context_definitions - ) - - def augment(self, injected_solids): - - new_deps = defaultdict(dict) - new_solids = [] - - for from_solid_name, targets_by_input in injected_solids.items(): - for from_input_name, target_solid in targets_by_input.items(): - new_solids.append(target_solid) - new_deps[from_solid_name][from_input_name] = DependencyDefinition( - solid=target_solid.name - ) - - check.list_param(new_solids, 'new_solids', of_type=SolidDefinition) - - solids = self.solids + new_solids - - solid_dict = _build_named_dict(solids) - - handle_dict = InputToOutputHandleDict() - for input_handle, output_handle in self.dependency_structure.items(): - handle_dict[input_handle] = output_handle - - for input_handle, output_handle in create_handle_dict(solid_dict, new_deps).items(): - handle_dict[input_handle] = output_handle - - return ExecutionGraph(self.pipeline, solids, DependencyStructure(handle_dict)) - - def __init__(self, pipeline, solids, dependency_structure): - self.pipeline = pipeline - solids = check.list_param(solids, 'solids', of_type=SolidDefinition) - - self.dependency_structure = check.inst_param( - dependency_structure, 'dependency_structure', DependencyStructure - ) - - self._solid_dict = {solid.name: solid for solid in solids} - - for input_handle in dependency_structure.input_handles(): - check.invariant(input_handle.solid.name in self._solid_dict) - - solid_names = set([solid.name for solid in solids]) - check.invariant(len(solid_names) == len(solids), 'must have unique names') - - self.forward_edges, self.backward_edges = _create_adjacency_lists( - solids, self.dependency_structure - ) - self.topological_order = toposort_flatten(self.backward_edges, sort=True) - - self._transitive_deps = {} - - @property - def topological_solids(self): - return [self._solid_dict[name] for name in self.topological_order] - - @property - def solids(self): - return list(self._solid_dict.values()) - - def solid_named(self, name): - check.str_param(name, 'name') - return self._solid_dict[name] - - def transitive_dependencies_of(self, solid_name): - check.str_param(solid_name, 'solid_name') - - if solid_name in self._transitive_deps: - return self._transitive_deps[solid_name] - - trans_deps = set() - solid = self._solid_dict[solid_name] - for input_def in solid.input_defs: - input_handle = solid.input_handle(input_def.name) - if self.dependency_structure.has_dep(input_handle): - output_handle = self.dependency_structure.get_dep(input_handle) - trans_deps.add(output_handle.solid.name) - trans_deps.union(self.transitive_dependencies_of(output_handle.solid.name)) - - self._transitive_deps[solid_name] = trans_deps - return self._transitive_deps[solid_name] - - def _check_solid_name(self, solid_name): - check.str_param(solid_name, 'output_name') - check.param_invariant( - solid_name in self._solid_dict, 'output_name', - f'Solid {solid_name} must exist in {list(self._solid_dict.keys())}' - ) - - def create_execution_subgraph(self, from_solids, to_solids): - check.list_param(from_solids, 'from_solids', of_type=str) - check.list_param(to_solids, 'to_solids', of_type=str) - - involved_solid_set = self._compute_involved_solid_set(from_solids, to_solids) - - involved_solids = [self._solid_dict[name] for name in involved_solid_set] - - handle_dict = InputToOutputHandleDict() - - for solid in involved_solids: - for input_def in solid.input_defs: - input_handle = solid.input_handle(input_def.name) - if self.dependency_structure.has_dep(input_handle): - handle_dict[input_handle] = self.dependency_structure.get_dep(input_handle) - - return ExecutionGraph(self.pipeline, involved_solids, DependencyStructure(handle_dict)) - - def _compute_involved_solid_set(self, from_solids, to_solids): - from_solid_set = set(from_solids) - involved_solid_set = from_solid_set - - def visit(solid): - if solid.name in involved_solid_set: - return - - involved_solid_set.add(solid.name) - - for input_def in solid.input_defs: - input_handle = solid.input_handle(input_def.name) - if not self.dependency_structure.has_dep(input_handle): - continue - - output_handle = self.dependency_structure.get_dep(input_handle) - - next_solid = output_handle.solid.name - if next_solid in from_solid_set: - continue - - visit(self._solid_dict[next_solid]) - - for to_solid in to_solids: - visit(self._solid_dict[to_solid]) - - return involved_solid_set - - -def _build_named_dict(things): - ddict = {} - for thing in things: - ddict[thing.name] = thing - return ddict - - -def _all_depended_on_solids(execution_graph): - check.inst_param(execution_graph, 'execution_graph', ExecutionGraph) - - dependency_structure = execution_graph.dependency_structure - - for solid in execution_graph.solids: - for input_def in solid.input_defs: - input_handle = solid.input_handle(input_def.name) - if dependency_structure.has_dep(input_handle): - output_handle = dependency_structure.get_dep(input_handle) - yield execution_graph.solid_named(output_handle.solid.name) - - -def _all_sink_solids(execution_graph): - check.inst_param(execution_graph, 'execution_graph', ExecutionGraph) - all_names = set([solid.name for solid in execution_graph.solids]) - all_depended_on_names = set([solid.name for solid in _all_depended_on_solids(execution_graph)]) - return all_names.difference(all_depended_on_names) - - -def _create_subgraph(execution_graph, from_solids, through_solids): - check.inst_param(execution_graph, 'execution_graph', ExecutionGraph) - check.opt_list_param(from_solids, 'from_solids', of_type=str) - check.opt_list_param(through_solids, 'through_solids', of_type=str) - - if not through_solids: - through_solids = list(_all_sink_solids(execution_graph)) - - if not from_solids: - all_deps = set() - for through_solid in through_solids: - all_deps.union(execution_graph.transitive_dependencies_of(through_solid)) - - from_solids = list(all_deps) - - return execution_graph.create_execution_subgraph(from_solids, through_solids) From da53a7ffbe88faedad25855a077cea19d14f64d0 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Thu, 30 Aug 2018 11:30:34 -0700 Subject: [PATCH 041/103] fix formatting in config --- python_modules/dagster/dagster/config.py | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/python_modules/dagster/dagster/config.py b/python_modules/dagster/dagster/config.py index 08e4e3fa8ec4d..237b4f6fa2e54 100644 --- a/python_modules/dagster/dagster/config.py +++ b/python_modules/dagster/dagster/config.py @@ -7,14 +7,18 @@ class Context(namedtuple('ContextData', 'name args')): def __new__(cls, name, args): return super(Context, cls).__new__( - cls, check.str_param(name, 'name'), check.dict_param(args, 'args', key_type=str) + cls, + check.str_param(name, 'name'), + check.dict_param(args, 'args', key_type=str), ) class Solid(namedtuple('Solid', 'config_dict')): def __new__(cls, config_dict): - return super(Solid, - cls).__new__(cls, check.dict_param(config_dict, 'config_dict', key_type=str)) + return super(Solid, cls).__new__( + cls, + check.dict_param(config_dict, 'config_dict', key_type=str), + ) class Environment(namedtuple('EnvironmentData', 'context solids expectations')): @@ -42,7 +46,8 @@ def empty(): class Expectations(namedtuple('ExpectationsData', 'evaluate')): def __new__(cls, evaluate): return super(Expectations, cls).__new__( - cls, evaluate=check.bool_param(evaluate, 'evaluate') + cls, + evaluate=check.bool_param(evaluate, 'evaluate'), ) From da5fa1f463a4b0f4e83523a6b72bc4bd707098c0 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Thu, 30 Aug 2018 11:47:56 -0700 Subject: [PATCH 042/103] Disallow returning Result from single_output_transform This almost certainly represents programmer error so disallow. --- .../dagster/core/core_tests/test_pipeline_errors.py | 12 ++++++++++++ python_modules/dagster/dagster/core/definitions.py | 9 ++++++++- 2 files changed, 20 insertions(+), 1 deletion(-) diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py index 92354490a3b87..04f7accb2c782 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py @@ -148,3 +148,15 @@ def _tn(*_args, **_kwargs): message="Tranform for solid yield_wrong_thing yielded 'foo'", ): execute_single_solid(ExecutionContext(), solid_inst, config.Environment()) + + +def test_single_transform_returning_result(): + solid_inst = SolidDefinition.single_output_transform( + 'test_return_result', + inputs=[], + transform_fn=lambda *_args, **_kwargs: Result(None), + output=OutputDefinition() + ) + + with pytest.raises(DagsterInvariantViolationError): + execute_single_solid(ExecutionContext(), solid_inst, config.Environment()) diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index 144da41dab767..d769539e5df5c 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -11,7 +11,10 @@ define_colored_console_logger, ) -from .errors import DagsterInvalidDefinitionError +from .errors import ( + DagsterInvalidDefinitionError, + DagsterInvariantViolationError, +) from .execution_context import ExecutionContext @@ -648,6 +651,10 @@ def single_output_transform(name, inputs, transform_fn, output, description=None def _new_transform_fn(context, inputs, _config_dict): value = transform_fn(context, inputs) + if isinstance(value, Result): + raise DagsterInvariantViolationError( + '''Single output transform Solid {name} returned a Result. Just return + value directly without wrapping it in Result''') yield Result(output_name=DEFAULT_OUTPUT, value=value) return SolidDefinition( From e722cd70188587bf6ec27f421e440df99eea5474 Mon Sep 17 00:00:00 2001 From: Mikhail Novikov Date: Fri, 31 Aug 2018 10:21:10 +0300 Subject: [PATCH 043/103] A Yapf run --- .pre-commit-config.yaml | 4 + python_modules/.style.yapf | 247 +---------------- python_modules/dagit/.style.yapf | 248 +----------------- .../dagster/.pre-commit-config.yaml | 8 - .../dagster/dagster/core/compute_nodes.py | 5 +- .../dagster/dagster/core/definitions.py | 3 +- .../dagster/dagster/core/execution.py | 6 +- ...est_pandas_hello_world_no_library_slide.py | 13 +- .../dagster/dagster/utils/__init__.py | 2 + .../dagster/utils/indenting_printer.py | 1 + .../dagster/dagster/utils/logging.py | 9 +- python_modules/dagster/dagster/utils/test.py | 2 - .../dagster/dagster/utils/timing.py | 2 + 13 files changed, 36 insertions(+), 514 deletions(-) create mode 100644 .pre-commit-config.yaml mode change 100644 => 120000 python_modules/dagit/.style.yapf delete mode 100644 python_modules/dagster/.pre-commit-config.yaml diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml new file mode 100644 index 0000000000000..b76d8a22c91d9 --- /dev/null +++ b/.pre-commit-config.yaml @@ -0,0 +1,4 @@ +- repo: https://github.com/pre-commit/mirrors-yapf + sha: v0.22.0 + hooks: + - id: yapf diff --git a/python_modules/.style.yapf b/python_modules/.style.yapf index 90b61186780f7..413ec75db5c3a 100644 --- a/python_modules/.style.yapf +++ b/python_modules/.style.yapf @@ -1,247 +1,4 @@ [style] -# Align closing bracket with visual indentation. -align_closing_bracket_with_visual_indent=True - -# Allow dictionary keys to exist on multiple lines. For example: -# -# x = { -# ('this is the first element of a tuple', -# 'this is the second element of a tuple'): -# value, -# } -allow_multiline_dictionary_keys=False - -# Allow lambdas to be formatted on more than one line. -allow_multiline_lambdas=False - -# Allow splits before the dictionary value. -allow_split_before_dict_value=True - -# Number of blank lines surrounding top-level function and class -# definitions. -blank_lines_around_top_level_definition=2 - -# Insert a blank line before a class-level docstring. -blank_line_before_class_docstring=False - -# Insert a blank line before a module docstring. -blank_line_before_module_docstring=False - -# Insert a blank line before a 'def' or 'class' immediately nested -# within another 'def' or 'class'. For example: -# -# class Foo: -# # <------ this blank line -# def method(): -# ... -blank_line_before_nested_class_or_def=False - -# Do not split consecutive brackets. Only relevant when -# dedent_closing_brackets is set. For example: -# -# call_func_that_takes_a_dict( -# { -# 'key1': 'value1', -# 'key2': 'value2', -# } -# ) -# -# would reformat to: -# -# call_func_that_takes_a_dict({ -# 'key1': 'value1', -# 'key2': 'value2', -# }) -coalesce_brackets=False - -# The column limit. +based_on_style=pep8 column_limit=100 - -# The style for continuation alignment. Possible values are: -# -# - SPACE: Use spaces for continuation alignment. This is default behavior. -# - FIXED: Use fixed number (CONTINUATION_INDENT_WIDTH) of columns -# (ie: CONTINUATION_INDENT_WIDTH/INDENT_WIDTH tabs) for continuation -# alignment. -# - LESS: Slightly left if cannot vertically align continuation lines with -# indent characters. -# - VALIGN-RIGHT: Vertically align continuation lines with indent -# characters. Slightly right (one more indent character) if cannot -# vertically align continuation lines with indent characters. -# -# For options FIXED, and VALIGN-RIGHT are only available when USE_TABS is -# enabled. -continuation_align_style=SPACE - -# Indent width used for line continuations. -continuation_indent_width=4 - -# Put closing brackets on a separate line, dedented, if the bracketed -# expression can't fit in a single line. Applies to all kinds of brackets, -# including function definitions and calls. For example: -# -# config = { -# 'key1': 'value1', -# 'key2': 'value2', -# } # <--- this bracket is dedented and on a separate line -# -# time_series = self.remote_client.query_entity_counters( -# entity='dev3246.region1', -# key='dns.query_latency_tcp', -# transform=Transformation.AVERAGE(window=timedelta(seconds=60)), -# start_ts=now()-timedelta(days=3), -# end_ts=now(), -# ) # <--- this bracket is dedented and on a separate line -dedent_closing_brackets=True - -# Place each dictionary entry onto its own line. -each_dict_entry_on_separate_line=True - -# The regex for an i18n comment. The presence of this comment stops -# reformatting of that line, because the comments are required to be -# next to the string they translate. -i18n_comment= - -# The i18n function call names. The presence of this function stops -# reformattting on that line, because the string it has cannot be moved -# away from the i18n comment. -i18n_function_call= - -# Indent the dictionary value if it cannot fit on the same line as the -# dictionary key. For example: -# -# config = { -# 'key1': -# 'value1', -# 'key2': value1 + -# value2, -# } -indent_dictionary_value=False - -# The number of columns to use for indentation. -indent_width=4 - -# Join short lines into one line. E.g., single line 'if' statements. -join_multiple_lines=True - -# Do not include spaces around selected binary operators. For example: -# -# 1 + 2 * 3 - 4 / 5 -# -# will be formatted as follows when configured with a value "*,/": -# -# 1 + 2*3 - 4/5 -# -no_spaces_around_selected_binary_operators=set() - -# Use spaces around default or named assigns. -spaces_around_default_or_named_assign=False - -# Use spaces around the power operator. -spaces_around_power_operator=False - -# The number of spaces required before a trailing comment. -spaces_before_comment=2 - -# Insert a space between the ending comma and closing bracket of a list, -# etc. -space_between_ending_comma_and_closing_bracket=True - -# Split before arguments -split_all_comma_separated_values=False - -# Split before arguments if the argument list is terminated by a -# comma. -split_arguments_when_comma_terminated=False - -# Set to True to prefer splitting before '&', '|' or '^' rather than -# after. -split_before_bitwise_operator=True - -# Split before the closing bracket if a list or dict literal doesn't fit on -# a single line. -split_before_closing_bracket=True - -# Split before a dictionary or set generator (comp_for). For example, note -# the split before the 'for': -# -# foo = { -# variable: 'Hello world, have a nice day!' -# for variable in bar if variable != 42 -# } -split_before_dict_set_generator=True - -# Split after the opening paren which surrounds an expression if it doesn't -# fit on a single line. -split_before_expression_after_opening_paren=False - -# If an argument / parameter list is going to be split, then split before -# the first argument. -split_before_first_argument=False - -# Set to True to prefer splitting before 'and' or 'or' rather than -# after. -split_before_logical_operator=True - -# Split named assignments onto individual lines. -split_before_named_assigns=True - -# Set to True to split list comprehensions and generators that have -# non-trivial expressions and multiple clauses before each of these -# clauses. For example: -# -# result = [ -# a_long_var + 100 for a_long_var in xrange(1000) -# if a_long_var % 10] -# -# would reformat to something like: -# -# result = [ -# a_long_var + 100 -# for a_long_var in xrange(1000) -# if a_long_var % 10] -split_complex_comprehension=False - -# The penalty for splitting right after the opening bracket. -split_penalty_after_opening_bracket=30 - -# The penalty for splitting the line after a unary operator. -split_penalty_after_unary_operator=10000 - -# The penalty for splitting right before an if expression. -split_penalty_before_if_expr=0 - -# The penalty of splitting the line around the '&', '|', and '^' -# operators. -split_penalty_bitwise_operator=300 - -# The penalty for splitting a list comprehension or generator -# expression. -split_penalty_comprehension=80 - -# The penalty for characters over the column limit. -split_penalty_excess_character=4500 - -# The penalty incurred by adding a line split to the unwrapped line. The -# more line splits added the higher the penalty. -split_penalty_for_added_line_split=30 - -# The penalty of splitting a list of "import as" names. For example: -# -# from a_very_long_or_indented_module_name_yada_yad import (long_argument_1, -# long_argument_2, -# long_argument_3) -# -# would reformat to something like: -# -# from a_very_long_or_indented_module_name_yada_yad import ( -# long_argument_1, long_argument_2, long_argument_3) -split_penalty_import_names=0 - -# The penalty of splitting the line around the 'and' and 'or' -# operators. -split_penalty_logical_operator=300 - -# Use the Tab character for indentation. -use_tabs=False - +dedent_closing_brackets=true diff --git a/python_modules/dagit/.style.yapf b/python_modules/dagit/.style.yapf deleted file mode 100644 index 90b61186780f7..0000000000000 --- a/python_modules/dagit/.style.yapf +++ /dev/null @@ -1,247 +0,0 @@ -[style] -# Align closing bracket with visual indentation. -align_closing_bracket_with_visual_indent=True - -# Allow dictionary keys to exist on multiple lines. For example: -# -# x = { -# ('this is the first element of a tuple', -# 'this is the second element of a tuple'): -# value, -# } -allow_multiline_dictionary_keys=False - -# Allow lambdas to be formatted on more than one line. -allow_multiline_lambdas=False - -# Allow splits before the dictionary value. -allow_split_before_dict_value=True - -# Number of blank lines surrounding top-level function and class -# definitions. -blank_lines_around_top_level_definition=2 - -# Insert a blank line before a class-level docstring. -blank_line_before_class_docstring=False - -# Insert a blank line before a module docstring. -blank_line_before_module_docstring=False - -# Insert a blank line before a 'def' or 'class' immediately nested -# within another 'def' or 'class'. For example: -# -# class Foo: -# # <------ this blank line -# def method(): -# ... -blank_line_before_nested_class_or_def=False - -# Do not split consecutive brackets. Only relevant when -# dedent_closing_brackets is set. For example: -# -# call_func_that_takes_a_dict( -# { -# 'key1': 'value1', -# 'key2': 'value2', -# } -# ) -# -# would reformat to: -# -# call_func_that_takes_a_dict({ -# 'key1': 'value1', -# 'key2': 'value2', -# }) -coalesce_brackets=False - -# The column limit. -column_limit=100 - -# The style for continuation alignment. Possible values are: -# -# - SPACE: Use spaces for continuation alignment. This is default behavior. -# - FIXED: Use fixed number (CONTINUATION_INDENT_WIDTH) of columns -# (ie: CONTINUATION_INDENT_WIDTH/INDENT_WIDTH tabs) for continuation -# alignment. -# - LESS: Slightly left if cannot vertically align continuation lines with -# indent characters. -# - VALIGN-RIGHT: Vertically align continuation lines with indent -# characters. Slightly right (one more indent character) if cannot -# vertically align continuation lines with indent characters. -# -# For options FIXED, and VALIGN-RIGHT are only available when USE_TABS is -# enabled. -continuation_align_style=SPACE - -# Indent width used for line continuations. -continuation_indent_width=4 - -# Put closing brackets on a separate line, dedented, if the bracketed -# expression can't fit in a single line. Applies to all kinds of brackets, -# including function definitions and calls. For example: -# -# config = { -# 'key1': 'value1', -# 'key2': 'value2', -# } # <--- this bracket is dedented and on a separate line -# -# time_series = self.remote_client.query_entity_counters( -# entity='dev3246.region1', -# key='dns.query_latency_tcp', -# transform=Transformation.AVERAGE(window=timedelta(seconds=60)), -# start_ts=now()-timedelta(days=3), -# end_ts=now(), -# ) # <--- this bracket is dedented and on a separate line -dedent_closing_brackets=True - -# Place each dictionary entry onto its own line. -each_dict_entry_on_separate_line=True - -# The regex for an i18n comment. The presence of this comment stops -# reformatting of that line, because the comments are required to be -# next to the string they translate. -i18n_comment= - -# The i18n function call names. The presence of this function stops -# reformattting on that line, because the string it has cannot be moved -# away from the i18n comment. -i18n_function_call= - -# Indent the dictionary value if it cannot fit on the same line as the -# dictionary key. For example: -# -# config = { -# 'key1': -# 'value1', -# 'key2': value1 + -# value2, -# } -indent_dictionary_value=False - -# The number of columns to use for indentation. -indent_width=4 - -# Join short lines into one line. E.g., single line 'if' statements. -join_multiple_lines=True - -# Do not include spaces around selected binary operators. For example: -# -# 1 + 2 * 3 - 4 / 5 -# -# will be formatted as follows when configured with a value "*,/": -# -# 1 + 2*3 - 4/5 -# -no_spaces_around_selected_binary_operators=set() - -# Use spaces around default or named assigns. -spaces_around_default_or_named_assign=False - -# Use spaces around the power operator. -spaces_around_power_operator=False - -# The number of spaces required before a trailing comment. -spaces_before_comment=2 - -# Insert a space between the ending comma and closing bracket of a list, -# etc. -space_between_ending_comma_and_closing_bracket=True - -# Split before arguments -split_all_comma_separated_values=False - -# Split before arguments if the argument list is terminated by a -# comma. -split_arguments_when_comma_terminated=False - -# Set to True to prefer splitting before '&', '|' or '^' rather than -# after. -split_before_bitwise_operator=True - -# Split before the closing bracket if a list or dict literal doesn't fit on -# a single line. -split_before_closing_bracket=True - -# Split before a dictionary or set generator (comp_for). For example, note -# the split before the 'for': -# -# foo = { -# variable: 'Hello world, have a nice day!' -# for variable in bar if variable != 42 -# } -split_before_dict_set_generator=True - -# Split after the opening paren which surrounds an expression if it doesn't -# fit on a single line. -split_before_expression_after_opening_paren=False - -# If an argument / parameter list is going to be split, then split before -# the first argument. -split_before_first_argument=False - -# Set to True to prefer splitting before 'and' or 'or' rather than -# after. -split_before_logical_operator=True - -# Split named assignments onto individual lines. -split_before_named_assigns=True - -# Set to True to split list comprehensions and generators that have -# non-trivial expressions and multiple clauses before each of these -# clauses. For example: -# -# result = [ -# a_long_var + 100 for a_long_var in xrange(1000) -# if a_long_var % 10] -# -# would reformat to something like: -# -# result = [ -# a_long_var + 100 -# for a_long_var in xrange(1000) -# if a_long_var % 10] -split_complex_comprehension=False - -# The penalty for splitting right after the opening bracket. -split_penalty_after_opening_bracket=30 - -# The penalty for splitting the line after a unary operator. -split_penalty_after_unary_operator=10000 - -# The penalty for splitting right before an if expression. -split_penalty_before_if_expr=0 - -# The penalty of splitting the line around the '&', '|', and '^' -# operators. -split_penalty_bitwise_operator=300 - -# The penalty for splitting a list comprehension or generator -# expression. -split_penalty_comprehension=80 - -# The penalty for characters over the column limit. -split_penalty_excess_character=4500 - -# The penalty incurred by adding a line split to the unwrapped line. The -# more line splits added the higher the penalty. -split_penalty_for_added_line_split=30 - -# The penalty of splitting a list of "import as" names. For example: -# -# from a_very_long_or_indented_module_name_yada_yad import (long_argument_1, -# long_argument_2, -# long_argument_3) -# -# would reformat to something like: -# -# from a_very_long_or_indented_module_name_yada_yad import ( -# long_argument_1, long_argument_2, long_argument_3) -split_penalty_import_names=0 - -# The penalty of splitting the line around the 'and' and 'or' -# operators. -split_penalty_logical_operator=300 - -# Use the Tab character for indentation. -use_tabs=False - diff --git a/python_modules/dagit/.style.yapf b/python_modules/dagit/.style.yapf new file mode 120000 index 0000000000000..f854f7a83e209 --- /dev/null +++ b/python_modules/dagit/.style.yapf @@ -0,0 +1 @@ +../.style.yapf \ No newline at end of file diff --git a/python_modules/dagster/.pre-commit-config.yaml b/python_modules/dagster/.pre-commit-config.yaml deleted file mode 100644 index d5db1ded860a8..0000000000000 --- a/python_modules/dagster/.pre-commit-config.yaml +++ /dev/null @@ -1,8 +0,0 @@ -- repo: https://github.com/pre-commit/mirrors-yapf - sha: v0.22.0 - hooks: - - id: yapf -# - repo: https://github.com/pre-commit/pre-commit-hooks -# rev: v1.2.3 -# hooks: -# - id: flake8 diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index d7d6589222397..9193ee96ff1d5 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -43,7 +43,9 @@ from .types import DagsterType -class ComputeNodeExecutionInfo(namedtuple('_ComputeNodeExecutionInfo', 'context execution_graph environment')): +class ComputeNodeExecutionInfo( + namedtuple('_ComputeNodeExecutionInfo', 'context execution_graph environment') +): def __new__(cls, context, execution_graph, environment): return super(ComputeNodeExecutionInfo, cls).__new__( cls, @@ -493,6 +495,7 @@ def __setitem__(self, key, val): check.inst_param(val, 'val', ComputeNodeOutputHandle) return dict.__setitem__(self, key, val) + def validate_config_dict(execution_info, solid): check.inst_param(execution_info, 'execution_info', ComputeNodeExecutionInfo) check.inst_param(solid, 'solid', SolidDefinition) diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index d769539e5df5c..493359e58d28a 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -654,7 +654,8 @@ def _new_transform_fn(context, inputs, _config_dict): if isinstance(value, Result): raise DagsterInvariantViolationError( '''Single output transform Solid {name} returned a Result. Just return - value directly without wrapping it in Result''') + value directly without wrapping it in Result''' + ) yield Result(output_name=DEFAULT_OUTPUT, value=value) return SolidDefinition( diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index f919fd30b7bc2..4c0c26608b574 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -27,7 +27,7 @@ from .definitions import ( DEFAULT_OUTPUT, - ExecutionGraph, + ExecutionGraph, PipelineDefinition, SolidDefinition, ) @@ -199,8 +199,8 @@ def yield_context(pipeline, environment): context_definition = pipeline.context_definitions[context_name] args_to_pass = validate_args( - pipeline.context_definitions[context_name].argument_def_dict, - environment.context.args, 'pipeline {pipeline_name} context {context_name}'.format( + pipeline.context_definitions[context_name].argument_def_dict, environment.context.args, + 'pipeline {pipeline_name} context {context_name}'.format( pipeline_name=pipeline.name, context_name=context_name, ) diff --git a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_no_library_slide.py b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_no_library_slide.py index 59bacd118f288..fead0e8804c19 100644 --- a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_no_library_slide.py +++ b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_no_library_slide.py @@ -191,12 +191,13 @@ def solid_two_transform(_context, args): environment=environment, ) - assert execute_pipeline_result.result_for_solid('solid_two').transformed_value().to_dict('list') == { - 'num1': [1, 3], - 'num2': [2, 4], - 'sum': [3, 7], - 'sum_sq': [9, 49], - } + assert execute_pipeline_result.result_for_solid('solid_two' + ).transformed_value().to_dict('list') == { + 'num1': [1, 3], + 'num2': [2, 4], + 'sum': [3, 7], + 'sum_sq': [9, 49], + } sum_sq_out_path = '/tmp/sum_sq.csv' import os diff --git a/python_modules/dagster/dagster/utils/__init__.py b/python_modules/dagster/dagster/utils/__init__.py index 95bd47a77e2e8..8f7979dbae965 100644 --- a/python_modules/dagster/dagster/utils/__init__.py +++ b/python_modules/dagster/dagster/utils/__init__.py @@ -2,6 +2,8 @@ import os from dagster import check + + def script_relative_path(file_path): ''' Useful for testing with local files. Use a path relative to where the diff --git a/python_modules/dagster/dagster/utils/indenting_printer.py b/python_modules/dagster/dagster/utils/indenting_printer.py index 3e064bd19070c..da2e6c716ecb2 100644 --- a/python_modules/dagster/dagster/utils/indenting_printer.py +++ b/python_modules/dagster/dagster/utils/indenting_printer.py @@ -2,6 +2,7 @@ from dagster import check + class IndentingPrinter: def __init__(self, indent_level=2, printer=print): self.lines = [] diff --git a/python_modules/dagster/dagster/utils/logging.py b/python_modules/dagster/dagster/utils/logging.py index b6e29d4dbd55c..5183e3f5a3097 100644 --- a/python_modules/dagster/dagster/utils/logging.py +++ b/python_modules/dagster/dagster/utils/logging.py @@ -18,10 +18,12 @@ 'WARNING': WARNING, } + def level_from_string(string): check.str_param(string, 'string') return LOOKUP[string] + class CompositeLogger: def __init__(self, loggers=None): self.loggers = check.opt_list_param(loggers, 'loggers', of_type=logging.Logger) @@ -31,6 +33,7 @@ def _invoke_logger_method(*args, **kwargs): for logger in self.loggers: logger_method = check.is_callable(getattr(logger, name)) logger_method(*args, **kwargs) + return _invoke_logger_method @@ -46,7 +49,7 @@ def emit(self, record): text_line = json.dumps(logged_properties) ff.write(text_line + '\n') # Need to catch Exception here, so disabling lint - except Exception as e: # pylint: disable=W0703 + except Exception as e: # pylint: disable=W0703 logging.critical('Error during logging!') logging.exception(str(e)) @@ -81,17 +84,21 @@ def define_colored_console_logger(name, level=INFO): coloredlogs.install(logger=logger, level=level, fmt=default_format_string()) return logger + def default_format_string(): return '%(asctime)s - %(name)s - %(levelname)s - %(message)s' + def define_default_formatter(): return logging.Formatter(default_format_string()) + def debug_format_string(): return '''%(name)s.%(levelname)s: %(message)s time: %(asctime)s relative: %(relativeCreated)dms path: %(pathname)s line: %(lineno)d''' + def define_debug_formatter(): return logging.Formatter(debug_format_string()) diff --git a/python_modules/dagster/dagster/utils/test.py b/python_modules/dagster/dagster/utils/test.py index b2bf36301c0b9..3a3ca3117d4ee 100644 --- a/python_modules/dagster/dagster/utils/test.py +++ b/python_modules/dagster/dagster/utils/test.py @@ -6,8 +6,6 @@ from dagster import check - - def _unlink_swallow_errors(path): check.str_param(path, 'path') try: diff --git a/python_modules/dagster/dagster/utils/timing.py b/python_modules/dagster/dagster/utils/timing.py index c0bce97f4baae..d8dfae9cdb800 100644 --- a/python_modules/dagster/dagster/utils/timing.py +++ b/python_modules/dagster/dagster/utils/timing.py @@ -3,6 +3,7 @@ from dagster import check + class TimerResult: def __init__(self): self.start_time = time.time() @@ -17,6 +18,7 @@ def seconds(self): def millis(self): return self.seconds * 1000 + @contextmanager def time_execution_scope(): '''Usage: From 7eca860098a356de7679f1f615e74f07a1330a9b Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Fri, 24 Aug 2018 11:49:21 -0700 Subject: [PATCH 044/103] Add RepositoryDefinition Before we had no way of grouping pipelines together aside from the pipelines.yml file. Instead, this moves that concept to a Python API called the repository. A repository is a collection of pipelines. Instead of specifying every pipeline in a project individually in a yaml file, we now just have a single repository. The repository object creates pipelines on demand (so that they can be loaded by name without loading all the pipelines). Repositories *also* have the concept of library solids (which can be thought of higher order solids) which are ways to create solids in a config-driven way. Authors will be able to design library solids that are resuable. This will enable a layer of solid creation via config, and much easier testing because authors will be able to inject pre-made solids into existing pipelines to stub out values. --- python_modules/dagit/dagit/app.py | 38 ++++- python_modules/dagit/dagit/cli.py | 40 ++--- python_modules/dagit/dagit/schema.py | 12 +- python_modules/dagster/dagster/__init__.py | 3 + .../dagster/dagster/check/__init__.py | 21 ++- .../dagster/check/check_tests/test_check.py | 4 + .../dagster/dagster/cli/__init__.py | 25 +-- python_modules/dagster/dagster/cli/context.py | 83 ---------- .../dagster/dagster/cli/pipeline.py | 77 +++++---- .../dagster/dagster/cli/repository_config.py | 76 +++++++++ .../dagster/dagster/core/argument_handling.py | 53 +++++- .../core/core_tests/test_library_solids.py | 51 ++++++ .../core_tests/test_repository_definition.py | 112 +++++++++++++ .../dagster/dagster/core/definitions.py | 152 +++++++++++------- .../dagster/dagster_examples/pipelines.yml | 5 - .../dagster/dagster_examples/repository.py | 9 ++ .../dagster/dagster_examples/repository.yml | 3 + python_modules/dagster/dagster/graphviz.py | 8 +- python_modules/dagster/setup.py | 2 +- 19 files changed, 523 insertions(+), 251 deletions(-) delete mode 100644 python_modules/dagster/dagster/cli/context.py create mode 100644 python_modules/dagster/dagster/cli/repository_config.py create mode 100644 python_modules/dagster/dagster/core/core_tests/test_library_solids.py create mode 100644 python_modules/dagster/dagster/core/core_tests/test_repository_definition.py delete mode 100644 python_modules/dagster/dagster/dagster_examples/pipelines.yml create mode 100644 python_modules/dagster/dagster/dagster_examples/repository.py create mode 100644 python_modules/dagster/dagster/dagster_examples/repository.yml diff --git a/python_modules/dagit/dagit/app.py b/python_modules/dagit/dagit/app.py index 2a4b2f1f370e4..949dd45becfc3 100644 --- a/python_modules/dagit/dagit/app.py +++ b/python_modules/dagit/dagit/app.py @@ -4,16 +4,44 @@ from flask_graphql import GraphQLView from flask_cors import CORS +from dagster import check +from dagster.cli.repository_config import ( + RepositoryInfo, + reload_repository_info, +) + from .schema import create_schema +class RepositoryContainer: + ''' + This class solely exists to implement reloading semantics. We need to have a single object + that the graphql server has access that stays the same object between reload. This container + object allows the RepositoryInfo to be written in an immutable fashion. + ''' + + def __init__(self, repository_info): + self.repository_info = check.inst_param(repository_info, 'repository_info', RepositoryInfo) + + def reload(self): + self.repository_info = reload_repository_info(self.repository_info) + + @property + def repository(self): + return self.repository_info.repository + + class DagsterGraphQLView(GraphQLView): - def __init__(self, pipeline_config=None, **kwargs): + def __init__(self, repository_container, **kwargs): super(DagsterGraphQLView, self).__init__(**kwargs) - self.pipeline_config = pipeline_config + self.repository_container = check.inst_param( + repository_container, + 'repository_container', + RepositoryContainer, + ) def get_context(self): - return {'pipeline_config': self.pipeline_config} + return {'repository_container': self.repository_container} def static_view(path, file): @@ -26,7 +54,7 @@ def index_view(_path): return send_file(os.path.join(os.path.dirname(__file__), './webapp/build/index.html')) -def create_app(pipeline_config): +def create_app(repository_container): app = Flask('dagster-ui') schema = create_schema() @@ -37,7 +65,7 @@ def create_app(pipeline_config): schema=schema, graphiql=True, executor=AsyncioExecutor(), - pipeline_config=pipeline_config, + repository_container=repository_container, ) ) app.add_url_rule('/static//', 'static_view', static_view) diff --git a/python_modules/dagit/dagit/cli.py b/python_modules/dagit/dagit/cli.py index ca4980f5799d9..bebb404657fde 100644 --- a/python_modules/dagit/dagit/cli.py +++ b/python_modules/dagit/dagit/cli.py @@ -6,9 +6,15 @@ from watchdog.observers import Observer from watchdog.events import FileSystemEventHandler -from dagster.cli.context import Config +from dagster.cli.repository_config import ( + load_repository_from_file, + repository_config_argument, +) -from .app import create_app +from .app import ( + create_app, + RepositoryContainer, +) def create_dagit_cli(): @@ -16,40 +22,28 @@ def create_dagit_cli(): class ReloaderHandler(FileSystemEventHandler): - def __init__(self, pipeline_config): + def __init__(self, repository_container): super(ReloaderHandler, self).__init__() - self.pipeline_config = pipeline_config + self.repository_container = repository_container def on_any_event(self, event): if event.src_path.endswith('.py'): - self.pipeline_config.reload() + self.repository_container.reload() @click.command(name='ui', help='run web ui') -@click.option( - '--config', - '-c', - type=click.Path( - exists=True, - file_okay=True, - dir_okay=False, - readable=True, - resolve_path=True, - ), - default='pipelines.yml', - help="Path to config file. Defaults to ./pipelines.yml." -) +@repository_config_argument @click.option('--host', '-h', type=click.STRING, default='127.0.0.1', help="Host to run server on") @click.option('--port', '-p', type=click.INT, default=3000, help="Port to run server on") -def ui(config, host, port): +def ui(conf, host, port): sys.path.append(os.getcwd()) - pipeline_config = Config.from_file(config) + repository_container = RepositoryContainer(load_repository_from_file(conf)) observer = Observer() - handler = ReloaderHandler(pipeline_config) - observer.schedule(handler, os.path.dirname(os.path.abspath(config)), recursive=True) + handler = ReloaderHandler(repository_container) + observer.schedule(handler, os.path.dirname(os.path.abspath(conf)), recursive=True) observer.start() try: - app = create_app(pipeline_config) + app = create_app(repository_container) serve(app, host=host, port=port) except KeyboardInterrupt: observer.stop() diff --git a/python_modules/dagit/dagit/schema.py b/python_modules/dagit/dagit/schema.py index a0ecb8fb1967e..4c34fee80b6fd 100644 --- a/python_modules/dagit/dagit/schema.py +++ b/python_modules/dagit/dagit/schema.py @@ -6,13 +6,15 @@ class Query(graphene.ObjectType): pipelines = graphene.NonNull(graphene.List(lambda: graphene.NonNull(Pipeline))) def resolve_pipeline(self, info, name): - config = info.context['pipeline_config'] - pipeline_config = config.get_pipeline(name) - return Pipeline(pipeline_config.pipeline) + repository = info.context['repository_container'].repository + return repository.get_pipeline(name) def resolve_pipelines(self, info): - config = info.context['pipeline_config'] - return [Pipeline(c.pipeline) for c in config.create_pipelines()] + repository = info.context['repository_container'].repository + pipelines = [] + for pipeline_def in repository.get_all_pipelines(): + pipelines.append(Pipeline(pipeline_def)) + return pipelines # (XXX) Some stuff is named, other stuffed is keyed in dict. diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index 760c618dedc43..c4c7317cbb2db 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -11,10 +11,13 @@ ExpectationDefinition, ExpectationResult, InputDefinition, + LibraryDefinition, + LibrarySolidDefinition, OutputDefinition, PipelineContextDefinition, PipelineDefinition, Result, + RepositoryDefinition, SolidDefinition, ) diff --git a/python_modules/dagster/dagster/check/__init__.py b/python_modules/dagster/dagster/check/__init__.py index c85f445277d08..fc8d1becf98c1 100644 --- a/python_modules/dagster/dagster/check/__init__.py +++ b/python_modules/dagster/dagster/check/__init__.py @@ -77,9 +77,26 @@ def inst(obj, ttype, desc=None): return obj -def is_callable(obj): +def is_callable(obj, desc=None): if not callable(obj): - raise_with_traceback(CheckError('must be callable')) + if desc: + raise_with_traceback( + CheckError( + 'Must be callable. Got {obj}. Description: {desc}'.format( + obj=repr(obj), + desc=desc, + ) + ) + ) + else: + raise_with_traceback( + CheckError( + 'Must be callable. Got {obj}. Description: {desc}'.format( + obj=obj, + desc=desc, + ) + ) + ) return obj diff --git a/python_modules/dagster/dagster/check/check_tests/test_check.py b/python_modules/dagster/dagster/check/check_tests/test_check.py index 37e4517030d2b..9a6d9841d9602 100644 --- a/python_modules/dagster/dagster/check/check_tests/test_check.py +++ b/python_modules/dagster/dagster/check/check_tests/test_check.py @@ -485,6 +485,7 @@ def fn(): assert check.is_callable(fn) == fn assert check.is_callable(lambda: None) + assert check.is_callable(lambda: None, 'some desc') with pytest.raises(CheckError): check.is_callable(None) @@ -492,6 +493,9 @@ def fn(): with pytest.raises(CheckError): check.is_callable(1) + with pytest.raises(CheckError, message='some other desc'): + check.is_callable(1, 'some other desc') + def test_tuple_param(): assert check.tuple_param((1, 2), 'something') diff --git a/python_modules/dagster/dagster/cli/__init__.py b/python_modules/dagster/dagster/cli/__init__.py index cce06228e40b8..22559e64ebecc 100644 --- a/python_modules/dagster/dagster/cli/__init__.py +++ b/python_modules/dagster/dagster/cli/__init__.py @@ -1,32 +1,9 @@ -import os -import sys - import click -from .context import Config from .pipeline import create_pipeline_cli def create_dagster_cli(): - group = click.group()(dagster_cli) + group = click.Group() group.add_command(create_pipeline_cli()) return group - - -@click.option( - '--config', - '-c', - type=click.Path( - exists=True, - file_okay=True, - dir_okay=False, - readable=True, - resolve_path=True, - ), - default='pipelines.yml', - help="Path to config file. Defaults to ./pipelines.yml." -) -@click.pass_context -def dagster_cli(ctx, config): - ctx.obj = Config.from_file(config) - sys.path.append(os.getcwd()) diff --git a/python_modules/dagster/dagster/cli/context.py b/python_modules/dagster/dagster/cli/context.py deleted file mode 100644 index e656ed4a08dde..0000000000000 --- a/python_modules/dagster/dagster/cli/context.py +++ /dev/null @@ -1,83 +0,0 @@ -import importlib - -import click -import yaml - -from dagster import (check, PipelineDefinition) - - -def define_config_class(): - class _Config: - def __init__(self, pipeline_configs): - self.pipeline_configs = pipeline_configs - self.pipelines = None - - @staticmethod - def from_file(filepath): - with open(filepath, 'r') as ff: - config = yaml.load(ff) - - pipeline_configs = [ - PipelineConfig( - module=check.str_elem(entry, 'module'), fn=check.str_elem(entry, 'fn') - ) for entry in check.list_elem(config, 'pipelines') - ] - - return Config(pipeline_configs=pipeline_configs) - - def create_pipelines(self): - for pipeline_config in self.pipeline_configs: - pipeline_config.create_pipeline() - - return self.pipeline_configs - - def get_pipeline(self, name): - if not self.pipelines: - self.create_pipelines() - - for pipeline_config in self.pipeline_configs: - if pipeline_config.pipeline.name == name: - return pipeline_config - - check.failed(f'pipeline {name} not found') - - def reload(self): - for pipeline_config in self.pipeline_configs: - if pipeline_config.module: - importlib.reload(pipeline_config.module) - pipeline_config.create_pipeline() - - return self.pipeline_configs - - # This lets you ask cli commands to have this object extracted from context and - # passed as first arg - _Config.pass_object = click.make_pass_decorator(_Config) - return _Config - - -Config = define_config_class() - - -def define_pipeline_config_class(): - class _PipelineConfig: - def __init__(self, module, fn): - self.module_name = module - self.fn_name = fn - self.module = None - self.fn = None - self.pipeline = None - - def create_pipeline(self): - self.module = importlib.import_module(self.module_name) - self.fn = getattr(self.module, self.fn_name) - check.is_callable(self.fn) - self.pipeline = check.inst(self.fn(), PipelineDefinition) - return self.pipeline - - # This lets you ask cli commands to have this object extracted from context and - # passed as first arg - _PipelineConfig.pass_object = click.make_pass_decorator(_PipelineConfig) - return _PipelineConfig - - -PipelineConfig = define_pipeline_config_class() diff --git a/python_modules/dagster/dagster/cli/pipeline.py b/python_modules/dagster/dagster/cli/pipeline.py index cd3fd95f8e83c..7f4887c0b70d1 100644 --- a/python_modules/dagster/dagster/cli/pipeline.py +++ b/python_modules/dagster/dagster/cli/pipeline.py @@ -1,20 +1,26 @@ import logging -import os import re import textwrap import click import yaml -import dagster -from dagster import check +from dagster import ( + PipelineDefinition, + check, + config, +) + from dagster.core.definitions import ExecutionGraph from dagster.core.execution import execute_pipeline_iterator from dagster.core.errors import DagsterExecutionFailureReason from dagster.graphviz import build_graphviz_graph from dagster.utils.indenting_printer import IndentingPrinter -from .context import Config +from .repository_config import ( + load_repository_from_file, + repository_config_argument, +) def create_pipeline_cli(): @@ -27,12 +33,11 @@ def create_pipeline_cli(): @click.command(name='list', help="list") -@Config.pass_object -def list_command(config): - pipeline_configs = config.create_pipelines() - - for pipeline_config in pipeline_configs: - pipeline = pipeline_config.pipeline +@repository_config_argument +def list_command(conf): + repository = load_repository_from_file(conf).repository + click.echo('Repository {name}'.format(name=repository.name)) + for pipeline in repository.get_all_pipelines(): click.echo('Pipeline: {name}'.format(name=pipeline.name)) if pipeline.description: click.echo('Description:') @@ -54,26 +59,25 @@ def format_description(desc, indent): return filled -def set_pipeline(ctx, _arg, value): - ctx.params['pipeline_config'] = ctx.find_object(Config).get_pipeline(value) - - -def pipeline_name_argument(f): - return click.argument('pipeline_name', callback=set_pipeline, expose_value=False)(f) +def pipeline_from_conf(conf, name): + repository = load_repository_from_file(conf).repository + return repository.get_pipeline(name) @click.command(name='print', help="print <>") +@repository_config_argument @click.option('--verbose', is_flag=True) -@pipeline_name_argument -def print_command(pipeline_config, verbose): +@click.argument('name') +def print_command(conf, name, verbose): + pipeline = pipeline_from_conf(conf, name) if verbose: - print_pipeline(pipeline_config.pipeline, full=True, print_fn=click.echo) + print_pipeline(pipeline, full=True, print_fn=click.echo) else: - print_solids(pipeline_config.pipeline, print_fn=click.echo) + print_solids(pipeline, print_fn=click.echo) def print_solids(pipeline, print_fn): - check.inst_param(pipeline, 'pipeline', dagster.PipelineDefinition) + check.inst_param(pipeline, 'pipeline', PipelineDefinition) check.callable_param(print_fn, 'print_fn') printer = IndentingPrinter(indent_level=2, printer=print_fn) @@ -86,7 +90,7 @@ def print_solids(pipeline, print_fn): def print_pipeline(pipeline, full, print_fn): - check.inst_param(pipeline, 'pipeline', dagster.PipelineDefinition) + check.inst_param(pipeline, 'pipeline', PipelineDefinition) check.bool_param(full, 'full') check.callable_param(print_fn, 'print_fn') @@ -164,16 +168,11 @@ def format_argument_dict(arg_def_dict): @click.command(name='graphviz', help="graphviz <>") -@pipeline_name_argument -def graphviz_command(pipeline_config): - build_graphviz_graph(pipeline_config.pipeline).view(cleanup=True) - - -def get_default_config_for_pipeline(): - ctx = click.get_current_context() - pipeline_config = ctx.params['pipeline_config'] - module_path = os.path.dirname(pipeline_config.module.__file__) - return os.path.join(module_path, 'env.yml') +@repository_config_argument +@click.argument('name') +def graphviz_command(conf, name): + pipeline = pipeline_from_conf(conf, name) + build_graphviz_graph(pipeline).view(cleanup=True) LOGGING_DICT = { @@ -192,7 +191,8 @@ def load_yaml_from_path(path): @click.command(name='execute', help="execute <>") -@pipeline_name_argument +@repository_config_argument +@click.argument('name') @click.option( '-e', '--env', @@ -203,21 +203,20 @@ def load_yaml_from_path(path): readable=True, resolve_path=True, ), - default=get_default_config_for_pipeline, - help="Path to environment file. Defaults to ./PIPELINE_DIR/env.yml." ) -def execute_command(pipeline_config, env): - do_execute_command(pipeline_config.pipeline, env, print) +def execute_command(conf, name, env): + pipeline = pipeline_from_conf(conf, name) + do_execute_command(pipeline, env, print) def do_execute_command(pipeline, env, printer): - check.inst_param(pipeline, 'pipeline', dagster.PipelineDefinition) + check.inst_param(pipeline, 'pipeline', PipelineDefinition) check.str_param(env, 'env') check.callable_param(printer, 'printer') env_config = load_yaml_from_path(env) - environment = dagster.config.construct_environment(env_config) + environment = config.construct_environment(env_config) pipeline_iter = execute_pipeline_iterator(pipeline, environment) diff --git a/python_modules/dagster/dagster/cli/repository_config.py b/python_modules/dagster/dagster/cli/repository_config.py new file mode 100644 index 0000000000000..91da9db5c201c --- /dev/null +++ b/python_modules/dagster/dagster/cli/repository_config.py @@ -0,0 +1,76 @@ +from collections import namedtuple +import importlib + +import click +import yaml + +from dagster import ( + check, + RepositoryDefinition, +) + + +class RepositoryInfo(namedtuple('_RepositoryInfo', 'repository module fn module_name fn_name')): + pass + + +def load_repository_from_file(file_path): + check.str_param(file_path, 'file_path') + with open(file_path, 'r') as ff: + config = yaml.load(ff) + repository_config = check.dict_elem(config, 'repository') + module_name = check.str_elem(repository_config, 'module') + fn_name = check.str_elem(repository_config, 'fn') + + module, fn, repository = load_repository(module_name, fn_name) + + return RepositoryInfo( + repository=repository, + module=module, + fn=fn, + module_name=module_name, + fn_name=fn_name, + ) + + +def load_repository(module_name, fn_name): + module = importlib.import_module(module_name) + fn = getattr(module, fn_name) + check.is_callable(fn) + repository = check.inst(fn(), RepositoryDefinition) + return (module, fn, repository) + + +def reload_repository_info(repository_info): + check.inst_param(repository_info, 'repository_info', RepositoryInfo) + + module_name, fn_name = repository_info.module_name, repository_info.fn_name + + module = importlib.reload(repository_info.module) + fn = getattr(module, fn_name) + check.is_callable(fn) + repository = check.inst(fn(), RepositoryDefinition) + + return RepositoryInfo( + repository=repository, + module=module, + fn=fn, + module_name=module_name, + fn_name=fn_name, + ) + + +def repository_config_argument(f): + return click.option( + '--conf', + '-c', + type=click.Path( + exists=True, + file_okay=True, + dir_okay=False, + readable=True, + resolve_path=True, + ), + default='repository.yml', + help="Path to config file. Defaults to ./pipelines.yml." + )(f) diff --git a/python_modules/dagster/dagster/core/argument_handling.py b/python_modules/dagster/dagster/core/argument_handling.py index 8fb7d3d56db48..a9ce4dec1c999 100644 --- a/python_modules/dagster/dagster/core/argument_handling.py +++ b/python_modules/dagster/dagster/core/argument_handling.py @@ -1,7 +1,58 @@ from dagster import check -from .definitions import ArgumentDefinitionDictionary from .errors import DagsterTypeError +from .types import DagsterType + + +# We wrap the passed in dictionary of str : ArgumentDefinition to +# 1) enforce typing +# 2) enforce immutability +# 3) make type checks throughout execution cheaper +class ArgumentDefinitionDictionary(dict): + def __init__(self, ddict): + super().__init__( + check.dict_param(ddict, 'ddict', key_type=str, value_type=ArgumentDefinition) + ) + + def __setitem__(self, _key, _value): + check.failed('This dictionary is readonly') + + +class __ArgumentValueSentinel: + pass + + +NO_DEFAULT_PROVIDED = __ArgumentValueSentinel + + +class ArgumentDefinition: + def __init__( + self, dagster_type, default_value=NO_DEFAULT_PROVIDED, is_optional=False, description=None + ): + '''Definition of an argument passed through the config system. Used in a few different + contexts: to configure a context, to configure a solid, and more to come. + + We have decided to allow arguments to be explictly made *optional* and separate that + concept from the nullability of the type. That means one could have a *required* argument + that is nullable, because sometimes an argument set to null has a distinct semantic meaning + from the lack of an argument. Optional arguments can have default values. Required arguments + cannot. + ''' + if not is_optional: + check.param_invariant( + default_value == NO_DEFAULT_PROVIDED, + 'default_value', + 'required arguments should not specify default values', + ) + + self.dagster_type = check.inst_param(dagster_type, 'dagster_type', DagsterType) + self.description = check.opt_str_param(description, 'description') + self.is_optional = check.bool_param(is_optional, 'is_optional') + self.default_value = default_value + + @property + def default_provided(self): + return self.default_value != NO_DEFAULT_PROVIDED def validate_args(argument_def_dict, arg_dict, error_context_str): diff --git a/python_modules/dagster/dagster/core/core_tests/test_library_solids.py b/python_modules/dagster/dagster/core/core_tests/test_library_solids.py new file mode 100644 index 0000000000000..af269cce6e5be --- /dev/null +++ b/python_modules/dagster/dagster/core/core_tests/test_library_solids.py @@ -0,0 +1,51 @@ +from dagster import ( + ArgumentDefinition, + LibrarySolidDefinition, + OutputDefinition, + PipelineDefinition, + Result, + SolidDefinition, + execute_pipeline, + config, + types, +) + + +def test_basic_library_solid(): + def get_t_fn(args): + def _t_fn(*_args, **_kwargs): + yield Result({args['key']: args['value']}) + + return _t_fn + + library_solid_def = LibrarySolidDefinition( + name='return_value_in_key', + argument_def_dict={ + 'key' : ArgumentDefinition(types.String), + 'value' : ArgumentDefinition(types.String), + }, + solid_creation_fn=lambda name, args: SolidDefinition( + name=name, + inputs=[], + outputs=[OutputDefinition()], + transform_fn=get_t_fn(args), + ) + ) + + solid = library_solid_def.create_solid( + 'instance_name', { + 'key': 'some_key', + 'value': 'some_value' + } + ) + + assert isinstance(solid, SolidDefinition) + assert solid.name == 'instance_name' + + pipeline = PipelineDefinition(solids=[solid]) + + result = execute_pipeline(pipeline, config.Environment()) + assert result.success + + solid_result = result.result_for_solid('instance_name') + assert solid_result.transformed_value() == {'some_key': 'some_value'} diff --git a/python_modules/dagster/dagster/core/core_tests/test_repository_definition.py b/python_modules/dagster/dagster/core/core_tests/test_repository_definition.py new file mode 100644 index 0000000000000..cd1c76f93402d --- /dev/null +++ b/python_modules/dagster/dagster/core/core_tests/test_repository_definition.py @@ -0,0 +1,112 @@ +from collections import defaultdict + +from dagster import ( + ArgumentDefinition, + LibraryDefinition, + LibrarySolidDefinition, + OutputDefinition, + PipelineDefinition, + RepositoryDefinition, + Result, + SolidDefinition, + types, +) + + +def create_single_node_pipeline(name, called): + called[name] = called[name] + 1 + return PipelineDefinition( + name=name, + solids=[ + SolidDefinition( + name=name + '_solid', + inputs=[], + outputs=[], + transform_fn=lambda *_args, **_kwargs: None, + ) + ] + ) + + +def test_repo_definition(): + called = defaultdict(int) + repo = RepositoryDefinition( + name='some_repo', + pipeline_dict={ + 'foo': lambda: create_single_node_pipeline('foo', called), + 'bar': lambda: create_single_node_pipeline('bar', called), + } + ) + + foo_pipeline = repo.get_pipeline('foo') + assert isinstance(foo_pipeline, PipelineDefinition) + assert foo_pipeline.name == 'foo' + + assert 'foo' in called + assert called['foo'] == 1 + assert 'bar' not in called + + bar_pipeline = repo.get_pipeline('bar') + assert isinstance(bar_pipeline, PipelineDefinition) + assert bar_pipeline.name == 'bar' + + assert 'foo' in called + assert called['foo'] == 1 + assert 'bar' in called + assert called['bar'] == 1 + + foo_pipeline = repo.get_pipeline('foo') + assert isinstance(foo_pipeline, PipelineDefinition) + assert foo_pipeline.name == 'foo' + + assert 'foo' in called + assert called['foo'] == 1 + + pipelines = repo.get_all_pipelines() + + assert set(['foo', 'bar']) == set([pipeline.name for pipeline in pipelines]) + + +def create_library_solid_def(): + def get_t_fn(args): + def _t_fn(*_args, **_kwargs): + yield Result({args['key']: args['value']}) + + return _t_fn + + return LibrarySolidDefinition( + name='return_value_in_key', + argument_def_dict={ + 'key' : ArgumentDefinition(types.String), + 'value' : ArgumentDefinition(types.String), + }, + solid_creation_fn=lambda name, args: SolidDefinition( + name=name, + inputs=[], + outputs=[OutputDefinition()], + transform_fn=get_t_fn(args), + ) + ) + + +def test_repo_with_libraries(): + called = defaultdict(int) + repo = RepositoryDefinition( + name='some_repo', + pipeline_dict={ + 'foo': lambda: create_single_node_pipeline('foo', called), + 'bar': lambda: create_single_node_pipeline('bar', called), + }, + libraries=[ + LibraryDefinition( + name='some_library', + library_solids=[create_library_solid_def()], + ) + ] + ) + + assert isinstance(repo, RepositoryDefinition) + + +def test_repo_def_errors(): + pass # TODO diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index 493359e58d28a..7553180d4d8b4 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -1,4 +1,7 @@ -from collections import (defaultdict, namedtuple) +from collections import ( + defaultdict, + namedtuple, +) import copy import keyword import re @@ -11,6 +14,12 @@ define_colored_console_logger, ) +from .argument_handling import ( + validate_args, + ArgumentDefinition, + ArgumentDefinitionDictionary, +) + from .errors import ( DagsterInvalidDefinitionError, DagsterInvariantViolationError, @@ -53,20 +62,6 @@ def check_valid_name(name): return name -# We wrap the passed in dictionary of str : ArgumentDefinition to -# 1) enforce typing -# 2) enforce immutability -# 3) make type checks throughout execution cheaper -class ArgumentDefinitionDictionary(dict): - def __init__(self, ddict): - super().__init__( - check.dict_param(ddict, 'ddict', key_type=str, value_type=ArgumentDefinition) - ) - - def __setitem__(self, _key, _value): - check.failed('This dictionary is readonly') - - class PipelineContextDefinition: '''Pipelines declare the different context types they support, in the form of PipelineContextDefinitions. For example a pipeline could declare a context @@ -691,53 +686,35 @@ def output_def_named(self, name): return self._output_dict[name] -class __ArgumentValueSentinel: - pass - - -NO_DEFAULT_PROVIDED = __ArgumentValueSentinel - - -class ArgumentDefinition: - '''Definition of an argument passed through the config system. Used in a few different - contexts: to configure a context, to configure a solid, and more to come. +class LibrarySolidDefinition: + def __init__(self, name, argument_def_dict, solid_creation_fn): + self.name = check.str_param(name, 'name') + self.argument_def_dict = ArgumentDefinitionDictionary(argument_def_dict) + self.solid_creation_fn = check.callable_param(solid_creation_fn, 'solid_creation_fn') - We have decided to allow arguments to be explictly made *optional* and separate that - concept from the nullability of the type. That means one could have a *required* argument - that is nullable, because sometimes an argument set to null has a distinct semantic meaning - from the lack of an argument. Optional arguments can have default values. Required arguments - cannot. - ''' + def create_solid(self, name, args): + check.str_param(name, 'name') + check.dict_param(args, 'args', key_type=str) - def __init__( - self, dagster_type, default_value=NO_DEFAULT_PROVIDED, is_optional=False, description=None - ): - ''' - Parameters - --------- - dagster_type: DagsterType - The type of the argument. - default_value: Any (optional, defaults to a sentinel value) - Default value of argument. Can only be provided if argument is optional - is_optional: bool (optional, defaults to false) - Optional. Can the solid execute with the argument even set. - description: str (optional) - ''' - if not is_optional: - check.param_invariant( - default_value == NO_DEFAULT_PROVIDED, - 'default_value', - 'required arguments should not specify default values', + args = validate_args( + self.argument_def_dict, + args, + 'Args of library_solid {name} invalid'.format(name=self.name), + ) + solid = self.solid_creation_fn(name, args) + return check.inst_param(solid, 'solid', SolidDefinition) + + +class LibraryDefinition: + def __init__(self, name, library_solids): + self.name = check.str_param(name, 'name') + self._library_solid_dict = _build_named_dict( + check.list_param( + library_solids, + 'library_solids', + of_type=LibrarySolidDefinition, ) - - self.dagster_type = check.inst_param(dagster_type, 'dagster_type', types.DagsterType) - self.description = check.opt_str_param(description, 'description') - self.is_optional = check.bool_param(is_optional, 'is_optional') - self.default_value = default_value - - @property - def default_provided(self): - return self.default_value != NO_DEFAULT_PROVIDED + ) def _create_adjacency_lists(solids, dep_structure): @@ -976,3 +953,60 @@ def _create_subgraph(execution_graph, from_solids, through_solids): from_solids = list(all_deps) return execution_graph.create_execution_subgraph(from_solids, through_solids) + + +class RepositoryDefinition: + def __init__(self, name, pipeline_dict, libraries=None): + self.name = check.str_param(name, 'name') + + check.dict_param( + pipeline_dict, + 'pipeline_dict', + key_type=str, + ) + + for val in pipeline_dict.values(): + check.is_callable(val, 'Value in pipeline_dict must be function') + + self.pipeline_dict = pipeline_dict + + self._pipeline_cache = {} + + self._library_dict = _build_named_dict( + check.opt_list_param( + libraries, + 'libraries', + of_type=LibraryDefinition, + ) + ) + + def get_library(self, name): + check.str_param(name, 'name') + return self._library_dict[name] + + def get_pipeline(self, name): + if name in self._pipeline_cache: + return self._pipeline_cache[name] + + pipeline = self.pipeline_dict[name]() + check.invariant( + pipeline.name == name, + f'Name does not match. Name in dict {name}. Name in pipeline {pipeline.name}' + ) + + check.inst( + pipeline, + PipelineDefinition, + 'Function passed into pipeline_dict with key {key} must return a PipelineDefinition'. + format(key=name), + ) + + self._pipeline_cache[name] = pipeline + return pipeline + + def iterate_over_pipelines(self): + for name in self.pipeline_dict.keys(): + yield self.get_pipeline(name) + + def get_all_pipelines(self): + return list(self.iterate_over_pipelines()) diff --git a/python_modules/dagster/dagster/dagster_examples/pipelines.yml b/python_modules/dagster/dagster/dagster_examples/pipelines.yml deleted file mode 100644 index bf17c1db98522..0000000000000 --- a/python_modules/dagster/dagster/dagster_examples/pipelines.yml +++ /dev/null @@ -1,5 +0,0 @@ -pipelines: - - module: dagster.dagster_examples.pandas_hello_world.pipeline - fn: define_pipeline - - module: dagster.dagster_examples.pandas_hello_world.pipeline - fn: define_success_pipeline diff --git a/python_modules/dagster/dagster/dagster_examples/repository.py b/python_modules/dagster/dagster/dagster_examples/repository.py new file mode 100644 index 0000000000000..52d437cb63be3 --- /dev/null +++ b/python_modules/dagster/dagster/dagster_examples/repository.py @@ -0,0 +1,9 @@ +from dagster import (RepositoryDefinition) +from dagster.dagster_examples.pandas_hello_world.pipeline import define_success_pipeline + + +def define_example_repository(): + return RepositoryDefinition( + name='example_repo', + pipeline_dict={'pandas_hello_world': define_success_pipeline}, + ) diff --git a/python_modules/dagster/dagster/dagster_examples/repository.yml b/python_modules/dagster/dagster/dagster_examples/repository.yml new file mode 100644 index 0000000000000..50925f965f79c --- /dev/null +++ b/python_modules/dagster/dagster/dagster_examples/repository.yml @@ -0,0 +1,3 @@ +repository: + module: dagster.dagster_examples.repository + fn: define_example_repository diff --git a/python_modules/dagster/dagster/graphviz.py b/python_modules/dagster/dagster/graphviz.py index be26fb7bae2b8..3f7a8449a5c85 100644 --- a/python_modules/dagster/dagster/graphviz.py +++ b/python_modules/dagster/dagster/graphviz.py @@ -22,9 +22,9 @@ def build_graphviz_graph(pipeline): for input_def in solid.input_defs: scoped_name = solid.name + '.' + input_def.name graphviz_graph.edge(scoped_name, solid.name) - - if pipeline.dependency_structure.has_dep(solid.name, input_def.name): - output_handle = pipeline.dependency_structure.get_dep(solid.name, input_def.name) - graphviz_graph.edge(output_handle.solid_name, scoped_name) + input_handle = solid.input_handle(input_def.name) + if pipeline.dependency_structure.has_dep(input_handle): + output_handle = pipeline.dependency_structure.get_dep(input_handle) + graphviz_graph.edge(output_handle.solid.name, scoped_name) return graphviz_graph diff --git a/python_modules/dagster/setup.py b/python_modules/dagster/setup.py index 92d3089ca9a01..ffda6ecb4f5ab 100644 --- a/python_modules/dagster/setup.py +++ b/python_modules/dagster/setup.py @@ -18,7 +18,7 @@ def long_description(): setup( name='dagster', - version='0.2.0.dev2', + version='0.2.0.dev3', author='Elementl', author_email='schrockn@elementl.com', license='Apache-2.0', From 4b37265f7d3a4c635ebad973b0b96975debca718 Mon Sep 17 00:00:00 2001 From: Mikhail Novikov Date: Mon, 3 Sep 2018 15:58:14 +0200 Subject: [PATCH 045/103] Python 2.7 and 3.5 tests (#120) Python 2.7 and 3.5 support and tests --- .circleci/config.yml | 81 +++++++++++++++---- python_modules/dagit/dagit/app.py | 9 ++- python_modules/dagit/dagit_tests/__init__.py | 0 .../dagit/dagit_tests/test_smoke.py | 25 ++++++ python_modules/dagit/setup.py | 4 +- python_modules/dagit/tox.ini | 6 +- .../dagster/dagster/check/__init__.py | 10 +++ .../dagster/check/check_tests/test_check.py | 26 +++--- .../dagster/dagster/cli/pipeline.py | 1 + .../dagster/dagster/cli/structured_flags.py | 21 +++-- python_modules/dagster/dagster/config.py | 2 +- .../dagster/dagster/core/argument_handling.py | 6 +- .../dagster/dagster/core/compute_nodes.py | 67 +++++++++------ .../core/core_tests/test_context_logging.py | 17 ++-- .../core_tests/test_pipeline_execution.py | 5 +- .../dagster/core/core_tests/test_types.py | 2 +- .../dagster/dagster/core/decorators.py | 43 ++++++---- .../dagster/dagster/core/definitions.py | 81 +++++++++++-------- python_modules/dagster/dagster/core/errors.py | 8 +- .../dagster/dagster/core/execution.py | 14 ++-- .../dagster/dagster/core/execution_context.py | 13 ++- python_modules/dagster/dagster/core/types.py | 12 +-- .../test_sql_project_pipeline.py | 4 +- .../sql_project_example/pipelines.py | 2 +- .../pandas/pandas_tests/test_pandas_solids.py | 5 +- .../dagster/dagster/sqlalchemy/common.py | 4 +- .../sqlalchemy_tests/math_test_db.py | 8 +- .../sqlalchemy_tests/test_basic_solid.py | 23 +++--- .../test_isolated_templated_sql_tests.py | 4 +- .../sqlalchemy_tests/test_mocked_context.py | 2 +- .../subquery_builder_experimental.py | 8 +- .../dagster/utils/indenting_printer.py | 3 +- .../dagster/dagster/utils/logging.py | 16 ++-- .../dagster/dagster/utils/timing.py | 4 +- .../utils_tests/test_indenting_printer.py | 4 +- python_modules/dagster/setup.py | 1 + python_modules/dagster/tox.ini | 2 +- 37 files changed, 363 insertions(+), 180 deletions(-) create mode 100644 python_modules/dagit/dagit_tests/__init__.py create mode 100644 python_modules/dagit/dagit_tests/test_smoke.py diff --git a/.circleci/config.yml b/.circleci/config.yml index 5798b207f0bb1..20219952ed687 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -1,16 +1,22 @@ version: 2 -jobs: - build: - docker: - # specify the version you desire here - # use `-browsers` prefix for selenium tests, e.g. `3.6.1-browsers` - - image: circleci/python:3.6.1 - # Specify service dependencies here if necessary - # CircleCI maintains a library of pre-built images - # documented at https://circleci.com/docs/2.0/circleci-images/ - # - image: circleci/postgres:9.4 +workflows: + version: 2 + test: + jobs: + - dagster-python-3.6 + - dagster-python-3.5 + - dagster-python-2.7 + - dagit-python-3.6 + - dagit-python-3.5 + - dagit-python-2.7 +jobs: + dagster-python-3.6: &dagster-template + docker: + - image: circleci/python:3.6.6 + environment: + TOXENV: py36 working_directory: ~/repo steps: @@ -19,17 +25,62 @@ jobs: - run: name: Install Dependencies command: | - python3 -m venv venv - . venv/bin/activate - pip install tox + sudo pip install tox - run: name: Run Dagster Tests command: | - . venv/bin/activate cd python_modules/dagster - tox + tox -e $TOXENV + + - store_artifacts: + path: test-reports + destination: test-reports + + dagit-python-3.6: &dagit-template + <<: *dagster-template + steps: + - checkout + + - run: + name: Install Dependencies + command: | + sudo pip install tox + + - run: + name: Run Dagit Tests + command: | + cd python_modules/dagit + tox -e $TOXENV - store_artifacts: path: test-reports destination: test-reports + + dagster-python-3.5: + <<: *dagster-template + docker: + - image: circleci/python:3.5.6 + environment: + TOXENV: py35 + + dagit-python-3.5: + <<: *dagit-template + docker: + - image: circleci/python:3.5.6 + environment: + TOXENV: py35 + + dagster-python-2.7: + <<: *dagster-template + docker: + - image: circleci/python:2.7.15 + environment: + TOXENV: py27 + + dagit-python-2.7: + <<: *dagit-template + docker: + - image: circleci/python:2.7.15 + environment: + TOXENV: py27 diff --git a/python_modules/dagit/dagit/app.py b/python_modules/dagit/dagit/app.py index 949dd45becfc3..36fe9799f4715 100644 --- a/python_modules/dagit/dagit/app.py +++ b/python_modules/dagit/dagit/app.py @@ -1,5 +1,8 @@ import os -from graphql.execution.executors.asyncio import AsyncioExecutor +try: + from graphql.execution.executors.asyncio import AsyncioExecutor as Executor +except ImportError: + from graphql.execution.executors.thread import ThreadExecutor as Executor from flask import Flask, send_file, send_from_directory from flask_graphql import GraphQLView from flask_cors import CORS @@ -13,7 +16,7 @@ from .schema import create_schema -class RepositoryContainer: +class RepositoryContainer(object): ''' This class solely exists to implement reloading semantics. We need to have a single object that the graphql server has access that stays the same object between reload. This container @@ -64,7 +67,7 @@ def create_app(repository_container): 'graphql', schema=schema, graphiql=True, - executor=AsyncioExecutor(), + executor=Executor(), repository_container=repository_container, ) ) diff --git a/python_modules/dagit/dagit_tests/__init__.py b/python_modules/dagit/dagit_tests/__init__.py new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/python_modules/dagit/dagit_tests/test_smoke.py b/python_modules/dagit/dagit_tests/test_smoke.py new file mode 100644 index 0000000000000..39de41d1ccfab --- /dev/null +++ b/python_modules/dagit/dagit_tests/test_smoke.py @@ -0,0 +1,25 @@ +import json +from dagit import app +from dagster.cli.repository_config import RepositoryInfo +from dagster.dagster_examples.repository import define_example_repository + + +def test_smoke_app(): + repository_container = app.RepositoryContainer( + RepositoryInfo( + repository=define_example_repository(), + module=None, + fn_name=None, + fn=None, + module_name=None, + ) + ) + + flask_app = app.create_app(repository_container) + client = flask_app.test_client() + + result = client.post('/graphql', data={'query': 'query { pipelines { name }}'}) + + data = json.loads(result.data.decode('utf-8')) + + assert data == {"data": {"pipelines": [{"name": "pandas_hello_world"}]}} diff --git a/python_modules/dagit/setup.py b/python_modules/dagit/setup.py index 2b1c729c2e2b6..900e17f27d300 100644 --- a/python_modules/dagit/setup.py +++ b/python_modules/dagit/setup.py @@ -18,7 +18,7 @@ def long_description(): setup( name='dagit', - version='0.1.11', + version='0.2.0.dev3', author='Elementl', author_email='schrockn@elementl.com', license='Apache-2.0', @@ -51,7 +51,7 @@ def long_description(): 'click>=6.7', # dagster - 'dagster>=0.1.7', + 'dagster>=0.2.0.dev3', # graphql 'graphql-core>=2.1', diff --git a/python_modules/dagit/tox.ini b/python_modules/dagit/tox.ini index 306a6bcb5f4c3..b60fdbb098324 100644 --- a/python_modules/dagit/tox.ini +++ b/python_modules/dagit/tox.ini @@ -1,8 +1,10 @@ [tox] -envlist = py36 +envlist = py36,py35,py27 [testenv] deps = pytest pytest-runner -commands = pytest +commands = + pip install -e ../dagster + pytest -v diff --git a/python_modules/dagster/dagster/check/__init__.py b/python_modules/dagster/dagster/check/__init__.py index fc8d1becf98c1..24631a34415fb 100644 --- a/python_modules/dagster/dagster/check/__init__.py +++ b/python_modules/dagster/dagster/check/__init__.py @@ -222,6 +222,10 @@ def opt_tuple_param(obj, param_name, default=None): def _check_list_items(obj_list, of_type): for obj in obj_list: + + if of_type is str: + key_type = string_types + if not isinstance(obj, of_type): raise_with_traceback( CheckError( @@ -262,6 +266,12 @@ def type_param(obj, param_name): def _check_key_value_types(obj, key_type, value_type): + if key_type is str: + key_type = string_types + + if value_type is str: + value_type = string_types + for key, value in obj.items(): if key_type and not isinstance(key, key_type): raise_with_traceback( diff --git a/python_modules/dagster/dagster/check/check_tests/test_check.py b/python_modules/dagster/dagster/check/check_tests/test_check.py index 9a6d9841d9602..d250cc34d4340 100644 --- a/python_modules/dagster/dagster/check/check_tests/test_check.py +++ b/python_modules/dagster/dagster/check/check_tests/test_check.py @@ -39,10 +39,10 @@ def test_list_param(): def test_typed_list_param(): - class Foo: + class Foo(object): pass - class Bar: + class Bar(object): pass assert check.list_param([], 'list_param', Foo) == [] @@ -73,10 +73,10 @@ def test_opt_list_param(): def test_opt_typed_list_param(): - class Foo: + class Foo(object): pass - class Bar: + class Bar(object): pass assert check.opt_list_param(None, 'list_param', Foo) == [] @@ -127,7 +127,7 @@ def test_dict_param_with_type(): assert check.dict_param({}, 'str_to_int', key_type=str) == {} assert check.dict_param({}, 'str_to_int') == {} - class Wrong: + class Wrong(object): pass with pytest.raises(CheckError): @@ -163,7 +163,7 @@ def test_opt_dict_param_with_type(): assert check.opt_dict_param(None, 'str_to_int', key_type=str) == {} assert check.opt_dict_param(None, 'str_to_int') == {} - class Wrong: + class Wrong(object): pass with pytest.raises(CheckError): @@ -375,10 +375,10 @@ def test_not_implemented(): def test_inst(): - class Foo: + class Foo(object): pass - class Bar: + class Bar(object): pass obj = Foo() @@ -390,10 +390,10 @@ class Bar: def test_inst_param(): - class Foo: + class Foo(object): pass - class Bar: + class Bar(object): pass obj = Foo() @@ -408,10 +408,10 @@ class Bar: def test_opt_inst_param(): - class Foo: + class Foo(object): pass - class Bar: + class Bar(object): pass obj = Foo() @@ -535,7 +535,7 @@ def test_opt_tuple_param(): def test_type_param(): - class Bar: + class Bar(object): pass assert check.type_param(int, 'foo') diff --git a/python_modules/dagster/dagster/cli/pipeline.py b/python_modules/dagster/dagster/cli/pipeline.py index 7f4887c0b70d1..9bc29ecedd6a5 100644 --- a/python_modules/dagster/dagster/cli/pipeline.py +++ b/python_modules/dagster/dagster/cli/pipeline.py @@ -1,3 +1,4 @@ +from __future__ import print_function import logging import re import textwrap diff --git a/python_modules/dagster/dagster/cli/structured_flags.py b/python_modules/dagster/dagster/cli/structured_flags.py index 47368e60f88ae..2e6a06669bfb9 100644 --- a/python_modules/dagster/dagster/cli/structured_flags.py +++ b/python_modules/dagster/dagster/cli/structured_flags.py @@ -38,7 +38,9 @@ def argument_for_name(self, name, key): value = self._for_name(name, key) if value is None: - raise StructuredArgumentsError(f'Could not find name {name} key {key}') + raise StructuredArgumentsError( + 'Could not find name {name} key {key}'.format(name=name, key=key) + ) return value @@ -91,10 +93,14 @@ def structure_flags(inputs): raise StructuredArgumentsError('Single arg must be alone') if arg.name in named_argument_set: - raise StructuredArgumentsError(f'Cannot specify name twice: {arg.name}') + raise StructuredArgumentsError( + 'Cannot specify name twice: {arg.name}'.format(arg=arg) + ) if arg.name in named_argument_key_dict: - raise StructuredArgumentsError(f'Cannot specify name and keyed value: {arg.name}') + raise StructuredArgumentsError( + 'Cannot specify name and keyed value: {arg.name}'.format(arg=arg) + ) named_arguments.append(arg) named_argument_set.add(arg.name) @@ -103,18 +109,19 @@ def structure_flags(inputs): raise StructuredArgumentsError('Single arg must be alone') if arg.name in named_argument_set: - raise StructuredArgumentsError(f'Cannot specify name twice: {arg.name}') + raise StructuredArgumentsError( + 'Cannot specify name twice: {arg.name}'.format(arg=arg) + ) if arg.key in named_argument_key_dict[arg.name]: raise StructuredArgumentsError( - f'Cannot specify key ({arg.key}) in name ({arg.name}) twice' + 'Cannot specify key ({arg.key}) in name ({arg.name}) twice'.format(arg=arg) ) named_key_arguments.append(arg) named_argument_key_dict[arg.name].add(arg.key) else: - check.failed(f'Not supported type: {type(arg)}') - + check.failed('Not supported type: {type}'.format(type=type(arg))) return StructuredArguments( single_argument=single_argument, named_arguments=named_arguments, diff --git a/python_modules/dagster/dagster/config.py b/python_modules/dagster/dagster/config.py index 237b4f6fa2e54..2fd59b968cb82 100644 --- a/python_modules/dagster/dagster/config.py +++ b/python_modules/dagster/dagster/config.py @@ -22,7 +22,7 @@ def __new__(cls, config_dict): class Environment(namedtuple('EnvironmentData', 'context solids expectations')): - def __new__(cls, *, solids=None, context=None, expectations=None): + def __new__(cls, solids=None, context=None, expectations=None): check.opt_inst_param(context, 'context', Context) if context is None: diff --git a/python_modules/dagster/dagster/core/argument_handling.py b/python_modules/dagster/dagster/core/argument_handling.py index a9ce4dec1c999..1180bfb50c3fe 100644 --- a/python_modules/dagster/dagster/core/argument_handling.py +++ b/python_modules/dagster/dagster/core/argument_handling.py @@ -10,7 +10,7 @@ # 3) make type checks throughout execution cheaper class ArgumentDefinitionDictionary(dict): def __init__(self, ddict): - super().__init__( + super(ArgumentDefinitionDictionary, self).__init__( check.dict_param(ddict, 'ddict', key_type=str, value_type=ArgumentDefinition) ) @@ -18,14 +18,14 @@ def __setitem__(self, _key, _value): check.failed('This dictionary is readonly') -class __ArgumentValueSentinel: +class __ArgumentValueSentinel(object): pass NO_DEFAULT_PROVIDED = __ArgumentValueSentinel -class ArgumentDefinition: +class ArgumentDefinition(object): def __init__( self, dagster_type, default_value=NO_DEFAULT_PROVIDED, is_optional=False, description=None ): diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index 9193ee96ff1d5..10b05694b2de2 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -1,3 +1,4 @@ +from __future__ import print_function from collections import (namedtuple, defaultdict) from contextlib import contextmanager from enum import Enum @@ -71,8 +72,8 @@ def __new__(cls, compute_node, output_name): def __str__(self): return ( - f'ComputeNodeOutputHandle' - f'(guid="{self.compute_node.guid}", output_name="{self.output_name}")' + 'ComputeNodeOutputHandle' + '(guid="{cn.compute_node.guid}", output_name="{cn.output_name}")'.format(cn=self) ) def __hash__(self): @@ -215,7 +216,7 @@ def _execute_core_transform(context, compute_node, values_dict, config_dict): context.metric('core_transform_time_ms', timer_result.millis) -class ComputeNodeInput: +class ComputeNodeInput(object): def __init__(self, name, dagster_type, prev_output_handle): self.name = check.str_param(name, 'name') self.dagster_type = check.inst_param(dagster_type, 'dagster_type', DagsterType) @@ -226,13 +227,13 @@ def __init__(self, name, dagster_type, prev_output_handle): ) -class ComputeNodeOutput: +class ComputeNodeOutput(object): def __init__(self, name, dagster_type): self.name = check.str_param(name, 'name') self.dagster_type = check.inst_param(dagster_type, 'dagster_type', DagsterType) -class ComputeNode: +class ComputeNode(object): def __init__(self, friendly_name, node_inputs, node_outputs, arg_dict, compute_fn, tag, solid): self.guid = str(uuid.uuid4()) self.friendly_name = check.str_param(friendly_name, 'friendly_name') @@ -271,9 +272,10 @@ def _create_compute_node_result(self, result): if not node_output.dagster_type.is_python_valid_value(result.value): raise DagsterInvariantViolationError( - f'''Solid {self.solid.name} output {result.value} + '''Solid {cn.solid.name} output {result.value} which does not match the type for Dagster Type {node_output.dagster_type.name}''' + .format(cn=self, result=result, node_output=node_output) ) return ComputeNodeResult.success_result( @@ -289,16 +291,22 @@ def execute(self, context, inputs): check.inst_param(context, 'context', ExecutionContext) check.dict_param(inputs, 'inputs', key_type=str) - logger.debug(f'Entering execution for {self.friendly_name}') + logger.debug('Entering execution for {self.friendly_name}'.format(self=self)) # do runtime type checks of inputs versus node inputs for input_name, input_value in inputs.items(): compute_node_input = self._node_input_dict[input_name] if not compute_node_input.dagster_type.is_python_valid_value(input_value): raise DagsterInvariantViolationError( - f'''Solid {self.solid.name} input {input_name} + '''Solid {cn.solid.name} input {input_name} received value {input_value} which does not match the type for Dagster type - {compute_node_input.dagster_type.name}. Compute node {self.friendly_name}''' + {compute_node_input.dagster_type.name}. Compute node {cn.friendly_name}''' + .format( + cn=self, + input_name=input_name, + input_value=input_value, + compute_node_input=compute_node_input + ) ) error_str = 'TODO error string' @@ -320,15 +328,16 @@ def execute(self, context, inputs): if not self.has_node(result.output_name): output_names = list([output_def.name for output_def in self.solid.output_defs]) raise DagsterInvariantViolationError( - f'''Core transform for {self.solid.name} returned an output + '''Core transform for {cn.solid.name} returned an output {result.output_name} that does not exist. The available outputs are {output_names}''' + .format(cn=self, result=result, output_names=output_names) ) if result.output_name in seen_outputs: raise DagsterInvariantViolationError( - f'''Core transform for {self.solid.name} returned an output - {result.output_name} multiple times''' + '''Core transform for {cn.solid.name} returned an output + {result.output_name} multiple times'''.format(cn=self, result=result) ) seen_outputs.add(result.output_name) @@ -352,7 +361,7 @@ def output_named(self, name): if node_output.name == name: return node_output - check.failed(f'output {name} not found') + check.failed('output {name} not found'.format(name=name)) def _all_inputs_covered(cn, results): @@ -389,17 +398,25 @@ def print_graph(graph, printer=print): printer = IndentingPrinter(printer=printer) for node in graph.topological_nodes(): - with printer.with_indent(f'Node {node.friendly_name} Id: {node.guid}'): + with printer.with_indent('Node {node.friendly_name} Id: {node.guid}'.format(node=node)): for node_input in node.node_inputs: - with printer.with_indent(f'Input: {node_input.name}'): - printer.line(f'Type: {node_input.dagster_type.name}') - printer.line(f'From: {node_input.prev_output_handle}') + with printer.with_indent('Input: {node_input.name}'.format(node_input=node_input)): + printer.line( + 'Type: {node_input.dagster_type.name}'.format(node_input=node_input) + ) + printer.line( + 'From: {node_input.prev_output_handle}'.format(node_input=node_input) + ) for node_output in node.node_outputs: - with printer.with_indent(f'Output: {node_output.name}'): - printer.line(f'Type: {node_output.dagster_type.name}') + with printer.with_indent( + 'Output: {node_output.name}'.format(node_output=node_output) + ): + printer.line( + 'Type: {node_output.dagster_type.name}'.format(node_output=node_output) + ) -class ComputeNodeGraph: +class ComputeNodeGraph(object): def __init__(self, cn_dict, deps): self.cn_dict = cn_dict self.deps = deps @@ -468,7 +485,9 @@ def create_expectations_cn_graph(solid, inout_def, prev_node_output_handle, tag) expect_compute_node = create_expectation_cn( solid=solid, expectation_def=expectation_def, - friendly_name=f'{solid.name}.{inout_def.name}.expectation.{expectation_def.name}', + friendly_name='{solid.name}.{inout_def.name}.expectation.{expectation_def.name}'.format( + solid=solid, inout_def=inout_def, expectation_def=expectation_def + ), tag=tag, prev_node_output_handle=prev_node_output_handle, inout_def=inout_def, @@ -530,7 +549,9 @@ def create_compute_node_graph(execution_info): check.invariant( dependency_structure.has_dep(input_handle), - f'{input_handle} not found in dependency structure', + '{input_handle} not found in dependency structure'.format( + input_handle=input_handle + ), ) solid_output_handle = dependency_structure.get_dep(input_handle) @@ -700,7 +721,7 @@ def create_compute_node_from_solid_transform(solid, node_inputs, config_args): check.dict_param(config_args, 'config_args', key_type=str) return ComputeNode( - friendly_name=f'{solid.name}.transform', + friendly_name='{solid.name}.transform'.format(solid=solid), node_inputs=node_inputs, node_outputs=[ ComputeNodeOutput(name=output_def.name, dagster_type=output_def.dagster_type) diff --git a/python_modules/dagster/dagster/core/core_tests/test_context_logging.py b/python_modules/dagster/dagster/core/core_tests/test_context_logging.py index 7a326c233cfbc..8d45a5a291aa7 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_context_logging.py +++ b/python_modules/dagster/dagster/core/core_tests/test_context_logging.py @@ -14,22 +14,27 @@ # weird name so that pytest ignores class LoggerForTest(logging.Logger): def __init__(self, name=None): - super().__init__(name if name else str(uuid.uuid4())) + super(LoggerForTest, self).__init__(name if name else str(uuid.uuid4())) self.messages = [] - def debug(self, msg, *args, extra=None, **kwargs): + def debug(self, msg, *args, **kwargs): + extra = kwargs.pop('extra', None) self.messages.append(LogMessage(msg=msg, level=DEBUG, extra=extra)) - def info(self, msg, *args, extra=None, **kwargs): + def info(self, msg, *args, **kwargs): + extra = kwargs.pop('extra', None) self.messages.append(LogMessage(msg=msg, level=INFO, extra=extra)) - def warning(self, msg, *args, extra=None, **kwargs): + def warning(self, msg, *args, **kwargs): + extra = kwargs.pop('extra', None) self.messages.append(LogMessage(msg=msg, level=WARNING, extra=extra)) - def error(self, msg, *args, extra=None, **kwargs): + def error(self, msg, *args, **kwargs): + extra = kwargs.pop('extra', None) self.messages.append(LogMessage(msg=msg, level=ERROR, extra=extra)) - def critical(self, msg, *args, extra=None, **kwargs): + def critical(self, msg, *args, **kwargs): + extra = kwargs.pop('extra', None) self.messages.append(LogMessage(msg=msg, level=CRITICAL, extra=extra)) diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py index e89b783252388..cc2a32fb51015 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py @@ -45,7 +45,10 @@ def transform(_context, args): seen.add(key) passed_rows.append(item) - return [*passed_rows, {name: 'transform_called'}] + result = [] + result.extend(passed_rows) + result.append({name: 'transform_called'}) + return result return transform diff --git a/python_modules/dagster/dagster/core/core_tests/test_types.py b/python_modules/dagster/dagster/core/core_tests/test_types.py index 0f9a7baee2071..c7a7eb97795fe 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_types.py +++ b/python_modules/dagster/dagster/core/core_tests/test_types.py @@ -8,7 +8,7 @@ def test_desc(): def test_python_object_type(): - class Bar: + class Bar(object): pass type_bar = PythonObjectType('Bar', Bar, description='A bar.') diff --git a/python_modules/dagster/dagster/core/decorators.py b/python_modules/dagster/dagster/core/decorators.py index 5d58a2b4fe0ee..593381977a590 100644 --- a/python_modules/dagster/dagster/core/decorators.py +++ b/python_modules/dagster/dagster/core/decorators.py @@ -1,4 +1,8 @@ import inspect +if hasattr(inspect, 'signature'): + funcsigs = inspect +else: + import funcsigs from collections import namedtuple from functools import wraps @@ -39,7 +43,7 @@ def with_context(fn): return _WithContext(fn) -class _WithContext: +class _WithContext(object): def __init__(self, fn): self.fn = fn @@ -48,7 +52,7 @@ def has_context(self): return True -class _Solid: +class _Solid(object): def __init__( self, name=None, @@ -89,7 +93,7 @@ def __call__(self, fn): ) -def solid(*, name=None, inputs=None, output=None, outputs=None, description=None): +def solid(name=None, inputs=None, output=None, outputs=None, description=None): return _Solid(name=name, inputs=inputs, output=output, outputs=outputs, description=description) @@ -107,12 +111,14 @@ def transform(context, args, _config): else: result = fn(**kwargs) if inspect.isgenerator(result): - yield from result + for item in result: + yield item else: if isinstance(result, Result): yield result elif isinstance(result, MultipleResults): - yield from result.results + for item in result.results: + yield item elif len(outputs) == 1: yield Result(value=result, output_name=outputs[0].name) elif result is not None: @@ -131,7 +137,7 @@ class FunctionValidationError(Exception): } def __init__(self, error_type, param=None, missing_names=None, **kwargs): - super().__init__(**kwargs) + super(FunctionValidationError, self).__init__(**kwargs) self.error_type = error_type self.param = param self.missing_names = missing_names @@ -148,20 +154,24 @@ def _validate_transform_fn(solid_name, transform_fn, inputs, expect_context=Fals except FunctionValidationError as e: if e.error_type == FunctionValidationError.TYPES['vararg']: raise DagsterInvalidDefinitionError( - f"solid '{solid_name}' transform function has positional vararg parameter '{e.param}'. Transform functions should only have keyword arguments that match input names and optionally a first positional parameter named 'context'." + "solid '{solid_name}' transform function has positional vararg parameter '{e.param}'. Transform functions should only have keyword arguments that match input names and optionally a first positional parameter named 'context'.". + format(solid_name=solid_name, e=e) ) elif e.error_type == FunctionValidationError.TYPES['missing_name']: raise DagsterInvalidDefinitionError( - f"solid '{solid_name}' transform function has parameter '{e.param}' that is not one of the solid inputs. Transform functions should only have keyword arguments that match input names and optionally a first positional parameter named 'context'." + "solid '{solid_name}' transform function has parameter '{e.param}' that is not one of the solid inputs. Transform functions should only have keyword arguments that match input names and optionally a first positional parameter named 'context'.". + format(solid_name=solid_name, e=e) ) elif e.error_type == FunctionValidationError.TYPES['missing_positional']: raise DagsterInvalidDefinitionError( - f"solid '{solid_name}' transform function do not have required positional parameter '{e.param}'. Transform functions should only have keyword arguments that match input names and optionally a first positional parameter named 'context'." + "solid '{solid_name}' transform function do not have required positional parameter '{e.param}'. Transform functions should only have keyword arguments that match input names and optionally a first positional parameter named 'context'.". + format(solid_name=solid_name, e=e) ) elif e.error_type == FunctionValidationError.TYPES['extra']: undeclared_inputs_printed = ", '".join(e.missing_names) raise DagsterInvalidDefinitionError( - f"solid '{solid_name}' transform function do not have parameter(s) '{undeclared_inputs_printed}', which are in solid's inputs. Transform functions should only have keyword arguments that match input names and optionally a first positional parameter named 'context'." + "solid '{solid_name}' transform function do not have parameter(s) '{undeclared_inputs_printed}', which are in solid's inputs. Transform functions should only have keyword arguments that match input names and optionally a first positional parameter named 'context'.". + format(solid_name=solid_name, undeclared_inputs_printed=undeclared_inputs_printed) ) else: raise e @@ -171,17 +181,20 @@ def _validate_decorated_fn(fn, names, expected_positionals): used_inputs = set() has_kwargs = False - signature = inspect.signature(fn) + signature = funcsigs.signature(fn) params = list(signature.parameters.values()) expected_positional_params = params[0:len(expected_positionals)] other_params = params[len(expected_positionals):] for expected, actual in zip(expected_positionals, expected_positional_params): - possible_names = [expected, f'_{expected}', f'{expected}_'] + possible_names = [ + expected, '_{expected}'.format(expected=expected), + '{expected}_'.format(expected=expected) + ] if ( actual.kind not in [ - inspect.Parameter.POSITIONAL_OR_KEYWORD, inspect.Parameter.POSITIONAL_ONLY + funcsigs.Parameter.POSITIONAL_OR_KEYWORD, funcsigs.Parameter.POSITIONAL_ONLY ] ) or (actual.name not in possible_names): raise FunctionValidationError( @@ -189,9 +202,9 @@ def _validate_decorated_fn(fn, names, expected_positionals): ) for param in other_params: - if param.kind == inspect.Parameter.VAR_KEYWORD: + if param.kind == funcsigs.Parameter.VAR_KEYWORD: has_kwargs = True - elif param.kind == inspect.Parameter.VAR_POSITIONAL: + elif param.kind == funcsigs.Parameter.VAR_POSITIONAL: raise FunctionValidationError(error_type=FunctionValidationError.TYPES['vararg']) else: diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index 7553180d4d8b4..e7f834b5a795c 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -62,7 +62,7 @@ def check_valid_name(name): return name -class PipelineContextDefinition: +class PipelineContextDefinition(object): '''Pipelines declare the different context types they support, in the form of PipelineContextDefinitions. For example a pipeline could declare a context definition for different operating environments: unittest, integration tests, @@ -81,7 +81,7 @@ def passthrough_context_definition(context): ) return {'default': context_definition} - def __init__(self, *, argument_def_dict, context_fn, description=None): + def __init__(self, argument_def_dict, context_fn, description=None): ''' Parameters ---------- @@ -192,7 +192,7 @@ def create_handle_dict(solid_dict, dep_dict): return handle_dict -class DependencyStructure: +class DependencyStructure(object): @staticmethod def from_definitions(solids, dep_dict): return DependencyStructure(create_handle_dict(_build_named_dict(solids), dep_dict)) @@ -243,7 +243,7 @@ def _build_named_dict(things): return ddict -class PipelineDefinition: +class PipelineDefinition(object): ''' A instance of a PipelineDefinition represents a pipeline in dagster. A pipeline is comprised of: @@ -349,12 +349,14 @@ def __validate_dependences(self, dependencies): for from_input, dep in dep_by_input.items(): if from_solid == dep.solid: raise DagsterInvalidDefinitionError( - f'Circular reference detected in solid {from_solid} input {from_input}.' + 'Circular reference detected in solid {from_solid} input {from_input}.'. + format(from_solid=from_solid, from_input=from_input) ) if not from_solid in self._solid_dict: raise DagsterInvalidDefinitionError( - f'Solid {from_solid} in dependency dictionary not found in solid list', + 'Solid {from_solid} in dependency dictionary not found in solid list'. + format(from_solid=from_solid), ) if not self._solid_dict[from_solid].has_input(from_input): @@ -362,18 +364,20 @@ def __validate_dependences(self, dependencies): input_def.name for input_def in self._solid_dict[from_solid].input_defs ] raise DagsterInvalidDefinitionError( - f'Solid {from_solid} does not have input {from_input}. ' + \ - f'Input list: {input_list}' + 'Solid {from_solid} does not have input {from_input}. '.format(from_solid=from_solid, from_input=from_input) + \ + 'Input list: {input_list}'.format(input_list=input_list) ) if not dep.solid in self._solid_dict: raise DagsterInvalidDefinitionError( - f'Solid {dep.solid} in DependencyDefinition not found in solid list', + 'Solid {dep.solid} in DependencyDefinition not found in solid list'.format( + dep=dep + ), ) if not self._solid_dict[dep.solid].has_output(dep.output): raise DagsterInvalidDefinitionError( - f'Solid {dep.solid} does not have output {dep.output}', + 'Solid {dep.solid} does not have output {dep.output}'.format(dep=dep), ) def __validate_dependency_structure(self, name, solids, dependency_structure): @@ -382,13 +386,13 @@ def __validate_dependency_structure(self, name, solids, dependency_structure): if not dependency_structure.has_dep(solid.input_handle(input_def.name)): if name: raise DagsterInvalidDefinitionError( - f'Dependency must be specified for solid {solid.name} input ' + \ - f'{input_def.name} in pipeline {name}' + 'Dependency must be specified for solid {solid.name} input '.format(solid=solid) + \ + '{input_def.name} in pipeline {name}'.format(input_def=input_def, name=name) ) else: raise DagsterInvalidDefinitionError( - f'Dependency must be specified for solid {solid.name} input ' + \ - f'{input_def.name}' + 'Dependency must be specified for solid {solid.name} input '.format(solid=solid) + \ + '{input_def.name}'.format(input_def=input_def) ) def __init__( @@ -453,7 +457,7 @@ def solid_named(self, name): return self._solid_dict[name] -class ExpectationResult: +class ExpectationResult(object): def __init__(self, success, solid=None, message=None, result_context=None): self.success = check.bool_param(success, 'success') self.solid = check.opt_inst_param(solid, SolidDefinition, 'solid') @@ -464,14 +468,14 @@ def copy(self): return copy.deepcopy(self) -class ExpectationDefinition: +class ExpectationDefinition(object): def __init__(self, name, expectation_fn, description=None): self.name = check_valid_name(name) self.expectation_fn = check.callable_param(expectation_fn, 'expectation_fn') self.description = check.opt_str_param(description, 'description') -class InputDefinition: +class InputDefinition(object): '''An InputDefinition instance represents an argument to a transform defined within a solid. Inputs are values within the dagster type system that are created from previous solids. @@ -496,7 +500,7 @@ def __init__(self, name, dagster_type=None, expectations=None, description=None) self.description = check.opt_str_param(description, 'description') -class OutputDefinition: +class OutputDefinition(object): '''An OutputDefinition represents an output from a solid. Solids can have multiple outputs. In those cases the outputs must be named. Frequently solids have only one output, and so the user can construct a single OutputDefinition that will have @@ -532,10 +536,14 @@ def __new__(cls, solid, input_def): ) def __str__(self): - return f'SolidInputHandle(solid="{self.solid.name}", input_name="{self.input_def.name}")' + return 'SolidInputHandle(solid="{self.solid.name}", input_name="{self.input_def.name}")'.format( + self=self + ) def __repr__(self): - return f'SolidInputHandle(solid="{self.solid.name}", input_name="{self.input_def.name}")' + return 'SolidInputHandle(solid="{self.solid.name}", input_name="{self.input_def.name}")'.format( + self=self + ) def __hash__(self): return hash((self.solid.name, self.input_def.name)) @@ -553,10 +561,14 @@ def __new__(cls, solid, output_def): ) def __str__(self): - return f'SolidOutputHandle(solid="{self.solid.name}", output.name="{self.output_def.name}")' + return 'SolidOutputHandle(solid="{self.solid.name}", output.name="{self.output_def.name}")'.format( + self=self + ) def __repr__(self): - return f'SolidOutputHandle(solid="{self.solid.name}", output.name="{self.output_def.name}")' + return 'SolidOutputHandle(solid="{self.solid.name}", output.name="{self.output_def.name}")'.format( + self=self + ) def __hash__(self): return hash((self.solid.name, self.output_def.name)) @@ -583,7 +595,7 @@ def __new__(cls, value, output_name=DEFAULT_OUTPUT): ) -class ConfigDefinition: +class ConfigDefinition(object): '''Solids have config, which determine how they interact with the external world. Example configs would be file paths, database table names, and so forth. @@ -595,7 +607,7 @@ def __init__(self, argument_def_dict): self.argument_def_dict = ArgumentDefinitionDictionary(argument_def_dict) -class SolidDefinition: +class SolidDefinition(object): '''A solid is a node of computation within a pipeline. Parameters: @@ -611,7 +623,7 @@ class SolidDefinition: description: str (optional) ''' - def __init__(self, *, name, inputs, transform_fn, outputs, config_def=None, description=None): + def __init__(self, name, inputs, transform_fn, outputs, config_def=None, description=None): self.name = check_valid_name(name) self.input_defs = check.list_param(inputs, 'inputs', InputDefinition) self.transform_fn = check.callable_param(transform_fn, 'transform_fn') @@ -686,7 +698,7 @@ def output_def_named(self, name): return self._output_dict[name] -class LibrarySolidDefinition: +class LibrarySolidDefinition(object): def __init__(self, name, argument_def_dict, solid_creation_fn): self.name = check.str_param(name, 'name') self.argument_def_dict = ArgumentDefinitionDictionary(argument_def_dict) @@ -705,7 +717,7 @@ def create_solid(self, name, args): return check.inst_param(solid, 'solid', SolidDefinition) -class LibraryDefinition: +class LibraryDefinition(object): def __init__(self, name, library_solids): self.name = check.str_param(name, 'name') self._library_solid_dict = _build_named_dict( @@ -755,7 +767,7 @@ def _dependency_structure_to_dep_dict(dependency_structure): return dep_dict -class ExecutionGraph: +class ExecutionGraph(object): @staticmethod def from_pipeline(pipeline): check.inst_param(pipeline, 'pipeline', PipelineDefinition) @@ -859,8 +871,11 @@ def transitive_dependencies_of(self, solid_name): def _check_solid_name(self, solid_name): check.str_param(solid_name, 'output_name') check.param_invariant( - solid_name in self._solid_dict, 'output_name', - f'Solid {solid_name} must exist in {list(self._solid_dict.keys())}' + solid_name in self._solid_dict, + 'output_name', + 'Solid {solid_name} must exist in {list(self._solid_dict.keys())}'.format( + solid_name=solid_name + ) ) def create_execution_subgraph(self, from_solids, to_solids): @@ -955,7 +970,7 @@ def _create_subgraph(execution_graph, from_solids, through_solids): return execution_graph.create_execution_subgraph(from_solids, through_solids) -class RepositoryDefinition: +class RepositoryDefinition(object): def __init__(self, name, pipeline_dict, libraries=None): self.name = check.str_param(name, 'name') @@ -991,7 +1006,9 @@ def get_pipeline(self, name): pipeline = self.pipeline_dict[name]() check.invariant( pipeline.name == name, - f'Name does not match. Name in dict {name}. Name in pipeline {pipeline.name}' + 'Name does not match. Name in dict {name}. Name in pipeline {pipeline.name}'.format( + name=name, pipeline=pipeline + ) ) check.inst( diff --git a/python_modules/dagster/dagster/core/errors.py b/python_modules/dagster/dagster/core/errors.py index 874888bb80656..5d83d4bde41d3 100644 --- a/python_modules/dagster/dagster/core/errors.py +++ b/python_modules/dagster/dagster/core/errors.py @@ -37,12 +37,14 @@ class DagsterTypeError(DagsterUserError): class DagsterUserCodeExecutionError(DagsterUserError): '''Indicates that user space code has raised an error''' - def __init__(self, *args, user_exception, original_exc_info, **kwargs): + def __init__(self, *args, **kwargs): # original_exc_info should be gotten from a sys.exc_info() call at the # callsite inside of the exception handler. this will allow consuming # code to *re-raise* the user error in it's original format # for cleaner error reporting that does not have framework code in it - super().__init__(*args, **kwargs) + user_exception = kwargs.pop('user_exception', None) + original_exc_info = kwargs.pop('original_exc_info', None) + super(DagsterUserCodeExecutionError, self).__init__(*args, **kwargs) self.user_exception = check.opt_inst_param(user_exception, 'user_exception', Exception) self.original_exc_info = original_exc_info @@ -52,7 +54,7 @@ class DagsterExpectationFailedError(DagsterError): '''Thrown with pipeline configured to throw on expectation failure''' def __init__(self, execution_result, *args, **kwargs): - super().__init__(*args, **kwargs) + super(DagsterExpectationFailedError, self).__init__(*args, **kwargs) # FIXME: need to reorganize to fix this circular dep # Probable fix is to move all "execution result" objects # to definitions diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index 4c0c26608b574..c7bc3cca6bf91 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -47,7 +47,7 @@ from .execution_context import ExecutionContext -class PipelineExecutionResult: +class PipelineExecutionResult(object): def __init__( self, context, @@ -72,8 +72,8 @@ def result_for_solid(self, name): ) -class SolidExecutionResult: - def __init__(self, *, context, solid, input_expectations, transforms, output_expectations): +class SolidExecutionResult(object): + def __init__(self, context, solid, input_expectations, transforms, output_expectations): self.context = check.inst_param(context, 'context', ExecutionContext) self.solid = check.inst_param(solid, 'solid', SolidDefinition) self.input_expectations = check.list_param( @@ -136,7 +136,8 @@ def transformed_value(self, output_name=DEFAULT_OUTPUT): if result.success_data.output_name == output_name: return result.success_data.value check.failed( - f'Did not find result {output_name} in solid {self.solid.name} execution result' + 'Did not find result {output_name} in solid {self.solid.name} execution result'. + format(output_name=output_name, self=self) ) else: return None @@ -182,8 +183,8 @@ def _validate_environment(environment, pipeline): if context_name not in pipeline.context_definitions: avaiable_context_keys = list(pipeline.context_definitions.keys()) - raise DagsterInvariantViolationError(f'Context {context_name} not found in ' + \ - f'pipeline definiton. Available contexts {repr(avaiable_context_keys)}' + raise DagsterInvariantViolationError('Context {context_name} not found in '.format(context_name=context_name) + \ + 'pipeline definiton. Available contexts {avaiable_context_keys}'.format(avaiable_context_keys=repr(avaiable_context_keys)) ) @@ -260,7 +261,6 @@ def _execute_graph_iterator(context, execution_graph, environment): def execute_pipeline( pipeline, environment, - *, throw_on_error=True, ): ''' diff --git a/python_modules/dagster/dagster/core/execution_context.py b/python_modules/dagster/dagster/core/execution_context.py index 3c57d9bf2cf04..94dc1ec3a4a19 100644 --- a/python_modules/dagster/dagster/core/execution_context.py +++ b/python_modules/dagster/dagster/core/execution_context.py @@ -1,3 +1,4 @@ +import itertools import copy from collections import ( OrderedDict, @@ -12,7 +13,7 @@ Metric = namedtuple('Metric', 'context_dict metric_name value') -class ExecutionContext: +class ExecutionContext(object): ''' A context object flowed through the entire scope of single execution of a pipeline of solids. This is used by both framework and user code to log @@ -46,7 +47,7 @@ def _kv_message(self, extra=None): return ' '.join( [ '{key}={value}'.format(key=key, value=self._maybe_quote(value)) - for key, value in [*self._context_dict.items(), *extra.items()] + for key, value in itertools.chain(self._context_dict.items(), extra.items()) ] ) @@ -70,7 +71,11 @@ def _log(self, method, msg, **kwargs): log_props['log_message'] = msg log_props['log_message_id'] = str(uuid.uuid4()) - getattr(self._logger, method)(full_message, extra={**log_props, **kwargs}) + extra = {} + extra.update(log_props) + extra.update(kwargs) + + getattr(self._logger, method)(full_message, extra=extra) def debug(self, msg, **kwargs): return self._log('debug', msg, **kwargs) @@ -147,4 +152,4 @@ def metrics_covering_context(self, needle_dict): def metrics_matching_context(self, needle_dict): for metric in self._metrics: if needle_dict == metric.context_dict: - yield metric \ No newline at end of file + yield metric diff --git a/python_modules/dagster/dagster/core/types.py b/python_modules/dagster/dagster/core/types.py index 8097d2abfa57a..1cfa2c0d81c69 100644 --- a/python_modules/dagster/dagster/core/types.py +++ b/python_modules/dagster/dagster/core/types.py @@ -3,7 +3,7 @@ from dagster import check -class DagsterType: +class DagsterType(object): def __init__(self, name, description=None): self.name = check.str_param(name, 'name') self.description = check.opt_str_param(description, 'description') @@ -14,12 +14,12 @@ def is_python_valid_value(self, _value): class DagsterScalarType(DagsterType): def __init__(self, *args, **kwargs): - super().__init__(*args, **kwargs) + super(DagsterScalarType, self).__init__(*args, **kwargs) class _DagsterAnyType(DagsterType): def __init__(self): - super().__init__( + super(_DagsterAnyType, self).__init__( name='Any', description='The type that allows any value, including no value.' ) @@ -38,7 +38,7 @@ def __init__( python_type, description=None, ): - super().__init__(name, description) + super(PythonObjectType, self).__init__(name, description) self.python_type = check.type_param(python_type, 'python_type') def is_python_valid_value(self, value): @@ -52,7 +52,7 @@ def is_python_valid_value(self, value): class _DagsterIntType(DagsterScalarType): def __init__(self): - super().__init__('Int', description='An integer.') + super(_DagsterIntType, self).__init__('Int', description='An integer.') def is_python_valid_value(self, value): return _nullable_isinstance(value, integer_types) @@ -60,7 +60,7 @@ def is_python_valid_value(self, value): class _DagsterBoolType(DagsterScalarType): def __init__(self): - super().__init__('Bool', description='A boolean.') + super(_DagsterBoolType, self).__init__('Bool', description='A boolean.') def is_python_valid_value(self, value): return _nullable_isinstance(value, bool) diff --git a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py index 5640abe977914..556c9716c006e 100644 --- a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py +++ b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py @@ -22,7 +22,7 @@ def pipeline_engine(pipeline_result): def create_persisted_context(): full_path = script_relative_path('testdb.db') - engine = sa.create_engine(f'sqlite:///{full_path}', echo=False) + engine = sa.create_engine('sqlite:///{full_path}'.format(full_path=full_path), echo=False) return dagster_sa.create_sql_alchemy_context_from_engine(engine=engine) @@ -46,7 +46,7 @@ def create_mem_sql_pipeline_context_tuple(solids, dependencies=None): def _get_sql_script_path(name): - return script_relative_path(f'../../sql_project_example/sql_files/{name}.sql') + return script_relative_path('../../sql_project_example/sql_files/{name}.sql'.format(name=name)) def _get_project_solid(name, inputs=None): diff --git a/python_modules/dagster/dagster/dagster_examples/sql_project_example/pipelines.py b/python_modules/dagster/dagster/dagster_examples/sql_project_example/pipelines.py index c0bee78c5eda7..520a6f6522304 100644 --- a/python_modules/dagster/dagster/dagster_examples/sql_project_example/pipelines.py +++ b/python_modules/dagster/dagster/dagster_examples/sql_project_example/pipelines.py @@ -9,7 +9,7 @@ def _get_sql_script_path(name): - return os.path.join(os.path.dirname(__file__), 'sql_files', f'{name}.sql') + return os.path.join(os.path.dirname(__file__), 'sql_files', '{name}.sql'.format(name=name)) def _get_project_solid(name, inputs=None): diff --git a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py index 78746a97ae016..1f8c6771f6cea 100644 --- a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py +++ b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py @@ -298,7 +298,10 @@ def test_no_transform_solid(): def create_diamond_pipeline(extra_solids=None, extra_dependencies=None): all_solids = list(create_diamond_dag()) + (extra_solids if extra_solids else []) - all_deps = {**create_diamond_deps(), **(extra_dependencies if extra_dependencies else {})} + all_deps = {} + all_deps.update(create_diamond_deps()) + if extra_dependencies: + all_deps.update(extra_dependencies) return PipelineDefinition(solids=all_solids, dependencies=all_deps) diff --git a/python_modules/dagster/dagster/sqlalchemy/common.py b/python_modules/dagster/dagster/sqlalchemy/common.py index aededda9812ae..e59eb558c53f0 100644 --- a/python_modules/dagster/dagster/sqlalchemy/common.py +++ b/python_modules/dagster/dagster/sqlalchemy/common.py @@ -4,13 +4,13 @@ from dagster.core.execution import ExecutionContext -class SqlAlchemyResource: +class SqlAlchemyResource(object): def __init__(self, engine, mock_sql=False): self.engine = check.inst_param(engine, 'engine', sqlalchemy.engine.Engine) self.mock_sql = check.bool_param(mock_sql, 'mock_sql') -class DefaultSqlAlchemyResources: +class DefaultSqlAlchemyResources(object): def __init__(self, sa): self.sa = check.inst_param(sa, 'sa', SqlAlchemyResource) diff --git a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/math_test_db.py b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/math_test_db.py index 5d7e5cad85d98..0bb125781f4ff 100644 --- a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/math_test_db.py +++ b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/math_test_db.py @@ -16,8 +16,12 @@ def create_num_table(engine, num_table_name='num_table'): conn = engine.connect() - conn.execute(f'''INSERT INTO {num_table_name} VALUES(1, 2)''') - conn.execute(f'''INSERT INTO {num_table_name} VALUES(3, 4)''') + conn.execute( + '''INSERT INTO {num_table_name} VALUES(1, 2)'''.format(num_table_name=num_table_name) + ) + conn.execute( + '''INSERT INTO {num_table_name} VALUES(3, 4)'''.format(num_table_name=num_table_name) + ) def in_mem_engine(num_table_name='num_table'): diff --git a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_basic_solid.py b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_basic_solid.py index 9e562d909aad1..ff95bc12e3a8a 100644 --- a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_basic_solid.py +++ b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_basic_solid.py @@ -111,20 +111,21 @@ def create_sum_sq_pipeline(context, expr, extra_solids=None, extra_deps=None): sql_text='SELECT num1, num2, sum, sum * sum as sum_sq from {sum_table}', ) + dependencies = { + sum_solid.name: { + 'num_table': DependencyDefinition('expr'), + }, + sum_sq_solid.name: { + sum_solid.name: DependencyDefinition(sum_solid.name), + } + } + if extra_deps: + dependencies.update(extra_deps) + return pipeline_test_def( solids=[expr_solid, sum_solid, sum_sq_solid] + (extra_solids if extra_solids else []), context=context, - dependencies={ - **{ - sum_solid.name: { - 'num_table': DependencyDefinition('expr'), - }, - sum_sq_solid.name: { - sum_solid.name: DependencyDefinition(sum_solid.name), - }, - }, - **(extra_deps if extra_deps else {}) - }, + dependencies=dependencies ) diff --git a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_templated_sql_tests.py b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_templated_sql_tests.py index 39105362c8c21..888779fd2e894 100644 --- a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_templated_sql_tests.py +++ b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_templated_sql_tests.py @@ -18,7 +18,9 @@ def _load_table(context, table_name): - return context.resources.sa.engine.connect().execute(f'SELECT * FROM {table_name}').fetchall() + return context.resources.sa.engine.connect().execute( + 'SELECT * FROM {table_name}'.format(table_name=table_name) + ).fetchall() def table_name_source(table_name): diff --git a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_mocked_context.py b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_mocked_context.py index f4ab7d7a11442..513a53dc6ab22 100644 --- a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_mocked_context.py +++ b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_mocked_context.py @@ -6,7 +6,7 @@ # pylint: disable=W0221, W0223 class MockEngine(sqlalchemy.engine.Engine): def __init__(self): - super().__init__(None, None, None) + super(MockEngine, self).__init__(None, None, None) def connect(self): raise Exception('should not call') diff --git a/python_modules/dagster/dagster/sqlalchemy/subquery_builder_experimental.py b/python_modules/dagster/dagster/sqlalchemy/subquery_builder_experimental.py index e279e9285fc35..96389f10859b2 100644 --- a/python_modules/dagster/dagster/sqlalchemy/subquery_builder_experimental.py +++ b/python_modules/dagster/dagster/sqlalchemy/subquery_builder_experimental.py @@ -13,7 +13,7 @@ from dagster.sqlalchemy import execute_sql_text_on_context -class DagsterSqlExpression: +class DagsterSqlExpression(object): @property def from_target(self): check.not_implemented('must implemented in subclass') @@ -21,7 +21,7 @@ def from_target(self): class DagsterSqlQueryExpression(DagsterSqlExpression): def __init__(self, subquery_text): - super().__init__() + super(DagsterSqlQueryExpression, self).__init__() self._subquery_text = check.str_param(subquery_text, 'subquery_text') @property @@ -30,12 +30,12 @@ def query_text(self): @property def from_target(self): - return f'({self._subquery_text})' + return '({subquery_text})'.format(subquery_text=self._subquery_text) class DagsterSqlTableExpression(DagsterSqlExpression): def __init__(self, table_name): - super().__init__() + super(DagsterSqlTableExpression, self).__init__() self._table_name = check.str_param(table_name, 'table_name') @property diff --git a/python_modules/dagster/dagster/utils/indenting_printer.py b/python_modules/dagster/dagster/utils/indenting_printer.py index da2e6c716ecb2..d391f7bdbabe2 100644 --- a/python_modules/dagster/dagster/utils/indenting_printer.py +++ b/python_modules/dagster/dagster/utils/indenting_printer.py @@ -1,9 +1,10 @@ +from __future__ import print_function from contextlib import contextmanager from dagster import check -class IndentingPrinter: +class IndentingPrinter(object): def __init__(self, indent_level=2, printer=print): self.lines = [] self.current_indent = 0 diff --git a/python_modules/dagster/dagster/utils/logging.py b/python_modules/dagster/dagster/utils/logging.py index 5183e3f5a3097..f9399b7913c02 100644 --- a/python_modules/dagster/dagster/utils/logging.py +++ b/python_modules/dagster/dagster/utils/logging.py @@ -1,3 +1,4 @@ +from __future__ import absolute_import import copy import json import logging @@ -24,7 +25,7 @@ def level_from_string(string): return LOOKUP[string] -class CompositeLogger: +class CompositeLogger(object): def __init__(self, loggers=None): self.loggers = check.opt_list_param(loggers, 'loggers', of_type=logging.Logger) @@ -39,7 +40,7 @@ def _invoke_logger_method(*args, **kwargs): class JsonFileHandler(logging.Handler): def __init__(self, json_path): - super().__init__() + super(JsonFileHandler, self).__init__() self.json_path = check.str_param(json_path, 'json_path') def emit(self, record): @@ -60,7 +61,7 @@ def define_json_file_logger(name, json_path, level): check.param_invariant( level in VALID_LEVELS, 'level', - f'Must be valid python logging level. Got {level}', + 'Must be valid python logging level. Got {level}'.format(level=level), ) klass = logging.getLoggerClass() @@ -76,7 +77,7 @@ def define_colored_console_logger(name, level=INFO): check.param_invariant( level in VALID_LEVELS, 'level', - f'Must be valid python logging level. Got {level}', + 'Must be valid python logging level. Got {level}'.format(level=level), ) klass = logging.getLoggerClass() @@ -105,4 +106,9 @@ def define_debug_formatter(): def get_formatted_stack_trace(exception): check.inst_param(exception, 'exception', Exception) - return ''.join(traceback.format_tb(exception.__traceback__)) + if hasattr(exception, '__traceback__'): + tb = exception.__traceback__ + else: + import sys + exc_type, exc_value, tb = sys.exc_info() + return ''.join(traceback.format_tb(tb)) diff --git a/python_modules/dagster/dagster/utils/timing.py b/python_modules/dagster/dagster/utils/timing.py index d8dfae9cdb800..8168713d1c7fb 100644 --- a/python_modules/dagster/dagster/utils/timing.py +++ b/python_modules/dagster/dagster/utils/timing.py @@ -4,7 +4,7 @@ from dagster import check -class TimerResult: +class TimerResult(object): def __init__(self): self.start_time = time.time() self.end_time = None @@ -26,7 +26,7 @@ def time_execution_scope(): from solid_util.timing import time_execution_scope with time_execution_scope() as timer_result: do_some_operation() - print(f'do_some_operation took {timer_result.millis} milliseconds') + print('do_some_operation took {timer_result.millis} milliseconds'.format(timer_result=timer_result)) ''' timer_result = TimerResult() diff --git a/python_modules/dagster/dagster/utils/utils_tests/test_indenting_printer.py b/python_modules/dagster/dagster/utils/utils_tests/test_indenting_printer.py index 2d47a067b5cad..277eba0ca738d 100644 --- a/python_modules/dagster/dagster/utils/utils_tests/test_indenting_printer.py +++ b/python_modules/dagster/dagster/utils/utils_tests/test_indenting_printer.py @@ -11,7 +11,7 @@ def _add_line(text): if str is not None: self.lines.append(text) - super().__init__(printer=_add_line, *args, **kwargs) + super(CollectingIndentingPrinter, self).__init__(printer=_add_line, *args, **kwargs) def result(self): return '\n'.join(self.lines) @@ -76,4 +76,4 @@ def test_double_indent(): assert printer.result() == '''test test indent - test double indent''' \ No newline at end of file + test double indent''' diff --git a/python_modules/dagster/setup.py b/python_modules/dagster/setup.py index ffda6ecb4f5ab..39d3d027f326e 100644 --- a/python_modules/dagster/setup.py +++ b/python_modules/dagster/setup.py @@ -36,6 +36,7 @@ def long_description(): # standard python 2/3 compatability things 'enum34>=1.1.6', 'future>=0.16.0', + 'funcsigs>=1.0.2', # cli 'click>=6.7', diff --git a/python_modules/dagster/tox.ini b/python_modules/dagster/tox.ini index 9fb2fd067d2e5..db94648435cd7 100644 --- a/python_modules/dagster/tox.ini +++ b/python_modules/dagster/tox.ini @@ -1,5 +1,5 @@ [tox] -envlist = py36 +envlist = py36,py35,py27 [testenv] deps = From 96047ae1f7d90197eba3314271c7a857acdcf316 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Sat, 1 Sep 2018 14:12:51 -0700 Subject: [PATCH 046/103] Move from "Argument Dictionary" to proper composited type system I decided that this move should be made before 0.2.0. This is, IMO, a much more sound system of managing config. Instead of having "argument dictionaries" everywhere, configs take a top level type to describe themselves. I've implemented one composite type, the accepts a set of Fields and then produced a python dict for implementors to process at runtime. This system has a number of advantages: 1) Configs can be arbitrarily nested and grouped together. This already would have been immediately useful. 2) We can move to strongly-typed APIs for configs. Instead of vending a dict, vend a namedtuple, for example. 3) Even further that 2) this is a step towards a formalized "resource" pattern for configs. Instead of the type producing a dumb struct, you could imagine the config producing a richer "resource" object that manages connections or so forth. Or alternatively a system stacked on top of this one that does that (and requires configs of a certain type). 4) This sets up the ability to support single value config quite easily. I'm not sure if this is a good idea, but it is possible now if we allow it. One clear deficiency is a regression of error messaging. However, we weren't testing error messages properly the entire time! (see https://github.com/dagster-io/dagster/issues/135) So we need to go through and audit type-system-related errors and figure out a way to construct them in this new structure. --- python_modules/.pylintrc | 3 +- python_modules/.vscode/settings.json | 6 +- python_modules/dagster/dagster/__init__.py | 5 +- .../dagster/dagster/cli/pipeline.py | 12 +- .../dagster/dagster/core/argument_handling.py | 120 ------- .../dagster/dagster/core/compute_nodes.py | 34 +- .../core/core_tests/test_argument_handling.py | 151 -------- .../core/core_tests/test_compute_nodes.py | 7 +- .../core_tests/test_config_type_system.py | 322 ++++++++++++++++++ .../core/core_tests/test_custom_context.py | 87 +++-- .../core/core_tests/test_library_solids.py | 51 --- .../core/core_tests/test_naming_collisions.py | 6 +- .../core/core_tests/test_pipeline_errors.py | 4 +- .../core_tests/test_repository_definition.py | 51 --- .../core/core_tests/test_solid_with_config.py | 10 +- .../dagster/dagster/core/decorators.py | 4 +- .../dagster/dagster/core/definitions.py | 158 ++++----- .../dagster/dagster/core/execution.py | 28 +- python_modules/dagster/dagster/core/types.py | 179 +++++++++- .../test_sql_project_pipeline.py | 30 +- .../dagster/dagster/pandas/__init__.py | 14 +- ...est_pandas_hello_world_no_library_slide.py | 10 +- .../sqlalchemy_tests/test_basic_solid.py | 7 +- .../test_isolated_sql_tests.py | 7 +- .../test_isolated_templated_sql_tests.py | 7 +- .../subquery_builder_experimental.py | 6 +- .../dagster/dagster/sqlalchemy/templated.py | 8 +- python_modules/dagster/setup.py | 2 +- 28 files changed, 722 insertions(+), 607 deletions(-) delete mode 100644 python_modules/dagster/dagster/core/argument_handling.py delete mode 100644 python_modules/dagster/dagster/core/core_tests/test_argument_handling.py create mode 100644 python_modules/dagster/dagster/core/core_tests/test_config_type_system.py delete mode 100644 python_modules/dagster/dagster/core/core_tests/test_library_solids.py diff --git a/python_modules/.pylintrc b/python_modules/.pylintrc index 436d7504aa8d0..92eca9f71c415 100644 --- a/python_modules/.pylintrc +++ b/python_modules/.pylintrc @@ -20,4 +20,5 @@ # R1705 Unnecessary else after return # R0902 Too many instance attributes. Sometimes big config objects are what you want # R1710 All returns should return an expression -disable=C0111,C0330,C0103,duplicate-code,R0201,R0903,R0913,W0511,W1201,W1202,R1705,R0902,R1710,C0102 +# C0302 too many lines in single module +disable=C0111,C0330,C0103,duplicate-code,R0201,R0903,R0913,W0511,W1201,W1202,R1705,R0902,R1710,C0102,C0302 diff --git a/python_modules/.vscode/settings.json b/python_modules/.vscode/settings.json index f3b097dd7d55c..bc3b95a833f5a 100644 --- a/python_modules/.vscode/settings.json +++ b/python_modules/.vscode/settings.json @@ -8,7 +8,8 @@ "dagster/dagster/pandas/pandas_tests", "dagster/dagster/sqlalchemy/sqlalchemy_tests", "dagster/dagster/dagster_examples/dagster_examples_tests", - "dagster-ge/dagster_ge_tests" + "dagster-ge/dagster_ge_tests", + "dagit/dagit_tests" ], "python.unitTest.pyTestEnabled": true, "python.pythonPath": "/Users/schrockn/code/venvs/dagster/bin/python", @@ -47,7 +48,8 @@ "**/build": true, "**/spark-warehouse": true, "**/.ipynb_checkpoints": true, - "dist": true + "dist": true, + "**/.tox": true }, "python.venvPath": "/Users/schrockn/venvs", "editor.formatOnSave": true, diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index c4c7317cbb2db..f8cb4d035c1b6 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -5,14 +5,12 @@ from dagster.core.execution_context import ExecutionContext from dagster.core.definitions import ( - ArgumentDefinition, ConfigDefinition, DependencyDefinition, ExpectationDefinition, ExpectationResult, + Field, InputDefinition, - LibraryDefinition, - LibrarySolidDefinition, OutputDefinition, PipelineContextDefinition, PipelineDefinition, @@ -30,6 +28,7 @@ from dagster.core.errors import ( DagsterInvalidDefinitionError, DagsterInvariantViolationError, + DagsterTypeError, ) import dagster.core.types as types diff --git a/python_modules/dagster/dagster/cli/pipeline.py b/python_modules/dagster/dagster/cli/pipeline.py index 9bc29ecedd6a5..6ede7a5ac3e7f 100644 --- a/python_modules/dagster/dagster/cli/pipeline.py +++ b/python_modules/dagster/dagster/cli/pipeline.py @@ -129,15 +129,9 @@ def print_context_definition(printer, context_name, context_definition): print_description(printer, context_definition.description) - printer.line('Args:') - - with printer.with_indent(): - for arg_name, arg_def in context_definition.argument_def_dict.items(): - printer.line('Arg: {name}'.format(name=arg_name)) - with printer.with_indent(): - printer.line('Type: {arg_type}'.format(arg_type=arg_def.dagster_type.name)) - - print_description(printer, arg_def.description) + printer.line( + 'Type: {config_type}'.format(config_type=context_definition.config_def.config_type.name) + ) def print_solid(printer, solid): diff --git a/python_modules/dagster/dagster/core/argument_handling.py b/python_modules/dagster/dagster/core/argument_handling.py deleted file mode 100644 index 1180bfb50c3fe..0000000000000 --- a/python_modules/dagster/dagster/core/argument_handling.py +++ /dev/null @@ -1,120 +0,0 @@ -from dagster import check - -from .errors import DagsterTypeError -from .types import DagsterType - - -# We wrap the passed in dictionary of str : ArgumentDefinition to -# 1) enforce typing -# 2) enforce immutability -# 3) make type checks throughout execution cheaper -class ArgumentDefinitionDictionary(dict): - def __init__(self, ddict): - super(ArgumentDefinitionDictionary, self).__init__( - check.dict_param(ddict, 'ddict', key_type=str, value_type=ArgumentDefinition) - ) - - def __setitem__(self, _key, _value): - check.failed('This dictionary is readonly') - - -class __ArgumentValueSentinel(object): - pass - - -NO_DEFAULT_PROVIDED = __ArgumentValueSentinel - - -class ArgumentDefinition(object): - def __init__( - self, dagster_type, default_value=NO_DEFAULT_PROVIDED, is_optional=False, description=None - ): - '''Definition of an argument passed through the config system. Used in a few different - contexts: to configure a context, to configure a solid, and more to come. - - We have decided to allow arguments to be explictly made *optional* and separate that - concept from the nullability of the type. That means one could have a *required* argument - that is nullable, because sometimes an argument set to null has a distinct semantic meaning - from the lack of an argument. Optional arguments can have default values. Required arguments - cannot. - ''' - if not is_optional: - check.param_invariant( - default_value == NO_DEFAULT_PROVIDED, - 'default_value', - 'required arguments should not specify default values', - ) - - self.dagster_type = check.inst_param(dagster_type, 'dagster_type', DagsterType) - self.description = check.opt_str_param(description, 'description') - self.is_optional = check.bool_param(is_optional, 'is_optional') - self.default_value = default_value - - @property - def default_provided(self): - return self.default_value != NO_DEFAULT_PROVIDED - - -def validate_args(argument_def_dict, arg_dict, error_context_str): - check.inst_param(argument_def_dict, 'argument_def_dict', ArgumentDefinitionDictionary) - check.dict_param(arg_dict, 'arg_dict', key_type=str) - check.str_param(error_context_str, 'error_context_str') - - defined_args = set(argument_def_dict.keys()) - received_args = set(arg_dict.keys()) - - for received_arg in received_args: - if received_arg not in defined_args: - raise DagsterTypeError( - 'Argument {received} not found in {error_context_str}. Defined args: {defined}'. - format( - error_context_str=error_context_str, - defined=repr(defined_args), - received=received_arg, - ) - ) - - for expected_arg, arg_def in argument_def_dict.items(): - if arg_def.is_optional: - continue - - check.invariant(not arg_def.default_provided) - - if expected_arg not in received_args: - raise DagsterTypeError( - 'Did not not find {expected} in {error_context_str}. Defined args: {defined}'. - format( - error_context_str=error_context_str, - expected=expected_arg, - defined=repr(defined_args), - ) - ) - - args_to_pass = {} - - for expected_arg, arg_def in argument_def_dict.items(): - if expected_arg in received_args: - args_to_pass[expected_arg] = arg_dict[expected_arg] - elif arg_def.default_provided: - args_to_pass[expected_arg] = arg_def.default_value - else: - check.invariant(arg_def.is_optional and not arg_def.default_provided) - - for arg_name, arg_value in arg_dict.items(): - arg_def = argument_def_dict[arg_name] - if not arg_def.dagster_type.is_python_valid_value(arg_value): - format_string = ( - 'Expected type {typename} for arg {arg_name} ' + - 'for {error_context_str} but got type "{arg_type}" value {arg_value}' - ) - raise DagsterTypeError( - format_string.format( - typename=arg_def.dagster_type.name, - arg_name=arg_name, - error_context_str=error_context_str, - arg_type=type(arg_value).__name__, - arg_value=repr(arg_value), - ) - ) - - return args_to_pass diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index 10b05694b2de2..b99c99ac0d411 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -21,8 +21,6 @@ from dagster.utils.timing import time_execution_scope -from .argument_handling import validate_args - from .definitions import ( ExecutionGraph, ExpectationDefinition, @@ -38,6 +36,7 @@ from .errors import ( DagsterExpectationFailedError, DagsterInvariantViolationError, + DagsterTypeError, DagsterUserCodeExecutionError, ) @@ -270,12 +269,16 @@ def _create_compute_node_result(self, result): node_output = self.node_named(result.output_name) - if not node_output.dagster_type.is_python_valid_value(result.value): + evaluation_result = node_output.dagster_type.evaluate_value(result.value) + if not evaluation_result.success: raise DagsterInvariantViolationError( - '''Solid {cn.solid.name} output {result.value} - which does not match the type for Dagster Type - {node_output.dagster_type.name}''' - .format(cn=self, result=result, node_output=node_output) + '''Solid {cn.solid.name} output name {output_name} output {result.value} + type failure: {error_msg}'''.format( + cn=self, + result=result, + error_msg=evaluation_result.error_msg, + output_name=result.output_name, + ) ) return ComputeNodeResult.success_result( @@ -283,7 +286,7 @@ def _create_compute_node_result(self, result): tag=self.tag, success_data=ComputeNodeSuccessData( output_name=result.output_name, - value=result.value, + value=evaluation_result.value, ), ) @@ -523,11 +526,16 @@ def validate_config_dict(execution_info, solid): solid_configs = execution_info.environment.solids config_dict = solid_configs[name].config_dict if name in solid_configs else {} - return validate_args( - solid.config_def.argument_def_dict, - config_dict, - 'config for solid {solid_name}'.format(solid_name=name), - ) + evaluation_result = solid.config_def.config_type.evaluate_value(config_dict) + if evaluation_result.success: + return evaluation_result.value + else: + raise DagsterTypeError( + 'Error evaluating config for {solid_name}: {error_msg}'.format( + solid_name=solid.name, + error_msg=evaluation_result.error_msg, + ) + ) def create_compute_node_graph(execution_info): diff --git a/python_modules/dagster/dagster/core/core_tests/test_argument_handling.py b/python_modules/dagster/dagster/core/core_tests/test_argument_handling.py deleted file mode 100644 index 447f3c92c3c65..0000000000000 --- a/python_modules/dagster/dagster/core/core_tests/test_argument_handling.py +++ /dev/null @@ -1,151 +0,0 @@ -import pytest - -from dagster import ( - ArgumentDefinition, - types, -) -from dagster.core.argument_handling import validate_args -from dagster.core.definitions import ArgumentDefinitionDictionary -from dagster.core.errors import DagsterTypeError - - -def _validate(argument_def_dict, arg_dict): - return validate_args(ArgumentDefinitionDictionary(argument_def_dict), arg_dict, 'dummy') - - -def _single_required_string_arg_def_dict(): - return {'string_arg': ArgumentDefinition(types.String)} - - -def _multiple_required_args_def_dict(): - return { - 'arg_one': ArgumentDefinition(types.String), - 'arg_two': ArgumentDefinition(types.String) - } - - -def _single_optional_string_arg_def_dict(): - return {'optional_arg': ArgumentDefinition(types.String, is_optional=True)} - - -def _single_optional_string_arg_def_dict_with_default(): - return { - 'optional_arg': - ArgumentDefinition(types.String, is_optional=True, default_value='some_default') - } - - -def _mixed_required_optional_string_arg_def_dict_with_default(): - return { - 'optional_arg': - ArgumentDefinition(types.String, is_optional=True, default_value='some_default'), - 'required_arg': - ArgumentDefinition(types.String, is_optional=False), - 'optional_arg_no_default': - ArgumentDefinition(types.String, is_optional=True), - } - - -def test_empty(): - _validate({}, {}) - - -def test_required_single_arg_passing(): - _validate(_single_required_string_arg_def_dict(), {'string_arg': 'value'}) - _validate(_single_required_string_arg_def_dict(), {'string_arg': None}) - - -def test_multiple_required_arg_passing(): - _validate(_multiple_required_args_def_dict(), {'arg_one': 'value_one', 'arg_two': 'value_two'}) - _validate(_multiple_required_args_def_dict(), {'arg_one': 'value_one', 'arg_two': None}) - - -def test_single_required_arg_failures(): - with pytest.raises(DagsterTypeError): - _validate(_single_required_string_arg_def_dict(), {}) - - with pytest.raises(DagsterTypeError): - _validate(_single_required_string_arg_def_dict(), {'extra': 'yup'}) - - with pytest.raises(DagsterTypeError): - _validate(_single_required_string_arg_def_dict(), {'string_arg': 'yupup', 'extra': 'yup'}) - - with pytest.raises(DagsterTypeError): - _validate(_single_required_string_arg_def_dict(), {'string_arg': 1}) - - -def test_multiple_required_args_failing(): - with pytest.raises(DagsterTypeError): - _validate(_multiple_required_args_def_dict(), {}) - - with pytest.raises(DagsterTypeError): - _validate(_multiple_required_args_def_dict(), {'arg_one': 'yup'}) - - with pytest.raises(DagsterTypeError): - _validate(_multiple_required_args_def_dict(), {'arg_one': 'yup', 'extra': 'yup'}) - - with pytest.raises(DagsterTypeError): - _validate(_multiple_required_args_def_dict(), {'arg_one': 'value_one', 'arg_two': 2}) - - -def test_single_optional_arg_passing(): - assert _validate(_single_optional_string_arg_def_dict(), {'optional_arg': 'value'}) == { - 'optional_arg': 'value' - } - assert _validate(_single_optional_string_arg_def_dict(), {}) == {} - - assert _validate(_single_optional_string_arg_def_dict(), {'optional_arg': None}) == { - 'optional_arg': None - } - - -def test_single_optional_arg_failing(): - with pytest.raises(DagsterTypeError): - _validate(_single_optional_string_arg_def_dict(), {'optional_arg': 1}) - - with pytest.raises(DagsterTypeError): - _validate(_single_optional_string_arg_def_dict(), {'optional_argdfd': 1}) - - -def test_single_optional_arg_passing_with_default(): - assert _validate(_single_optional_string_arg_def_dict_with_default(), {}) == { - 'optional_arg': 'some_default' - } - - assert _validate( - _single_optional_string_arg_def_dict_with_default(), {'optional_arg': 'override'} - ) == { - 'optional_arg': 'override' - } - - -def test_mixed_args_passing(): - assert _validate( - _mixed_required_optional_string_arg_def_dict_with_default(), { - 'optional_arg': 'value_one', - 'required_arg': 'value_two', - } - ) == { - 'optional_arg': 'value_one', - 'required_arg': 'value_two', - } - - assert _validate( - _mixed_required_optional_string_arg_def_dict_with_default(), { - 'required_arg': 'value_two', - } - ) == { - 'optional_arg': 'some_default', - 'required_arg': 'value_two', - } - - assert _validate( - _mixed_required_optional_string_arg_def_dict_with_default(), { - 'required_arg': 'value_two', - 'optional_arg_no_default': 'value_three', - } - ) == { - 'optional_arg': 'some_default', - 'required_arg': 'value_two', - 'optional_arg_no_default': 'value_three', - } diff --git a/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py b/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py index 4b4e7dccdc6c1..a83aa265cb5f4 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py +++ b/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py @@ -1,4 +1,5 @@ from dagster import ( + ConfigDefinition, ExecutionContext, OutputDefinition, PipelineContextDefinition, @@ -18,10 +19,8 @@ def silencing_default_context(): return { - 'default': - PipelineContextDefinition( - argument_def_dict={}, - context_fn=lambda _pipeline, _args: ExecutionContext(), + 'default': PipelineContextDefinition( + context_fn=lambda _pipeline, _config: ExecutionContext(), ) } diff --git a/python_modules/dagster/dagster/core/core_tests/test_config_type_system.py b/python_modules/dagster/dagster/core/core_tests/test_config_type_system.py new file mode 100644 index 0000000000000..db73ce6a513c9 --- /dev/null +++ b/python_modules/dagster/dagster/core/core_tests/test_config_type_system.py @@ -0,0 +1,322 @@ +from dagster import ( + ConfigDefinition, + types, + Field, +) + + +def test_noop_config(): + assert ConfigDefinition(types.Any) + + +def test_int_field(): + config_def = ConfigDefinition.config_dict({ + 'int_field': Field(types.Int), + }) + + assert config_def.config_type.evaluate_value({'int_field': 1}).value == {'int_field': 1} + + +def test_int_fails(): + config_def = ConfigDefinition.config_dict({ + 'int_field': Field(types.Int), + }) + + assert not config_def.config_type.evaluate_value({'int_field': 'fjkdj'}).success + + assert not config_def.config_type.evaluate_value({'int_field': True}).success + + +def test_default_arg(): + config_def = ConfigDefinition.config_dict( + { + 'int_field': Field(types.Int, default_value=2, is_optional=True), + } + ) + + assert config_def.config_type.evaluate_value({}).value == {'int_field': 2} + + +def _single_required_string_config_dict(): + return ConfigDefinition.config_dict({'string_field': Field(types.String)}) + + +def _multiple_required_fields_config_dict(): + return ConfigDefinition.config_dict( + { + 'field_one': Field(types.String), + 'field_two': Field(types.String), + } + ) + + +def _single_optional_string_config_dict(): + return ConfigDefinition.config_dict({'optional_field': Field(types.String, is_optional=True)}) + + +def _single_optional_string_field_config_dict_with_default(): + optional_field_def = Field( + types.String, + is_optional=True, + default_value='some_default', + ) + return ConfigDefinition.config_dict({'optional_field': optional_field_def}) + + +def _mixed_required_optional_string_config_dict_with_default(): + return ConfigDefinition.config_dict( + { + 'optional_arg': Field( + types.String, + is_optional=True, + default_value='some_default', + ), + 'required_arg': Field(types.String, is_optional=False), + 'optional_arg_no_default': Field(types.String, is_optional=True), + } + ) + + +def _validate(config_def, value): + return config_def.config_type.evaluate_value(value) + + +def test_single_required_string_field_config_type(): + assert _validate(_single_required_string_config_dict(), {'string_field': 'value'}).success + assert _validate(_single_required_string_config_dict(), {'string_field': None}).success + + assert not _validate(_single_required_string_config_dict(), {}).success + + assert not _validate(_single_required_string_config_dict(), {'extra': 'yup'}).success + + assert not _validate( + _single_required_string_config_dict(), { + 'string_field': 'yupup', + 'extra': 'yup' + } + ).success + + assert not _validate(_single_required_string_config_dict(), {'string_field': 1}).success + + +def test_multiple_required_fields_passing(): + assert _validate( + _multiple_required_fields_config_dict(), + { + 'field_one': 'value_one', + 'field_two': 'value_two', + }, + ).success + assert _validate( + _multiple_required_fields_config_dict(), + { + 'field_one': 'value_one', + 'field_two': None, + }, + ).success + + +def test_multiple_required_fields_failing(): + assert not _validate(_multiple_required_fields_config_dict(), {}).success + + assert not _validate(_multiple_required_fields_config_dict(), {'field_one': 'yup'}).success + + assert not _validate( + _multiple_required_fields_config_dict(), { + 'field_one': 'yup', + 'extra': 'yup' + } + ).success + + assert not _validate( + _multiple_required_fields_config_dict(), { + 'field_one': 'value_one', + 'field_two': 2 + } + ).success + + +def test_single_optional_field_passing(): + assert _validate(_single_optional_string_config_dict(), { + 'optional_field': 'value' + }).value == { + 'optional_field': 'value' + } + assert _validate(_single_optional_string_config_dict(), {}).value == {} + + assert _validate(_single_optional_string_config_dict(), { + 'optional_field': None + }).value == { + 'optional_field': None + } + + +def test_single_optional_field_failing(): + assert not _validate(_single_optional_string_config_dict(), {'optional_field': 1}).success + + assert not _validate(_single_optional_string_config_dict(), {'dlkjfalksdjflksaj': 1}).success + + +def test_single_optional_field_passing_with_default(): + assert _validate(_single_optional_string_field_config_dict_with_default(), {}).value == { + 'optional_field': 'some_default' + } + + assert _validate( + _single_optional_string_field_config_dict_with_default(), { + 'optional_field': 'override' + } + ).value == { + 'optional_field': 'override' + } + + +def test_mixed_args_passing(): + assert _validate( + _mixed_required_optional_string_config_dict_with_default(), { + 'optional_arg': 'value_one', + 'required_arg': 'value_two', + } + ).value == { + 'optional_arg': 'value_one', + 'required_arg': 'value_two', + } + + assert _validate( + _mixed_required_optional_string_config_dict_with_default(), { + 'required_arg': 'value_two', + } + ).value == { + 'optional_arg': 'some_default', + 'required_arg': 'value_two', + } + + assert _validate( + _mixed_required_optional_string_config_dict_with_default(), { + 'required_arg': 'value_two', + 'optional_arg_no_default': 'value_three', + } + ).value == { + 'optional_arg': 'some_default', + 'required_arg': 'value_two', + 'optional_arg_no_default': 'value_three', + } + + +def _single_nested_config(): + return ConfigDefinition( + config_type=types.ConfigDictionary( + { + 'nested': + Field(dagster_type=types.ConfigDictionary({ + 'int_field': Field(types.Int) + })), + } + ) + ) + + +def _nested_optional_config_with_default(): + return ConfigDefinition( + config_type=types.ConfigDictionary( + { + 'nested': + Field( + dagster_type=types.ConfigDictionary( + { + 'int_field': Field( + types.Int, + is_optional=True, + default_value=3, + ) + } + ) + ), + } + ) + ) + + +def _nested_optional_config_with_no_default(): + return ConfigDefinition( + config_type=types.ConfigDictionary( + { + 'nested': + Field( + dagster_type=types. + ConfigDictionary({ + 'int_field': Field( + types.Int, + is_optional=True, + ) + }) + ), + } + ) + ) + + +def test_single_nested_config(): + assert _validate(_single_nested_config(), { + 'nested': { + 'int_field': 2 + } + }).value == { + 'nested': { + 'int_field': 2 + } + } + + +def test_single_nested_config_failures(): + assert not _validate(_single_nested_config(), {'nested': 'dkjfdk'}).success + + assert not _validate(_single_nested_config(), {'nested': {'int_field': 'dkjfdk'}}).success + + assert not _validate( + _single_nested_config(), { + 'nested': { + 'int_field': { + 'too_nested': 'dkjfdk' + } + } + } + ).success + + +def test_nested_optional_with_default(): + assert _validate(_nested_optional_config_with_default(), { + 'nested': { + 'int_field': 2 + } + }).value == { + 'nested': { + 'int_field': 2 + } + } + + assert _validate(_nested_optional_config_with_default(), { + 'nested': {} + }).value == { + 'nested': { + 'int_field': 3 + } + } + + +def test_nested_optional_with_no_default(): + assert _validate(_nested_optional_config_with_no_default(), { + 'nested': { + 'int_field': 2 + } + }).value == { + 'nested': { + 'int_field': 2 + } + } + + assert _validate(_nested_optional_config_with_no_default(), { + 'nested': {} + }).value == { + 'nested': {} + } diff --git a/python_modules/dagster/dagster/core/core_tests/test_custom_context.py b/python_modules/dagster/dagster/core/core_tests/test_custom_context.py index d1e59dc8c7b0f..16f9f96c5babb 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_custom_context.py +++ b/python_modules/dagster/dagster/core/core_tests/test_custom_context.py @@ -1,8 +1,9 @@ import pytest from dagster import ( - ArgumentDefinition, + ConfigDefinition, ExecutionContext, + Field, OutputDefinition, PipelineDefinition, PipelineContextDefinition, @@ -59,31 +60,33 @@ def default_context_transform(context): def test_default_value(): - def _get_args_test_solid(arg_name, arg_value): + def _get_config_test_solid(config_key, config_value): @solid( inputs=[], outputs=[OutputDefinition()], ) @with_context - def args_test(context): - assert context.resources == {arg_name: arg_value} + def config_test(context): + assert context.resources == {config_key: config_value} - return args_test + return config_test pipeline = PipelineDefinition( - solids=[_get_args_test_solid('arg_one', 'heyo')], + solids=[_get_config_test_solid('field_one', 'heyo')], context_definitions={ 'custom_one': PipelineContextDefinition( - argument_def_dict={ - 'arg_one': - ArgumentDefinition( - dagster_type=types.String, - is_optional=True, - default_value='heyo', - ) - }, - context_fn=lambda _pipeline, args: ExecutionContext(resources=args), + config_def=ConfigDefinition.config_dict( + { + 'field_one': + Field( + dagster_type=types.String, + is_optional=True, + default_value='heyo', + ) + } + ), + context_fn=lambda _pipeline, config_value: ExecutionContext(resources=config_value), ), } ) @@ -100,32 +103,40 @@ def test_custom_contexts(): ) @with_context def custom_context_transform(context): - assert context.resources == {'arg_one': 'value_two'} + assert context.resources == {'field_one': 'value_two'} pipeline = PipelineDefinition( solids=[custom_context_transform], context_definitions={ 'custom_one': PipelineContextDefinition( - argument_def_dict={'arg_one': ArgumentDefinition(dagster_type=types.String)}, - context_fn=lambda _pipeline, args: ExecutionContext(resources=args), + config_def=ConfigDefinition.config_dict( + { + 'field_one': Field(dagster_type=types.String) + } + ), + context_fn=lambda _pipeline, config_value: ExecutionContext(resources=config_value), ), 'custom_two': PipelineContextDefinition( - argument_def_dict={'arg_one': ArgumentDefinition(dagster_type=types.String)}, - context_fn=lambda _pipeline, args: ExecutionContext(resources=args), + config_def=ConfigDefinition.config_dict( + { + 'field_one': Field(dagster_type=types.String) + } + ), + context_fn=lambda _pipeline, config_value: ExecutionContext(resources=config_value), ) }, ) environment_one = config.Environment( - context=config.Context('custom_one', {'arg_one': 'value_two'}) + context=config.Context('custom_one', {'field_one': 'value_two'}) ) execute_pipeline(pipeline, environment=environment_one) environment_two = config.Environment( - context=config.Context('custom_two', {'arg_one': 'value_two'}) + context=config.Context('custom_two', {'field_one': 'value_two'}) ) execute_pipeline(pipeline, environment=environment_two) @@ -140,13 +151,13 @@ def test_yield_context(): ) @with_context def custom_context_transform(context): - assert context.resources == {'arg_one': 'value_two'} + assert context.resources == {'field_one': 'value_two'} assert context._context_dict['foo'] == 'bar' # pylint: disable=W0212 events.append('during') - def _yield_context(_pipeline, args): + def _yield_context(_pipeline, config_value): events.append('before') - context = ExecutionContext(resources=args) + context = ExecutionContext(resources=config_value) with context.value('foo', 'bar'): yield context events.append('after') @@ -156,14 +167,18 @@ def _yield_context(_pipeline, args): context_definitions={ 'custom_one': PipelineContextDefinition( - argument_def_dict={'arg_one': ArgumentDefinition(dagster_type=types.String)}, + config_def=ConfigDefinition.config_dict( + { + 'field_one': Field(dagster_type=types.String) + } + ), context_fn=_yield_context, ), } ) environment_one = config.Environment( - context=config.Context('custom_one', {'arg_one': 'value_two'}) + context=config.Context('custom_one', {'field_one': 'value_two'}) ) execute_pipeline(pipeline, environment=environment_one) @@ -192,14 +207,14 @@ def never_transform(): throw_on_error=True ) - environment_arg_name_mismatch = config.Environment( + environment_field_name_mismatch = config.Environment( context=config.Context('default', {'unexpected': 'value'}) ) with pytest.raises(DagsterTypeError, message='Argument mismatch in context default'): execute_pipeline( default_context_pipeline, - environment=environment_arg_name_mismatch, + environment=environment_field_name_mismatch, throw_on_error=True ) @@ -208,23 +223,27 @@ def never_transform(): context_definitions={ 'default': PipelineContextDefinition( - argument_def_dict={'string_arg': ArgumentDefinition(types.String)}, - context_fn=lambda _pipeline, _args: _args + config_def=ConfigDefinition.config_dict( + { + 'string_field': Field(types.String) + } + ), + context_fn=lambda _pipeline, _config_value: _config_value ) } ) - environment_no_args_error = config.Environment(context=config.Context('default', {})) + environment_no_config_error = config.Environment(context=config.Context('default', {})) with pytest.raises(DagsterTypeError, message='Argument mismatch in context default'): execute_pipeline( with_argful_context_pipeline, - environment=environment_no_args_error, + environment=environment_no_config_error, throw_on_error=True ) environment_type_mismatch_error = config.Environment( - context=config.Context('default', {'string_arg': 1}) + context=config.Context('default', {'string_field': 1}) ) with pytest.raises(DagsterTypeError, message='Argument mismatch in context default'): diff --git a/python_modules/dagster/dagster/core/core_tests/test_library_solids.py b/python_modules/dagster/dagster/core/core_tests/test_library_solids.py deleted file mode 100644 index af269cce6e5be..0000000000000 --- a/python_modules/dagster/dagster/core/core_tests/test_library_solids.py +++ /dev/null @@ -1,51 +0,0 @@ -from dagster import ( - ArgumentDefinition, - LibrarySolidDefinition, - OutputDefinition, - PipelineDefinition, - Result, - SolidDefinition, - execute_pipeline, - config, - types, -) - - -def test_basic_library_solid(): - def get_t_fn(args): - def _t_fn(*_args, **_kwargs): - yield Result({args['key']: args['value']}) - - return _t_fn - - library_solid_def = LibrarySolidDefinition( - name='return_value_in_key', - argument_def_dict={ - 'key' : ArgumentDefinition(types.String), - 'value' : ArgumentDefinition(types.String), - }, - solid_creation_fn=lambda name, args: SolidDefinition( - name=name, - inputs=[], - outputs=[OutputDefinition()], - transform_fn=get_t_fn(args), - ) - ) - - solid = library_solid_def.create_solid( - 'instance_name', { - 'key': 'some_key', - 'value': 'some_value' - } - ) - - assert isinstance(solid, SolidDefinition) - assert solid.name == 'instance_name' - - pipeline = PipelineDefinition(solids=[solid]) - - result = execute_pipeline(pipeline, config.Environment()) - assert result.success - - solid_result = result.result_for_solid('instance_name') - assert solid_result.transformed_value() == {'some_key': 'some_value'} diff --git a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py index 1aaa6f24eaea6..8cbd72d70e558 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py +++ b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py @@ -1,9 +1,9 @@ import dagster from dagster import ( - ArgumentDefinition, ConfigDefinition, DependencyDefinition, + Field, InputDefinition, OutputDefinition, PipelineDefinition, @@ -28,8 +28,8 @@ def _value_t_fn(_context, _inputs, config_dict): description=description, inputs=[], outputs=[OutputDefinition(dagster_type=types.String)], - config_def=ConfigDefinition({ - 'value': ArgumentDefinition(types.String) + config_def=ConfigDefinition.config_dict({ + 'value': Field(types.String) }), transform_fn=_value_t_fn, ) diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py index 04f7accb2c782..99c0e6e15ef58 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py @@ -21,9 +21,7 @@ def silencing_default_context(): return { - 'default': - PipelineContextDefinition( - argument_def_dict={}, + 'default': PipelineContextDefinition( context_fn=lambda _pipeline, _args: ExecutionContext(), ) } diff --git a/python_modules/dagster/dagster/core/core_tests/test_repository_definition.py b/python_modules/dagster/dagster/core/core_tests/test_repository_definition.py index cd1c76f93402d..edb56f9c92403 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_repository_definition.py +++ b/python_modules/dagster/dagster/core/core_tests/test_repository_definition.py @@ -1,15 +1,9 @@ from collections import defaultdict from dagster import ( - ArgumentDefinition, - LibraryDefinition, - LibrarySolidDefinition, - OutputDefinition, PipelineDefinition, RepositoryDefinition, - Result, SolidDefinition, - types, ) @@ -65,48 +59,3 @@ def test_repo_definition(): pipelines = repo.get_all_pipelines() assert set(['foo', 'bar']) == set([pipeline.name for pipeline in pipelines]) - - -def create_library_solid_def(): - def get_t_fn(args): - def _t_fn(*_args, **_kwargs): - yield Result({args['key']: args['value']}) - - return _t_fn - - return LibrarySolidDefinition( - name='return_value_in_key', - argument_def_dict={ - 'key' : ArgumentDefinition(types.String), - 'value' : ArgumentDefinition(types.String), - }, - solid_creation_fn=lambda name, args: SolidDefinition( - name=name, - inputs=[], - outputs=[OutputDefinition()], - transform_fn=get_t_fn(args), - ) - ) - - -def test_repo_with_libraries(): - called = defaultdict(int) - repo = RepositoryDefinition( - name='some_repo', - pipeline_dict={ - 'foo': lambda: create_single_node_pipeline('foo', called), - 'bar': lambda: create_single_node_pipeline('bar', called), - }, - libraries=[ - LibraryDefinition( - name='some_library', - library_solids=[create_library_solid_def()], - ) - ] - ) - - assert isinstance(repo, RepositoryDefinition) - - -def test_repo_def_errors(): - pass # TODO diff --git a/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py b/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py index 63a2da35b4fea..a2f743bfa74f4 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py +++ b/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py @@ -1,8 +1,8 @@ import pytest from dagster import ( - ArgumentDefinition, ConfigDefinition, + Field, PipelineDefinition, SolidDefinition, config, @@ -23,8 +23,8 @@ def _t_fn(_context, _inputs, config_dict): name='with_context', inputs=[], outputs=[], - config_def=ConfigDefinition({ - 'some_config': ArgumentDefinition(types.String) + config_def=ConfigDefinition.config_dict({ + 'some_config': Field(types.String) }), transform_fn=_t_fn, ) @@ -50,8 +50,8 @@ def _t_fn(_context, _inputs, _config_dict): name='with_context', inputs=[], outputs=[], - config_def=ConfigDefinition({ - 'some_config': ArgumentDefinition(types.String) + config_def=ConfigDefinition.config_dict({ + 'some_config': Field(types.String) }), transform_fn=_t_fn, ) diff --git a/python_modules/dagster/dagster/core/decorators.py b/python_modules/dagster/dagster/core/decorators.py index 593381977a590..5f7da5a258ccc 100644 --- a/python_modules/dagster/dagster/core/decorators.py +++ b/python_modules/dagster/dagster/core/decorators.py @@ -16,6 +16,8 @@ check, ) +from .types import Any + # Error messages are long # pylint: disable=C0301 @@ -71,7 +73,7 @@ def __init__( self.outputs = check.opt_list_param(outputs, 'outputs', OutputDefinition) self.description = check.opt_str_param(description, 'description') - self.config_def = check.opt_inst_param(config_def, 'config_def', ConfigDefinition({})) + self.config_def = check.opt_inst_param(config_def, 'config_def', ConfigDefinition(Any)) def __call__(self, fn): expect_context = getattr(fn, 'has_context', False) diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index e7f834b5a795c..d4c7ef0f8e962 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -14,12 +14,6 @@ define_colored_console_logger, ) -from .argument_handling import ( - validate_args, - ArgumentDefinition, - ArgumentDefinitionDictionary, -) - from .errors import ( DagsterInvalidDefinitionError, DagsterInvariantViolationError, @@ -27,6 +21,11 @@ from .execution_context import ExecutionContext +from .types import ( + DagsterType, + Field, +) + DEFAULT_OUTPUT = 'result' DISALLOWED_NAMES = set( @@ -76,17 +75,15 @@ class PipelineContextDefinition(object): @staticmethod def passthrough_context_definition(context): check.inst_param(context, 'context', ExecutionContext) - context_definition = PipelineContextDefinition( - argument_def_dict={}, context_fn=lambda _pipeline, _args: context - ) + context_definition = PipelineContextDefinition(context_fn=lambda _pipeline, _args: context) return {'default': context_definition} - def __init__(self, argument_def_dict, context_fn, description=None): + def __init__(self, context_fn, config_def=None, description=None): ''' Parameters ---------- - argument_def_dict: str => ArgumentDefinition - Define the arguments expected by the context configuration + config_def: ConfigDefinition + Define the configuration for the context context_fn: callable (pipeline: PipelineDefinition, args: dict str => Any Returns *or* yields an ExecutionContext. @@ -102,7 +99,12 @@ def __init__(self, argument_def_dict, context_fn, description=None): description: str (optional) ''' - self.argument_def_dict = ArgumentDefinitionDictionary(argument_def_dict) + self.config_def = check.opt_inst_param( + config_def, + 'config_def', + ConfigDefinition, + ConfigDefinition(), + ) self.context_fn = check.callable_param(context_fn, 'context_fn') self.description = description @@ -116,10 +118,15 @@ def _default_context_fn(_pipeline, args): return context default_context_def = PipelineContextDefinition( - argument_def_dict={ - 'log_level': - ArgumentDefinition(dagster_type=types.String, is_optional=True, default_value='ERROR') - }, + config_def=ConfigDefinition.config_dict( + { + 'log_level': Field( + dagster_type=types.String, + is_optional=True, + default_value='ERROR', + ) + } + ), context_fn=_default_context_fn, ) return {'default': default_context_def} @@ -364,7 +371,10 @@ def __validate_dependences(self, dependencies): input_def.name for input_def in self._solid_dict[from_solid].input_defs ] raise DagsterInvalidDefinitionError( - 'Solid {from_solid} does not have input {from_input}. '.format(from_solid=from_solid, from_input=from_input) + \ + 'Solid {from_solid} does not have input {from_input}. '.format( + from_solid=from_solid, + from_input=from_input, + ) + \ 'Input list: {input_list}'.format(input_list=input_list) ) @@ -386,12 +396,19 @@ def __validate_dependency_structure(self, name, solids, dependency_structure): if not dependency_structure.has_dep(solid.input_handle(input_def.name)): if name: raise DagsterInvalidDefinitionError( - 'Dependency must be specified for solid {solid.name} input '.format(solid=solid) + \ - '{input_def.name} in pipeline {name}'.format(input_def=input_def, name=name) + 'Dependency must be specified for solid {solid.name} input '.format( + solid=solid + ) + \ + '{input_def.name} in pipeline {name}'.format( + input_def=input_def, + name=name, + ) ) else: raise DagsterInvalidDefinitionError( - 'Dependency must be specified for solid {solid.name} input '.format(solid=solid) + \ + 'Dependency must be specified for solid {solid.name} input '.format( + solid=solid, + ) + \ '{input_def.name}'.format(input_def=input_def) ) @@ -535,15 +552,17 @@ def __new__(cls, solid, input_def): check.inst_param(input_def, 'input_def', InputDefinition), ) - def __str__(self): - return 'SolidInputHandle(solid="{self.solid.name}", input_name="{self.input_def.name}")'.format( - self=self + def _inner_str(self): + return 'SolidInputHandle(solid="{sn}", input_name="{idn}")'.format( + sn=self.solid.name, + idn=self.input_def.name, ) + def __str__(self): + return self._inner_str() + def __repr__(self): - return 'SolidInputHandle(solid="{self.solid.name}", input_name="{self.input_def.name}")'.format( - self=self - ) + return self._inner_str() def __hash__(self): return hash((self.solid.name, self.input_def.name)) @@ -560,15 +579,17 @@ def __new__(cls, solid, output_def): check.inst_param(output_def, 'output_def', OutputDefinition), ) - def __str__(self): - return 'SolidOutputHandle(solid="{self.solid.name}", output.name="{self.output_def.name}")'.format( - self=self + def _inner_str(self): + return 'SolidOutputHandle(solid="{sn}", output.name="{on}")'.format( + sn=self.solid.name, + on=self.output_def.name, ) + def __str__(self): + return self._inner_str() + def __repr__(self): - return 'SolidOutputHandle(solid="{self.solid.name}", output.name="{self.output_def.name}")'.format( - self=self - ) + return self._inner_str() def __hash__(self): return hash((self.solid.name, self.output_def.name)) @@ -596,15 +617,18 @@ def __new__(cls, value, output_name=DEFAULT_OUTPUT): class ConfigDefinition(object): - '''Solids have config, which determine how they interact with the external world. - Example configs would be file paths, database table names, and so forth. + @staticmethod + def config_dict(field_dict): + return ConfigDefinition(types.ConfigDictionary(field_dict)) - Parameters: - ---------- - argument_def_dict: str => ArgumentDefinition''' + def __init__(self, config_type=types.Any): + '''Solids have config, which determine how they interact with the external world. + Example configs would be file paths, database table names, and so forth. - def __init__(self, argument_def_dict): - self.argument_def_dict = ArgumentDefinitionDictionary(argument_def_dict) + Parameters: + ---------- + config_type: DagsterType (optional defaults to types.Any)''' + self.config_type = check.inst_param(config_type, 'config_type', DagsterType) class SolidDefinition(object): @@ -633,7 +657,7 @@ def __init__(self, name, inputs, transform_fn, outputs, config_def=None, descrip config_def, 'config_def', ConfigDefinition, - ConfigDefinition({}), + ConfigDefinition(types.Any), ) input_handles = {} @@ -698,37 +722,6 @@ def output_def_named(self, name): return self._output_dict[name] -class LibrarySolidDefinition(object): - def __init__(self, name, argument_def_dict, solid_creation_fn): - self.name = check.str_param(name, 'name') - self.argument_def_dict = ArgumentDefinitionDictionary(argument_def_dict) - self.solid_creation_fn = check.callable_param(solid_creation_fn, 'solid_creation_fn') - - def create_solid(self, name, args): - check.str_param(name, 'name') - check.dict_param(args, 'args', key_type=str) - - args = validate_args( - self.argument_def_dict, - args, - 'Args of library_solid {name} invalid'.format(name=self.name), - ) - solid = self.solid_creation_fn(name, args) - return check.inst_param(solid, 'solid', SolidDefinition) - - -class LibraryDefinition(object): - def __init__(self, name, library_solids): - self.name = check.str_param(name, 'name') - self._library_solid_dict = _build_named_dict( - check.list_param( - library_solids, - 'library_solids', - of_type=LibrarySolidDefinition, - ) - ) - - def _create_adjacency_lists(solids, dep_structure): check.list_param(solids, 'solids', of_type=SolidDefinition) check.inst_param(dep_structure, 'dep_structure', DependencyStructure) @@ -871,10 +864,9 @@ def transitive_dependencies_of(self, solid_name): def _check_solid_name(self, solid_name): check.str_param(solid_name, 'output_name') check.param_invariant( - solid_name in self._solid_dict, - 'output_name', - 'Solid {solid_name} must exist in {list(self._solid_dict.keys())}'.format( - solid_name=solid_name + solid_name in self._solid_dict, 'output_name', + 'Solid {solid_name} must exist in {solid_names}'.format( + solid_name=solid_name, solid_names=list(self._solid_dict.keys()) ) ) @@ -971,7 +963,7 @@ def _create_subgraph(execution_graph, from_solids, through_solids): class RepositoryDefinition(object): - def __init__(self, name, pipeline_dict, libraries=None): + def __init__(self, name, pipeline_dict): self.name = check.str_param(name, 'name') check.dict_param( @@ -987,18 +979,6 @@ def __init__(self, name, pipeline_dict, libraries=None): self._pipeline_cache = {} - self._library_dict = _build_named_dict( - check.opt_list_param( - libraries, - 'libraries', - of_type=LibraryDefinition, - ) - ) - - def get_library(self, name): - check.str_param(name, 'name') - return self._library_dict[name] - def get_pipeline(self, name): if name in self._pipeline_cache: return self._pipeline_cache[name] diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index c7bc3cca6bf91..0dd12d1df5b83 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -32,9 +32,10 @@ SolidDefinition, ) -from .errors import DagsterInvariantViolationError - -from .argument_handling import validate_args +from .errors import ( + DagsterInvariantViolationError, + DagsterTypeError, +) from .compute_nodes import ( ComputeNodeExecutionInfo, @@ -183,8 +184,11 @@ def _validate_environment(environment, pipeline): if context_name not in pipeline.context_definitions: avaiable_context_keys = list(pipeline.context_definitions.keys()) - raise DagsterInvariantViolationError('Context {context_name} not found in '.format(context_name=context_name) + \ - 'pipeline definiton. Available contexts {avaiable_context_keys}'.format(avaiable_context_keys=repr(avaiable_context_keys)) + raise DagsterInvariantViolationError( + 'Context {context_name} not found in '.format(context_name=context_name) + \ + 'pipeline definiton. Available contexts {avaiable_context_keys}'.format( + avaiable_context_keys=repr(avaiable_context_keys), + ) ) @@ -196,17 +200,15 @@ def yield_context(pipeline, environment): _validate_environment(environment, pipeline) context_name = environment.context.name - context_definition = pipeline.context_definitions[context_name] + config_type = context_definition.config_def.config_type - args_to_pass = validate_args( - pipeline.context_definitions[context_name].argument_def_dict, environment.context.args, - 'pipeline {pipeline_name} context {context_name}'.format( - pipeline_name=pipeline.name, - context_name=context_name, + evaluation_result = config_type.evaluate_value(environment.context.args) + if not evaluation_result.success: + raise DagsterTypeError( + 'Invalid config value: {error_msg}'.format(error_msg=evaluation_result.error_msg) ) - ) - thing = context_definition.context_fn(pipeline, args_to_pass) + thing = context_definition.context_fn(pipeline, evaluation_result.value) return _wrap_in_yield(thing) diff --git a/python_modules/dagster/dagster/core/types.py b/python_modules/dagster/dagster/core/types.py index 1cfa2c0d81c69..9cace20ae3361 100644 --- a/python_modules/dagster/dagster/core/types.py +++ b/python_modules/dagster/dagster/core/types.py @@ -1,3 +1,4 @@ +from collections import namedtuple from six import (string_types, integer_types) from dagster import check @@ -8,7 +9,7 @@ def __init__(self, name, description=None): self.name = check.str_param(name, 'name') self.description = check.opt_str_param(description, 'description') - def is_python_valid_value(self, _value): + def evaluate_value(self, _value): check.not_implemented('Must implement in subclass') @@ -16,6 +17,21 @@ class DagsterScalarType(DagsterType): def __init__(self, *args, **kwargs): super(DagsterScalarType, self).__init__(*args, **kwargs) + def process_value(self, value): + return value + + def is_python_valid_value(self, _value): + raise Exception('must implement') + + def evaluate_value(self, value): + if not self.is_python_valid_value(value): + return IncomingValueResult.create_failure( + 'Expected valid value for {type_name} but got {value}'.format( + type_name=self.name, value=repr(value) + ) + ) + return IncomingValueResult.create_success(value) + class _DagsterAnyType(DagsterType): def __init__(self): @@ -26,8 +42,14 @@ def __init__(self): def is_python_valid_value(self, _value): return True + def process_value(self, value): + return value + + def evaluate_value(self, value): + return IncomingValueResult.create_success(value) -def _nullable_isinstance(value, typez): + +def nullable_isinstance(value, typez): return value is None or isinstance(value, typez) @@ -42,12 +64,21 @@ def __init__( self.python_type = check.type_param(python_type, 'python_type') def is_python_valid_value(self, value): - return _nullable_isinstance(value, self.python_type) + return nullable_isinstance(value, self.python_type) + + def evaluate_value(self, value): + if not self.is_python_valid_value(value): + return IncomingValueResult.create_failure( + 'Expected valid value for {type_name} but got {value}'.format( + type_name=self.name, value=repr(value) + ) + ) + return IncomingValueResult.create_success(value) class _DagsterStringType(DagsterScalarType): def is_python_valid_value(self, value): - return _nullable_isinstance(value, string_types) + return nullable_isinstance(value, string_types) class _DagsterIntType(DagsterScalarType): @@ -55,7 +86,9 @@ def __init__(self): super(_DagsterIntType, self).__init__('Int', description='An integer.') def is_python_valid_value(self, value): - return _nullable_isinstance(value, integer_types) + if isinstance(value, bool): + return False + return nullable_isinstance(value, integer_types) class _DagsterBoolType(DagsterScalarType): @@ -63,7 +96,141 @@ def __init__(self): super(_DagsterBoolType, self).__init__('Bool', description='A boolean.') def is_python_valid_value(self, value): - return _nullable_isinstance(value, bool) + return nullable_isinstance(value, bool) + + +class __FieldValueSentinel: + pass + + +FIELD_NO_DEFAULT_PROVIDED = __FieldValueSentinel + + +class Field: + def __init__( + self, + dagster_type, + default_value=FIELD_NO_DEFAULT_PROVIDED, + is_optional=False, + description=None + ): + if not is_optional: + check.param_invariant( + default_value == FIELD_NO_DEFAULT_PROVIDED, + 'default_value', + 'required arguments should not specify default values', + ) + + self.dagster_type = check.inst_param(dagster_type, 'dagster_type', DagsterType) + self.description = check.opt_str_param(description, 'description') + self.is_optional = check.bool_param(is_optional, 'is_optional') + self.default_value = default_value + + @property + def default_provided(self): + return self.default_value != FIELD_NO_DEFAULT_PROVIDED + + +class FieldDefinitionDictionary(dict): + def __init__(self, ddict): + check.dict_param(ddict, 'ddict', key_type=str, value_type=Field) + super(FieldDefinitionDictionary, self).__init__(ddict) + + def __setitem__(self, _key, _value): + check.failed('This dictionary is readonly') + + +class DagsterCompositeType(DagsterType): + def __init__(self, name, fields, ctor, description=None): + self.field_dict = FieldDefinitionDictionary(fields) + self.ctor = check.callable_param(ctor, 'ctor') + super(DagsterCompositeType, self).__init__(name, description) + + def evaluate_value(self, value): + if not isinstance(value, dict): + return IncomingValueResult.create_failure('Incoming value for composite must be dict') + return process_incoming_composite_value(self, value, self.ctor) + + +class ConfigDictionary(DagsterCompositeType): + def __init__(self, fields): + super(ConfigDictionary, self).__init__( + 'ConfigDictionary', + fields, + lambda val: val, + '''Configuration dictionary. + Typed-checked but then passed to implementations as a python dict''', + ) + + +class IncomingValueResult(namedtuple('_IncomingValueResult', 'success value error_msg')): + def __new__(cls, success, value, error_msg): + return super(IncomingValueResult, cls).__new__( + cls, + check.bool_param(success, 'success'), + value, + check.opt_str_param(error_msg, 'error_msg'), + ) + + @staticmethod + def create_success(value): + return IncomingValueResult(success=True, value=value, error_msg=None) + + @staticmethod + def create_failure(error_msg): + return IncomingValueResult(success=False, value=None, error_msg=error_msg) + + +def process_incoming_composite_value(dagster_composite_type, incoming_value, ctor): + check.inst_param(dagster_composite_type, 'dagster_composite_type', DagsterCompositeType) + check.dict_param(incoming_value, 'incoming_value', key_type=str) + # check.str_param(error_context_str, 'error_context_str') + check.callable_param(ctor, 'ctor') + + field_dict = dagster_composite_type.field_dict + + defined_args = set(field_dict.keys()) + received_args = set(incoming_value.keys()) + + for received_arg in received_args: + if received_arg not in defined_args: + return IncomingValueResult.create_failure( + 'Field {received} not found. Defined fields: {defined}'.format( + defined=repr(defined_args), + received=received_arg, + ) + ) + + for expected_field, field_def in field_dict.items(): + if field_def.is_optional: + continue + + check.invariant(not field_def.default_provided) + + if expected_field not in received_args: + return IncomingValueResult.create_failure( + 'Did not not find {expected}. Defined fields: {defined}'.format( + expected=expected_field, + defined=repr(defined_args), + ) + ) + + fields_to_pass = {} + + for expected_field, field_def in field_dict.items(): + if expected_field in received_args: + evaluation_result = field_def.dagster_type.evaluate_value( + incoming_value[expected_field] + ) + if not evaluation_result.success: + return evaluation_result + fields_to_pass[expected_field] = evaluation_result.value + elif field_def.default_provided: + fields_to_pass[expected_field] = field_def.default_value + else: + check.invariant(field_def.is_optional and not field_def.default_provided) + + return IncomingValueResult.create_success(ctor(fields_to_pass)) String = _DagsterStringType(name='String', description='A string.') diff --git a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py index 556c9716c006e..000a3dcd79782 100644 --- a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py +++ b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py @@ -1,6 +1,14 @@ import sqlalchemy as sa -import dagster -from dagster import (config, InputDefinition, DependencyDefinition) + +from dagster import ( + DependencyDefinition, + InputDefinition, + PipelineContextDefinition, + PipelineDefinition, + config, + execute_pipeline, +) + import dagster.sqlalchemy as dagster_sa from dagster.utils import script_relative_path @@ -27,15 +35,11 @@ def create_persisted_context(): def create_mem_sql_pipeline_context_tuple(solids, dependencies=None): - default_def = dagster.PipelineContextDefinition( - argument_def_dict={}, - context_fn=lambda _pipeline, _args: in_mem_context(), - ) - persisted_def = dagster.PipelineContextDefinition( - argument_def_dict={}, + default_def = PipelineContextDefinition(context_fn=lambda _pipeline, _args: in_mem_context(), ) + persisted_def = PipelineContextDefinition( context_fn=lambda _pipeline, _args: create_persisted_context(), ) - return dagster.PipelineDefinition( + return PipelineDefinition( solids=solids, dependencies=dependencies, context_definitions={ @@ -58,7 +62,7 @@ def test_sql_create_tables(): pipeline = create_mem_sql_pipeline_context_tuple(solids=[create_all_tables_solids]) - pipeline_result = dagster.execute_pipeline(pipeline, environment=config.Environment.empty()) + pipeline_result = execute_pipeline(pipeline, environment=config.Environment.empty()) assert pipeline_result.success assert set(pipeline_engine(pipeline_result).table_names()) == set( @@ -82,7 +86,7 @@ def test_sql_populate_tables(): } ) - pipeline_result = dagster.execute_pipeline(pipeline, environment=config.Environment.empty()) + pipeline_result = execute_pipeline(pipeline, environment=config.Environment.empty()) assert pipeline_result.success @@ -133,7 +137,7 @@ def create_full_pipeline(): def test_full_in_memory_pipeline(): pipeline = create_full_pipeline() - pipeline_result = dagster.execute_pipeline(pipeline, environment=config.Environment.empty()) + pipeline_result = execute_pipeline(pipeline, environment=config.Environment.empty()) assert pipeline_result.success engine = pipeline_engine(pipeline_result) @@ -144,7 +148,7 @@ def test_full_in_memory_pipeline(): def test_full_persisted_pipeline(): pipeline = create_full_pipeline() - pipeline_result = dagster.execute_pipeline( + pipeline_result = execute_pipeline( pipeline, environment=config.Environment(context=config.Context(name='persisted', args={})) ) diff --git a/python_modules/dagster/dagster/pandas/__init__.py b/python_modules/dagster/dagster/pandas/__init__.py index 6922c8f9a1885..cd4c3504018b6 100644 --- a/python_modules/dagster/dagster/pandas/__init__.py +++ b/python_modules/dagster/dagster/pandas/__init__.py @@ -4,8 +4,8 @@ import pandas as pd from dagster import ( - ArgumentDefinition, ConfigDefinition, + Field, ExecutionContext, InputDefinition, OutputDefinition, @@ -39,8 +39,8 @@ def _t_fn(_context, _inputs, config_dict): inputs=[], outputs=[OutputDefinition(dagster_type=DataFrame)], transform_fn=_t_fn, - config_def=ConfigDefinition({ - 'path': ArgumentDefinition(types.Path), + config_def=ConfigDefinition.config_dict({ + 'path': Field(types.Path), }), ) @@ -53,8 +53,8 @@ def _t_fn(_context, inputs, config_dict): name=name, inputs=[InputDefinition('df', DataFrame)], outputs=[], - config_def=ConfigDefinition({ - 'path': ArgumentDefinition(types.Path) + config_def=ConfigDefinition.config_dict({ + 'path': Field(types.Path) }), transform_fn=_t_fn, ) @@ -68,8 +68,8 @@ def _t_fn(_context, inputs, config_dict): name=name, inputs=[InputDefinition('df', DataFrame)], outputs=[], - config_def=ConfigDefinition({ - 'path': ArgumentDefinition(types.Path) + config_def=ConfigDefinition.config_dict({ + 'path': Field(types.Path) }), transform_fn=_t_fn, ) diff --git a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_no_library_slide.py b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_no_library_slide.py index fead0e8804c19..9db3abd9ede6e 100644 --- a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_no_library_slide.py +++ b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_no_library_slide.py @@ -1,9 +1,9 @@ import pandas as pd from dagster import ( - ArgumentDefinition, ConfigDefinition, DependencyDefinition, + Field, InputDefinition, OutputDefinition, PipelineDefinition, @@ -25,8 +25,8 @@ def _t_fn(_context, _inputs, config_dict): name=name, inputs=[], outputs=[OutputDefinition()], - config_def=ConfigDefinition({ - 'path': ArgumentDefinition(types.Path) + config_def=ConfigDefinition.config_dict({ + 'path': Field(types.Path) }), transform_fn=_t_fn ) @@ -40,8 +40,8 @@ def _t_fn(_context, inputs, config_dict): name=name, inputs=[InputDefinition('df')], outputs=[], - config_def=ConfigDefinition({ - 'path': ArgumentDefinition(types.Path) + config_def=ConfigDefinition.config_dict({ + 'path': Field(types.Path) }), transform_fn=_t_fn, ) diff --git a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_basic_solid.py b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_basic_solid.py index ff95bc12e3a8a..6086709fd7d48 100644 --- a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_basic_solid.py +++ b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_basic_solid.py @@ -24,11 +24,8 @@ def pipeline_test_def(solids, context, dependencies): return PipelineDefinition( solids=solids, context_definitions={ - 'default': - PipelineContextDefinition( - argument_def_dict={}, - context_fn=lambda _pipeline, _args: context, - ), + 'default': PipelineContextDefinition(context_fn=lambda _pipeline, _args: context, + ), }, dependencies=dependencies, ) diff --git a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_sql_tests.py b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_sql_tests.py index cad0cc066c9c3..38bb1a89e3723 100644 --- a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_sql_tests.py +++ b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_sql_tests.py @@ -20,11 +20,8 @@ def pipeline_test_def(solids, context, dependencies=None): return PipelineDefinition( solids=solids, context_definitions={ - 'default': - PipelineContextDefinition( - argument_def_dict={}, - context_fn=lambda _pipeline, _args: context, - ), + 'default': PipelineContextDefinition(context_fn=lambda _pipeline, _args: context, + ), }, dependencies=dependencies, ) diff --git a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_templated_sql_tests.py b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_templated_sql_tests.py index 888779fd2e894..cfec8ecb52326 100644 --- a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_templated_sql_tests.py +++ b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_templated_sql_tests.py @@ -32,11 +32,8 @@ def pipeline_test_def(solids, context, dependencies=None): return PipelineDefinition( solids=solids, context_definitions={ - 'default': - PipelineContextDefinition( - argument_def_dict={}, - context_fn=lambda _pipeline, _args: context, - ), + 'default': PipelineContextDefinition(context_fn=lambda _pipeline, _args: context, + ), }, dependencies=dependencies, ) diff --git a/python_modules/dagster/dagster/sqlalchemy/subquery_builder_experimental.py b/python_modules/dagster/dagster/sqlalchemy/subquery_builder_experimental.py index 96389f10859b2..25490e12811b7 100644 --- a/python_modules/dagster/dagster/sqlalchemy/subquery_builder_experimental.py +++ b/python_modules/dagster/dagster/sqlalchemy/subquery_builder_experimental.py @@ -1,8 +1,8 @@ import os from dagster import ( - ArgumentDefinition, ConfigDefinition, + Field, InputDefinition, OutputDefinition, SolidDefinition, @@ -62,8 +62,8 @@ def _materialization_fn(context, inputs, config_dict): inputs=[InputDefinition('expr')], outputs=[], transform_fn=_materialization_fn, - config_def=ConfigDefinition({ - 'table_name': ArgumentDefinition(types.String), + config_def=ConfigDefinition.config_dict({ + 'table_name': Field(types.String), }), ) diff --git a/python_modules/dagster/dagster/sqlalchemy/templated.py b/python_modules/dagster/dagster/sqlalchemy/templated.py index 619ffc030f1d0..b9e9e09efbca5 100644 --- a/python_modules/dagster/dagster/sqlalchemy/templated.py +++ b/python_modules/dagster/dagster/sqlalchemy/templated.py @@ -1,8 +1,8 @@ import jinja2 from dagster import ( - ArgumentDefinition, ConfigDefinition, + Field, InputDefinition, OutputDefinition, Result, @@ -23,14 +23,14 @@ def create_templated_sql_transform_solid(name, sql, table_arguments, dependant_s dependant_solids, 'dependant_solids', of_type=SolidDefinition ) - argument_def_dict = {} + field_dict = {} for table in table_arguments: - argument_def_dict[table] = ArgumentDefinition(types.String) + field_dict[table] = Field(types.String) return SolidDefinition( name=name, inputs=[InputDefinition(solid.name) for solid in dependant_solids], - config_def=ConfigDefinition(argument_def_dict), + config_def=ConfigDefinition.config_dict(field_dict), transform_fn=_create_templated_sql_transform_with_output(sql), outputs=[OutputDefinition()], ) diff --git a/python_modules/dagster/setup.py b/python_modules/dagster/setup.py index 39d3d027f326e..cbbb52692b33c 100644 --- a/python_modules/dagster/setup.py +++ b/python_modules/dagster/setup.py @@ -18,7 +18,7 @@ def long_description(): setup( name='dagster', - version='0.2.0.dev3', + version='0.2.0.dev5', author='Elementl', author_email='schrockn@elementl.com', license='Apache-2.0', From fd5e6b69f00347ef78ca649c41f74dbac88b0352 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Thu, 30 Aug 2018 13:12:29 -0700 Subject: [PATCH 047/103] Documentation Checkpoint This is a starting point for documentation. Two major outputs here. 1) A new intro laying out core dagster ideology. 2) Docstrings for major definition classes. For docstrings I settled on the google style docstrings. See section 3.8 of https://github.com/google/styleguide/blob/gh-pages/pyguide.md --- README.rst | 289 ++++++++++ assets/Dagster.png | Bin 0 -> 219118 bytes python_modules/dagster/README.rst | 532 ------------------ .../dagster/dagster/core/definitions.py | 190 ++++--- python_modules/dagster/docs/index.rst | 2 +- 5 files changed, 396 insertions(+), 617 deletions(-) create mode 100644 README.rst create mode 100644 assets/Dagster.png diff --git a/README.rst b/README.rst new file mode 100644 index 0000000000000..45209101158bc --- /dev/null +++ b/README.rst @@ -0,0 +1,289 @@ +.. image:: https://user-images.githubusercontent.com/28738937/44878798-b6e17e00-ac5c-11e8-8d25-2e47e5a53418.png + :align: center + +.. docs-include + +============ +Introduction +============ + +Dagster is an opinionated system and programming model for data pipelines. This process goes by many names -- ETL (extract-load-transform), ELT (extract-transform-load), model production, data integration, and so on -- but in essence they all describe the same activity: Performing a set of computations structured as a DAG (directed, acyclic graph) that end up producing data assets, whether those assets be tables, files, machine-learning models, etc. + +There are a few tools in this repo + +This repo has a few primary components: + +- **Dagster**: The core programming model and abstraction stack; a stateless single-node and -process execution engine; and a CLI tool for driving that engine. +* **Dagit**: Dagit is a rich viewer for Dagster assets. +* **Dagster GE**: A Dagster integration with Great Expectations. (see https://github.com/great-expectations/great_expectations) + +------------------- +Opinions and Values +------------------- + +As noted above Dagster has a point of view and values regarding how data pipelines should be built and structured. We list them in no particular order: + +* **Functional Data Pipelines**: We believe that data pipelines should be organized as DAGs of functional, idempotent computations. These computations injest input, do computation, and produce output, either with no side effects or well-known, un-externalized side effects. Given the same inputs and configuration, the computation should always produce the same output. These computations should also be parameterizable, so that they can execute in different environments. See https://bit.ly/2LxDgnr for an excellent overview of functional programing in batch computations. +* **Self-describing**: Pipelines should be self-describing with rich metadata and types. Users should be able to approach a pipeline, and use tooling to inspect the pipelines for their structure and capabilities. This metadata should be co-located with actual code of the pipeline. Documentation and code is delivered as a single artifact. +* **Compute-agnostic**: Dagster has opinions about the structure and best practices of data pipelines. It has no opinions about what libraries and engines use to do actual compute. The core computation within a dagster pipeline is user-specified code, meaning that it could be anything: Pandas, Spark, SQL computations on a data warehouse, vanilla python, or any combination therein. +* **Testable by design and by default**: Testing data pipelines is notoriously difficult. Because it is so difficult it is often never done, or done poorly. Dagster pipelines are designed to be tested. They have explicit support for pipeline authors to manage and maintain multiple operating environments -- for example, unit testing, integration testing, and production environments, among others. In addition dagster can execute arbitrary subsets and nodes of the pipeline, critical testability. (This capability happens to be useful in operational contexts as well). +* **First-class data quality tests**: Testing code is important in data pipelines, but it is not sufficient. Data quality tests -- run during every meaningful stage of production -- are critical to reduce the maintenance burden of data pipelines. Pipeline authors generally do not have control of their input data, and make many implicit assumptions about that data. The data formats can also change over time. In order to control this entropy, Dagster encourages users to computationally verify assumptions (known as expectations) about the data as part of the piplien process. This way if those assumptions are broken, the breakage can be reported quickly, easily, and with rich metadata and diagnostic information. These expectations can also serve as contracts between teams. See https://bit.ly/2mxDS1R for a primer on pipeline tests for data quality. +* **Gradual, optional typing**: Dagster contains a type system to describe the values flowing through the pipeline. This allows nodes in a pipeline know if they are compatible before execution, and serves as value documentation and runtime error checking. + + +The core abstraction in Dagster is a *solid*, a logical unit of computation in a data pipeline. At its core a solid is a configurable function that accepts abstract inputs and produces outputs. + +* Inputs: +* Outputs: +* Configuration: +* Transform: + +Inputs are things. + +* Name: +* Type: +* Expectations: + +Outputs are things: + +* Name +* Type +* Expectations + +Solids are group together in *pipelines*. Pipelines are comprised of: + +* Solids +* Dependencies +* Context Definitions + + + +Solid formally separates the notion of inputs, outputs, the core transform. The core goal of this is cleanly and clearly separate the domain logic of each transform and its operational environments. This allows solids or pipelines of solids to easily be executable in a number of environments: unit-testing, local development, integration tests, CI/CD, staging environments, production environments, and so on and so forth. + +Alongside with the core abstraction, Dagster provides helpers to create Solids that operate on Pandas dataframes and SQL databases. + +Example +------- + + + +.. code-block:: python + + + import pandas as pd + import dagster.core + from dagster.core.definitions import ( + InputDefinition, + OutputDefinition + ) + from dagster.core.decorators import ( + solid, + ) + + # Solids can be created by annotating transform function with + # a decorator + @solid( + # Solid inputs define arguments passed to transform function + inputs=[ + InputDefinition( + name='num', + + ) + ], + # Solid output determines what solid should return. + output=OutputDefinition(materializations=[ + dataframe_to_csv_materialization, + ]) + ) + def sum_solid(num): + sum_df = num.copy() + # Here we add a new column to dataframe to sum up num1 and + # num2 columns + sum_df['sum'] = sum_df['num1'] + sum_df['num2'] + return sum_df + + + @solid( + inputs=[ + InputDefinition(name="sum") + ], + output=OutputDefinition(materializations=[ + dataframe_to_csv_materialization, + ]) + ) + def sum_sq_solid(sum): + sum_sq = sum.copy() + sum_sq['sum_sq'] = sum['sum']**2 + return sum_sq + + + # After definining a solid, we are grouping them into a pipeline + pipeline = dagster.core.pipeline( + name='pandas_hello_world', + solids=[ + sum_solid, + sum_sq_solid, + ], + ) + +You might notice that there is no actual CSV file specified as inputs. This is because such parameters are passed in environment. This allows you to customize it in runtime. To run your solid, we'll pass that environment to the execution function. + +.. code-block:: python + + pipeline_result = dagster.execute_pipeline(pipeline, environment) + + +We can simplify the above example by using built-in dagster pandas inputs and outputs. + +.. code-block:: python + + # TODO updated example definition + +We can specify in order to get artifacts for the results. We can materialize output from any solid, this can be useful to see if intermediate results make sense. + +.. code-block:: python + + # TODO updated example config driving + +Dagster CLI +=========== + +In addition to programmatic API, you can also use dagster CLI to run the pipelines. In that case the environment is specified through yaml configuration files. + +The folder structure should be as follows. + +.. code-block + + pipeline_project_name/ + pipelines.yml + pipeline_module_1/ + env.yml + pipeline_module_2/ + env.yml + +Pipelines yml specify the pipelines that are present in current project. Env specifies environment for each particular pipeline. + +.. code-block:: yaml + + # TODO pipelines file + +.. code-block:: yaml + + # TODO example config file + + +.. code-block:: sh + +# TODO example CLI driving + + +Concepts +======== + +Transform +--------- + +This is core, user-defined transform that performs the logical data +computation. In this case the transform is ``hello_world_transform_fn`` +and it is passed as parameter into the solid. It takes one or more +inputs and produces an output. All other concepts in a solid are the +metadata and structure that surround this core computation + +Inputs +--------- + +For each argument to the transform function, there is one +``InputDefinition`` object. It has a name, which must match the +parameters to the transform function. The input definitions define a +name, a dependency for the input (what upstream solid produces its +value, see below) and a number of sources. An input definition must +specify at least a dependency or a source. The input can have any number +of sources. + + +Output +--------- + +The ``OutputDefinition`` represents the output of the transform +function. + + + +Higher-level APIs +------------------ + +# TODO keep this section? + +Execution +--------- + +These are useless without being able to execute them. In order to +execute a solid, you need to package it up into a pipeline. + +.. code-block:: python + + pipeline = dagster.PipelineDefinition(name='hello_world', solids=[sum_solid]) + +Then you an execute it by providing an environment. You must provide +enough source data to create all the inputs necessary for the pipeline. + +.. code-block:: python + + environment = config.Environment( + sources={ + 'sum' : { + 'num_df' : config.Source(name='CSV', args={'path': 'path/to/input.csv'}) + } + } + ) + + pipeline_result = dagster.execute_pipeline( + dagster.ExecutionContext(), + pipeline, + environment + ) + + print(pipeline_result.result_for_solid('sum').transformed_value) + + +Dependencies +------------ + +So far we have demonstrated a single stage pipeline, which is obviously +of limited value. + +Imagine we wanted to add another stage which took the sum we produced +and squared that value. (Fancy!) + +.. code-block:: python + + # TODO example dependencies + +Note that input specifies that dependency. This means that the input +value passed to the transform can be generated by an upstream dependency +OR by an external source. This allows for the solid to be executable in +isolation or in the context of a pipeline. + +.. code-block:: python + + # TODO example driving + +The above executed both solids, even though one input was provided. The +input into sum\_sq\_solid was provided by the upstream result from the +output of sum\_solid. + +You can also execute subsets of the pipeline. Given the above pipeline, +you could specify that you only want to specify the first solid: + +.. code-block:: python + + # TODO example subdag execution + + + +Expectations +------------ + +Expectations are another reason to introduce logical seams between data +computations. They are a way to perform data quality tests or +statistical process control on data pipelines. diff --git a/assets/Dagster.png b/assets/Dagster.png new file mode 100644 index 0000000000000000000000000000000000000000..eb1724b7afa4438d429a1ac83b14ee2f92638de4 GIT binary patch literal 219118 zcmYiNby$?!_dbq~dX$$1Dy5_cR2V5y$pJ=11WD-*=?>`}OmrxblvKKh9vIRfq=xQB z7`kH^VCMTg59jm#{r))D96i_Gd+oLEb+3D^{hUuPlw~hpV7dTDQ75RqGFb?6T8qQBuBQpPYpL;4WO**2O1O5J#YVqcnTgvO7d z!B!?zr!H;4&e+4IoJ&ieFc6=e{(9PPD8&9NEs?PnM}70)@NO?M=(kAqaaB z7ed$~4jbHH{39x9o@+Yq1p;G5P`A!bqH}|z7UAK7k{r0|E(1>l{e-PM^qb8@n#W(zvg<;%Z#`x!5)y?hs%)W5n zf5SKH4V#ktW4zHLpwo`0ZKG-IY}ezzb}Q@SjXMVFA3`H(gQxL$B|>mx`#oxrP|g{_ zp|3Y|S*BmXEb&LWblwg(N;nWhqGF==9QYjSxcg9sM+sl{Jz|XBq1J9ZWhegH(+>E0 z8CH~OQ9euxafg4 zAWG(GTc+Yo#h7ya7TS+zzdrf!h?o*jr1?coAFhh|G(x=E6{B-H<6ovn0cf4d&_roe$}e_`(!fBf0FFUrpU2HOzre8%s}+B=nBu~S!dE+AiLT@Ac)RXjEL-c^R5&z?kh z-8z5u-@MP)(ofqw8u`qY&R&IS8N9{rj2JnD1KOh0}6Xsa_;y zOZ92?aob+ly{h!gF14e(a{=bYD}Ey>nE#pfw}dX%h0=wqJEA1|b8o+!g$RaSM=7%0 zp)WmMdZU!NH%#pPji!*7nZ-K!;sY$p=*x|-icIYVD=zq6@xA7I77^zEmUG6bJW(cmLtupwr^%<954ipz4+RQTw6Bj)p;g97gB6 z*Hag{&75mSf8;(D{8rV=&`5LBSp3jac3*S&nV5!{w0G&N;H|TBvUI$SA7`YeRngps z+*aIX8t-ydEYz#|-20v_=`M+0mVS_VwoA<>qs+1i2w!)< zA$ecx0rkC35l5jqyC=dfBD$76Ca*f^%v$d&Js5ZJ8NMW3Dxx5mC&+w1sDrir?0j`c z5MLLAGQ*SLbHU7t*Lowr{<60S)9c}T%Xg8lnm0xFhEAitL+KKhWbNRT_@V*K9kl;49x?MbqAh!Ir^fRUW|u@9${k*o+-D^XyHwWjC$=Q8!3_`!?-u%lD3# zJ>3n0-Ngf&=YpFqm^oM`c|JjSixqf?&+iNj|Il6W8#-WQ%v6Y12yRRBZrUbpGZ8|o zma1MEEE{GxUcx@b);g+r8hVELzCAoc<4=<;8`=CWE5|CA@EC7pG+@Mmp~ei_TkXm3 z!JU4O;;VoAb_LDQ&Z)1uPHN*thOzygt@sJ2_mgl`8R|1dn8JKY@6=e}d|=L_=7#9k z{*TkYoqgQ@nC4B#8<`)H|2oT=wCsJZ-o8w@Nbr3BrYo>(_51$MkY6$H{n^9L(_F-y zM_qqOt9qroJ`iW}=S|F63`Z?T8^`+7uTP(+`lQjP(WWM(88>@1M>Wft6_{!_8KHYW z(*AaDHk(bGO_<4IB=L?_IZOPI`Yt+6;+U234(8__&`GQ#60bGhPP{F9t^1SUC%#YX zV&48W|S*rRobuduhAw`IZ-)%S#K2}sNAZ$6U`+l@1Zo2Eb(RN7vOE=_Msj9oPlzJ1}E zkcISca7lIJ!(n`1;P0m;M)%`qW7H$J?}&)+%O$@`zUO38HN2(N#sLMutLD>oz0M|&B=uP0g8;uB|@t^KDBr^ zU|Bzy9xc--B3x;oJ^FEkdq}j>W-T*UiCw|#Hlsp;0+0Np)4XFHF}g!L+Sq^OOPNJc z`*(Z1R=d{p!1CxAmX%wZA1}Z@%2rbAN@XVdCoC2byHfMwrM469qc`oe(+CR=;?2t|P znaCchgM_Pko|aR#_$y#KVDa_GGyWZ69UXk}d=>@HdFk$3CO?0)O?shv$D8c)oU*X7 zY^Q^61+k$Lh^xE59tZh`7loDiTM-Q%sLIpQX_pKp#QLLqZm)(dvq>>|m0_DRvSxkT z*7w7_DlG z^Q3)HwO+i=gNzGUIJmyExt0~8k`i!c|K0XsxU=P5KuMl=meX~G!T34Be@=O2AU$BP zo3Q6kA8U9eu8f~gOw#m6lGnZ0?kmW}y*5An+TNAGaV{OrjJ~kOz?d)Nl2M`ewCMSe zLtav5MKt>JSC<^JL2d=PFSp#beAhU`{kwZ!b>-)OPGA4>HSeE)N;>_?_{+gNTz3_| zSjFr!cS;Z>_AQC;+GD=rmlJK138w&)VCqU9OR{&VFkTypVzq`;8hMgbWu-MDPO?rh zGl88g>J!Y~1BM3GoZ)M{8EYSFc(!KZQ!BN{!u!<6^5)>0>VJ1uTEg4e!6JQCS|FJ|o!4;89FuBW7H4Y|^lXY??ma=;N90 zXT@e?)w(}6s#&wfDifB}c)Z%^?z=h0+K3rW7T>O^)NUX3#tF5}&U<8BZxm8m!wKaz z6;zKziWvLODv4l&v{%L^QEcFC?y+jLQG@@|&>m@_EAB;Xb|vg}NM`HoYNqkl6 z$J3yBcNtZ_GDgggd#r}ktW&B)ur5cRBt89;qUDkpJ>`?!NsYT<)t;T|xa9|weY5(? z?Y16)fW~oTVnCxnh_TNx*3c2tv! zd7jRMPDldU#kGzQKN+3q{neg|b;BP?M4i#(<9DhHEEuC#81YM=lylayYs3af z-o-~Yn*@=FtQyUus&Wpzeq)LSVZtkUWOnsWpKaFlNf$Hw35lAi`QzP$Gk_(5hw}e| zzBOTBW*HxfRq^1<)hPX?mQ3BI&Z0JE6t^`q@|9qNFYiHSsXGTR8F~m-FY{Z~FT8|T zDA6V%vqwE<6^f5KSvS>iZR~ao!>fp`{=>9Vp79D(-0iI)1Bptpbpy^xwVDEZN2`O( z{PBr0yTL-G!72;TFlgwx0K-FDRhI&Xf9*g;=}mMB1c}4ociFYLyUAYzz;0*yM0Ie%Em;jK(4#{ z5BMII?$^}VVfk!u#p`Ziu7)L6>7$baJ%*F>do07a zYO#TlC|LvAqzKQ#1EHKGVdeR1VMWVGW|2ZEKYj1|v(PC#Q1Aocwm;(Cx2YGFnRA~? z#mVxvsp+wKUt*x)v)f52&#&}Y_ckRpy2Rd7a;p=lx;3tqc-XjE9o<=9-r161MmtD| z5T^6QbaGC9-m1(^%F6Rt9VsT`E;3Tm#~le4WU#q~#l@doO#Y$sX83$(4*OX9)^&N^ zOEP~$Ke|0di>6}gou(?RGK{5K=`1TclV1g+8Iw2z*j=0XU3CR38 zjNbVQ-oD4?t*Ehys_o`MgUTHr3o+o3z+r_G=&&!dIN|cwLvBY{$!e*I-w*eCB!3Wb zb@3FH2dkf)b` zT_bP0{|cJWd*ujTV^8qJW3{*RU1e%r8h*%@@kG;0zU#x;w`#xNGV{dbbF<}6?p$dZ7Mo;_Anl)r!0dx z?G~z^7E(%qG$6g)91`IOcT#Tpi{+?5+5O7R&~Kx@2yAH&o#`SX_ zj+hgI7{?4=X#txC-cW)t7$+#+oXnqeh~_M{oqGwN`3~>-#dItyneLU{T;bFZFmMpq z%WL#7c=#eBxa!!Z1Ta)p@Yn~jg^{T~?$0YE9biY(q!IF+;I!zT*@>mS1Jkia$ogtM z1iuWqMWV+glw8WDI+m>M!5?V+z0`dS-PyvC#GredDYT9i@Ttgl6iTeTSNl%y-&m1+ zJumk^Vj#LQ`5+XV=(ThbpB1?LPBk0pZ%r$a3+HuY9aJw6GR5;N{+V!UaO>CfgCPou z+#UwzNlgZ~wwX9sU8^E2j&h6G$=iTk-X{t(lrtUy@fEZ19wvF+-m7ZWS=bzA{l5 z7KC==_-{9B1Dv-_u4QvpOPD}R_~P<90#~_If9h!;ME zUgzgoeVh?W;3GY*9`9_U2EF$}7~1L^or3J054P~6I5mk19Vz}it=Qeh*q!VY%lTp)41y0AxzFw7v9&%t@k5A${BydikmfRw*Lqyo@<&bGOJb=CAYVW5CzNbO{g!zU+u$d(ADV5s*nqJ?o%VlqBzw>!L_5+_bK zD7xsNy=*qS>#j$LH0w=EnJ*(i0}P2Il~g-$BK6z6barV#ME!NX@%ZR4V? zg|bC^*``slxfrq3ya-y@p@CHj-i#x|>@K`>aS^j>9ac`}2|uFP5jLH)Q))LxS3Lxw zLjC51CJ+{R1MZjkQ}&0VO0u4zv)r|pWPtz$_PEo}Zo+$JdEiEq*KbgsySmL7Z-2y> z%?{ulb;{lBhEIX8Tl;new5oLMOEBt{mv(`Nn)$Gk2$R1GuVV)vV2%37%ZQ#f_1|qg zf>!%oI?HX1M-QsEi91cDRCPA{pp^Qi2!;1vn|W41ddW^DEt_sf&jXIeUck9waA1pG zEg>0+%u1=uxSF-S@mZ;U<%6t>)m9{SaqK-diuDSxN1P#a7ZV0i^N!!ATn}g9-tu^R z7Sv2{u>M5=C=DTf!;)(|ygiGqw#EL|_A@0j6ImP{lr~&+kde9vXokr=#KwU)Y!9rL zyKIwF03vR$I$k{ASg@wy!40#FoY__ucc+lv9p65{FsSE}T}<}QJk#4ei$;`H#% z6Bgj7{bp#f5{;bdVNz69WFDODD%?We8K!u(`DwZtM_yefP?sHC^LPLJ@gV=nb4skt zm#i_h8lDDL27Wz3S!UH-T7g+!1d>aQl*GHlvp3fF=8TE0zCp+X9$oR$Zltq15R6{$ zR^*LQr-MG+#h6K*lRJ>qKn)gXb4S1Qj=uPM`)sc6kz@K!qAl8j39!Lz8;!TZt3)KT zkGpzOhbs9AZ|xL2jC!DsxG+-YsVmNa5e+-3P(*tlW&MK%Btf&L+FTVtn*mvs?$*O! z0iBnfCC4R$j`Z=(bFvF7mzoSaGOPAjIU?9~B3E!Fzd%%|r$DYyR86`zwgpgbq>Kc% z+j9UOG4r|P!KsD-4R{l6iJZ#9(h`$hSVi46I3;VmdW5}dY3#jrG%KiP|IQE}a|X9BZn`=%=PX_zrmeN}3`|`oi)H`g%=SDdhpp4X3QMf4pO>sJAqXSs+g)xE z1SUvp2IPp_x_eEZo)sh;y-+lM@!Upc*c)cTG*aqnfEF#RRzsNqu{3V1MYX3Le|P7h zs)&NjAEKN95~=6co%y!f$1W>VP9V3xH|^)kY*-Q2O)86R@&>#_At9{*FY}kZ#dGYa z*qxELzW5{tFDE zQI>S4fL({qgd^qn12bi>WN*W{`oxNf69@QqZz+givbA^wZ1Y)Bs>KFOe?Dcy?!9m4 zYqEj2DFd2!H}ZS;z#*p-oG3|EubM=MDi4i;2k=+;=F3RcO;fEh2VtX88O zH?n z&Y#N)ZZu$+gUoTw1AN~gH5jXwsG4JL1%M9Ou1=m9M^x0WFKshA-Gwf`kp~AQX#?io zL*W3$r#`xA-TO19-Y)>$1b#^uy)WbAUyw%-2x__`6)Ee3*Axjs)j$-`4ICawIjNd! zri(4^lDmEGwoiRzfT=)clswu^4K13}jl3uIaKuL?*`v<)xK##ua;`HobZ72M zMn1<8y5%Ia&w#()&}C{EvE!T?`l4DN-YHDUub6eSaD0`Oed7O|6*#O2)}=sQ9!R23 zHMnxL8LQ?DD1YpHZ4HC%D{t;f`xr26R{9J`jevDlQa4dYn5` ztfNu9CXayA&gB{jHI`H=D%i?x|3=Z$FFjuYX%)-4gu6?G8>LdQKGbGrfa zHX4c9MctUBWX-&K!?E0jAE#i4is)H1V{^ayeulIe?Rru>7X`1GM}6>kxENS%k_o}X zqYV0n8B*u*7DdNFn+=73-XxrWCqn!ba`~#c5FxYr+tDqdEPybs!rbn9Bjooo}8*x z!mALf(1b!@eh=&3mK0+zDRhLH%0Wp>RyDCyJNy4U_bSGd3+WQ*fkKQK` z1({3+GjYxm=-SfCrU1!axINcN%VD7c_`SSNY{X}Kpg@08Qc%K89nh>meD0paM_wOd zKi_4H{iVGmdN?&4K?8eqo+)WojX#pAf&22bdNH=Xg8h_W4 zF*7QR2Qaf?CxR`Wqqe>q)wj%?M(ZSxa13ZZDYc+$xq~71Wc(TMs}mohG!R5ccwCVV zo1q6AP=D>8$MiIIOYWBgG4ZV6;%Xj9acyZTkp^Kz&IIYDApq<47nf{1uv_wsqDo+B zvw-Y8NJyxW zHcTjAk&a;@#`nEvEA}0(8{ukK3;_QWi9jq^G|0#L^?I(wW^=#Z-0j8^Ex8Z)z|xh=Cl?h0byQ0#nQND~_TEtvEzAV15Uhq) znD4ub>S3`9T^*@dRnj)DBz`DS2OWDn9YhN(A9HFG7_UiHDVS z9*S!|v*lI-xQdk*z(Qvyh{K#QmyJez1dZ zJ^URwX=+dv6f#pHdQ!N9*ZtW?-Wl=9%xFNal`8S6`FQhX#8QWrlV55hD0!iIjQKne zV8%|0U_ge2Q=)a>(jq=Lhm>ln3+5m!BRBJ7>_F+x48T^D3NwN+#<*%h_Qb+!NJLJ> z@`C+^r|69PtKlv4C4Zg|i;RE~8Mh-uh(u14?inUcaagaz_`WSsIc~a2AeS zE9^5?<*f$c*$_1_{Fqp#Ct3SVOnh_Vk2R$qC?FVq$js6ND3gkbsY+CyyL|%LeZ6^T z3n!pl!s4`Sf#!inRm=e>(`&4&Nhtr-;ow5PlXa2cx&~fha6ra3 zQ!fwjrknOsJ@TRO=H4F(L%06igTYu9ISTSKiNEHpm-$Aqt2GNF+5#jSC=PL~5L7H9c-wlWTILxalBkTdd8xouY6uyi zg?BEm6{JW#6ffS(OhQ(hA01I0)j+;!w}&8I`n#)z7|Fz`&ySX);X?p3mD1LlsGg12 zhEyqH4@Yp>wYxD`M<|CtfVH1=Mi0iOG75q;?z*V9mSjaGodrs@!((}u=NYKZ;f{#S z!2-&(J8Cdsw79yJ8W>YHv_p03o-`lx4fcG?;pMyiUXM>O7C6evrn|S-guYD>dZve| zT1tFPgB@M{&&ErTjSt#SMqT>eN_=gd+%dbLWNN^4wU-dPrdo&x9o>A4Mpd@c78XR=%VwBI#g@c9q-U~v2WCag{(jczs++Y~{ z;Po5zWpPruE4%>NFq$`t^2w}v13DwYbAHsg-eq?hd6EUaK@dzH_EoxV_vEWpWvl&B z4wcMOJOzf@&@bB82vER302;yw+q0rveZ3kxA)dasdtxHz#ew_;PZQ%wcB}rSrG=x< zS70#9injm{F&&?=q7GHA@%c4;5jogGehIy!bXT^a23z{RR0}ey1>H zKd-?;V&qA}Jp$mN$Oi>XU1PSVL!nfDH#Ay)E~=yyh!))!jtoNolFKVs`G%Gf_U=yY+k(KOTUmV9T076)i_DqC@lM~Qv>jK!YGX6hoe3Lk_uK{WwH1g8m z8C=l-d|{b6W*nMQB0$Z_8eiQz(8jP0z3{jR+-MI8O4z__m%|)a+SJya%k}&<-ES6E^E7;rUHG6t*O^SCIqD-3h^ai#iR2jLUtuT>PpGqzRVxF7aP0)D^1)> z93Xv3B$6^Slf&`ihXN;W6=*^Q691kgblP#gSjoZ5x5rksz?h*1gwwsW4AFEV?$EGt za1vu5rkr2*s{c(}Ti8Eqo!ReR!ccD;bbdHT?D{RyTBWp2>tMa{1xOdWusPb`?ndi6eJLK`)B&;-=G6w&bF z%-a$tZ0GQ~{Y%6DmEEaWmL!W!c4@N$nw5#-)uTMTaZS$3Bft}5C@50Cy1-aL%+)r! zywUCXur3e@W1ftUi~~5IWU+_Bb(X*A2SdvzJyr@KpgT`Oh+Vne_N<92PuE9XelZT+lW z|CsY$!({pYqqDIG(4F>wq$*Uje^u`wbL94GHkLr4GQlLIsMX{N-X~K@o6TAk>Wgh8 zK^AT#KzABb=qN9UoY&xS3Z+wM>jbE0#TA&RlC#8w1%28I_ck(J87I^r$OZ^OAo9)j z{uA@MMoTy0?&eR(3x}FQ97%{QJ+X|c3itVFD?G(*_%-02N?*vX`o}RAHqNWW^(?)) z89EoRI|SN8H;+$+zq8`gXxS6T=&hVYoqo)ha02YGQ4iU5!~{-c&&f7T#&9I-crXBb zf|gr6i1v}Nur70-{!mIa(GT(vxa6cA46OdYZbcv8lfrnC=rHxp$M8l3lO49EQm|r? zep#EJqzYLF4NQnYNGn+Sy>>pWhtY*YfVHiJig@p#0ro(C)t zhAIFs^=%Td1t!=0`4-WGbZvzop}=~d5!Pw!0HUl%Jgk_TTh`(~(u2;igkI7De@-nx z4=FX(aZ>4E>NS+PE23LbBC`5+Sch>TzwwRl*cjIT0moE8+;3TK-$B3oqwXw9?fIe# z+M)mycMWCQpXdm?JA(mPVz{{-9&Nswd_O8&Aj4mk4S1F&|$vN|2fBk2o>}F3jC%Q=Xj6CDPP{*QtLsMQafR zoE-P*=nC-P;x^-B?Zt52$c;Kv(q{;6r4!$E&DouRS?^0L-^1L{JQ|AbiKnlD05xf9 zRX&!>*j%9P4Aq+xKwD^ofP<~gH16gfehQ7))GjoZ+}JGn1^i_@9pr?NvXxNU(56kU zI&2)9F$lImNWhK{~{TR7p(Uy6c;nzYT}4YX!-0qnoflWOz4K%NCwRk;to)bS4EsU z@RQ&>cL^lOBBQ+CB1>MZU%rbq8fDXOlTZp zqNl$nGPpO0tDa=g^)Z50?0tEL-Z=o|UC*BaY>mgSjhJI3#1$)H55A<%= znc^T!!hCfZ;GKCpc<&KFi1i95KAHOaGMJ3n zm=dl}I3ExM98n8Ve;|x0T0l1`>+v(o6RR}+a~Qn%3qnazN}qcApa`4)%=5Qt`pTn4 zw$uG5=j5K8V+N?;EsgL9I>9V3S73*R=w_|dt<|QX-|Pj(Ra5>J%^*gg7CI2_1Wb^q zfC0*=I$FIIWqY$6?K zFhjZ^PX?Ai1g75E6Y=T$Gs+tphTfnUPzwM(T_ch}j|%d6VrJiaaS&s$lLomd>NzOV zw zgugv1XaU`ce5AZ~Qh?WfgTnSehk)bi6HA;C@tD!mGj+q{qm|Rc6)g6$&|&CUcjam1?biRz$Z1pXC@R?jXWpr7(=lA zfyK!%JiSc@tHS07c7yHsz0t67O(+m2Jw5O$2Rvj6J!_%>k5JW+S*05CaDMB2omW4% z6e?<_a$OmXPFB=HBgr8`On60DL8cfxf96|=L~pX`7HHOWvm|*91LKAx_2HMGpYA|( zlVTedl48_R(rr3o2vP@Zt&b7z?`CapYX9ppEdl&C=!7#Tp!cH{+8|6-9X6|e4;x{T z1&PGK>R65f`yCpISJo>eLIKnSfH$kGhmxVT5GY)y#*cS9ujbG|PJ-2UbE@~%i2<^* z)}4nyQGo1Oqk-(f*oKrP$6D-qGW1qK#!s;!xMloIfs7Jit_Yx%pNKAiTc_$Ebnl@d zbd%3ATIuTPo?Ia^3K#}-*Uxmf@z?F0)lRARQinpl20af?1u!#J2Ru)6nv!nZsf4iV zjLK;4$NAI^8z9I(dGKsRgJB!c7nebsrUGn26eVuA*{+AvZ9@dmVD#Wh^vVNZiiH_2 z@|m721GUT}V8Q-7kT0E6%ynacnw@BDK*s54f!!*s!ASdggwdo9B4BK4p(*y{VeU1s zB!Ct=!w3OBh=J0P9xjQ2;=E!8S1$aWh`AgZN?F52pP7UsfJAG+Kq7052UHl;3n~;1 zMRpC~+Hw49@`4y18DmfMnI*W79X3>~F#59TL(FG3}{Z z8%k=fFdf4!SGcrwN(2~%6pU`ixy7@>4m)3{lvaVeumr)28ps2hN`h`LZik6hGQB;H))~s5u6;Wf$X&QlFe&X#GYseeI?M4d?Ndq6xq zh2Ga00_3SE0}=wzyrP^0{XGDdH9?6MA%fG+<+}8}v1Rc;bl-ffYkp`Jj6F^-%^UwA z*)M82Er9af;6gez<5HBH!cjOxKv#UodieT;ddxD*^aYsbX11O-eh`o|_*$q_0iEj% z@z0btQ_T*E^N>W6(oziP1-!d*F^N+JfvA_<>9Uph+odN#{5Sxv0AV!i^r&Fw$(X@uAiexwfC(_S#Uf{$pwrhFsYga)p?%%L-{%gskf8W-SZmSl}>3LPTS5k`2r}7zU97 zF-)gnioUhwbm6*s;mv$Gjb_2isa3ke5ti@HR0E$ZY*XC+I3WJ4+RFNQwyhCKnTDH_ zgKKCEz4lqLz~NiRFU^Y9A1lZfA}nT?excMnfIib?51M_gxq)?rnNxqlP*DZs`7!ow z-%_tvX*|i5HEiK}2En8isk83&*%nuu+4ROAD5I=ZRuV%btD6l2Itqz1LWo)PdD;Ba z?OyP(JlTCPGFb*ywm;^e0&NME5|i*oI}yIMMa;45Q}GHQGo6hAXg8_{;bsW?X^>_d;CR$C(TCw~0TSk~RZhiy@?C=R%8qF9ByaRR` zU|*4|Tf>0xw@ZPw;nAhAFk~(wkD@LJd`bkI;pC>-n`}6heB7|+HP)l25jB;We^3ig z-fk$AH1kBv|SDMi&FEj6q_ zU2Um>tNG0SU0|+cI-&_MMr9UTL^U!5yJ_@!N4%!T4bz_+IC+?n-YwIV8=n>XJ~HS?J{ko9bS zEa~w!VKGcsXZ!)bpQ4kWL+={#Hx0Hqt8nZF6agmK{bDPHJY523 zF-LcK7&K@#gf8~?&)_M9!Aoqj8bxo zJ&s!J`*BG7aOzcVu!*_14RUe=Z#T;gq@=}gLa&cZT3&8BG`v;u?E z(t!dtq9a(<1HVdx;?T0_*Py{9VhA;WO#cX!F~9-!)JuO%`(0``YtnFO@zF^i7FWI{ zmsvGeYD<)H@mJeg)e!VYXLtrz?;;pr^}{7m7RF3W4zbHJL3A+(dy`%&<)gnhr^}-(j)P1 z1D%LK2GPs4%zYv{Z$3PV6_BTKWKgr1oMU3TfqEg-M$U-i2dy3J=cK_j#y7KH2_?%9 ztHw>UiPm3p^4dD(Lf&+pS+B3K9|(S`08e| zlS4S4h)N9WMPNq2VKlsev~>o)YQ-m9XT#+{UG+{n_)ztrrEB50EVvugDw9^P zrmKbkRH;9{^}y0n&sDr(cexez zr^=DF?9u9=V`!e^i;y#M^1!#39Ja=eVfFz7(RnlnKG5i75t`{|R)L5Z6(#?sq3g8* zb>!N{ySZJjkMut5;<|z>3FGP$obb=iP@=*i$Tw{j`8yg z#9)kp&;}mrw=*I27}rEyZN2?wqHY98Viyn%VTHEnK?z4?3ChA+Nm_&Cmy98GwD0^? zL*Fnp|6Xm!P&=nGGr$FcTxxsWz$4JXvQ|fZ*$N;88g?MT+uH{=LK729^$fk=fn<2p zTv8`rvzYGf^7jaU!jw^)tfpBB z+xPd?uUTClogpCTfGi8tzRZT2c?pz5-|nXVASWIu8K7Bq!^7U|!&~jVd45H4dtlS1 zCSOddoDkifmIMM-n_JD~LX!5KsgiYXPN?^>^P8$3H5)WH^56~BEhznJCAmx<1HQtf z>T#D)xdg)gm%ao?(Q^PUVmF`Xb5GmKaWcIB*T1;t^nWEX1)f##WpiSS0D8NL_`E=g}O&DU9b1Fl#+;G9VujmTj?E zk3eK&jB7XYpe8Ewgt%r(9@MGY0iV9e3Le)&UOh9fJ)4A&XAENb9L;)muR2H|x2ZXK zl98QE-qSgp)1S@4RsiGTd-|8nb>~XH7Y}R?79>&xP9O!TrMREx_{rU4z)K9!PC*jG zjm6iRZ^C=4CQe@kxPOm%V*(`yC_la73+Q6Z7}NUU-d>zN;u-z7e_YWtf7pS6cI_*G zw$^!}NC`o-MJEKzX*U`Q-%1%OhJ(VzO~UfIA-uT$U|pfufL&9i&@BMOH`>&s-q8^*VjKvc!u0<2-wU0K`y0c zH7MO|Wd0EAC;(pd#fbx-d}zAE7Rhn=RB#n6eQpxA#rIjpn5jH6h;3XfR{-tdd9-MO z49|0l<3(I+$p69-fbE&|-2jTc&)&&9J_tn$RHflyBe)aQgJkHP03u95`tF}%clW2b z933Bzx8Hhw>mRx6H$F~1y1L&`YnP{GSD;nRLZmV%@aVSI$KJBeD=)-RQ5(4PgbmzX z`}A`C(#_9FW5mh#BO_X$T|6h5F;<7ZV~k%8k0(M5ehN4tjbW;03X-@@?pEi9m?8I% z`YHj~7lZY~-;wp>IgdWtxWGP(TE5ww>tIqH^dr9cBg12#1yDWgvkq8CV1c(@7>@4v z&&DM*4lV$HQXQTwNC^LU?VQW^pr zi2X`Iyfxpvl2?J641-+@a;gA#j?wHFO0i;bEhl++0Q6jUYBbGQ;*(gcuCBcx7sKLbs(@r|52bU!(g}0fu$bSaIM)zv9YXa zk~uBd-s%)=>Y@~Oar90(?_@FSSvj7ymY_bdYRkoM7cc=NgFGn*Q#}^gt#_(9v-btG zm7K(V*3xGN?VCZnuArbB!!9P$T2`?0(ip z%wZt|+my!z6`CEFOIgpz`F7j5P_T9}d(89Q2lxux>G*G}aQaE*a}CK)bp%{13zLQ>U3H{-lEQ)FWJQK%=_0WNj6=-R=LbD+%@hut%q(jO+X! z)&G2x#o1;E?N@(H{$_CbPugXwp@%OkWmS?5vf(O)-IK9rEkcTcseX^4((X>?k+=Fe zqw5#mU+)Iy{IK+^C%YwT8jNc!we3mSrLQ`G!L9dqjWQ0>@^NT>^hQF;HyMZX6|&U% z^z1S`N_z2b&ZU<7xZ_H)_^QCJk3&F$qVns+e2LAhe_)%YotSKgc6VFbPHLs3){n}- zAvcwv=?iQo+m?j>&W|A+w3Lj1@g|RclZXwF{I}WT(ygw_+#lk#^WyKd`gv9==gb-5 zSm$r@R57-Hi|((A+h%(du-(E+pKW8%{);7d_U39H&fGWL{>$<{RAFt5{AzADc6M_( zj0$GMwN>K#%wqaCOP1xp;`f}xv-SiKbVa++Tmt5z=I`!Wu50{(*%n|Jn6bsuJC-?b zxRk}0A7^!O`&h!#-+1KRqAkw%qguH_KD^$>yQLXioa?k660i1|Q+s|$3+0+tQnTn6 zZ1!hVq**1xsV;Rp{@QkHRa(!nb9j3?;@2jjHCp6RVdCiujkAG6BqNI-aS~QrW3O0; ztlsI%n~Hz0#~>*cAR{2b|6a1Hm09O|gl50ESFD43!=Ci8qtL0n?3O`fKvhtk%dxLs zUy;`L!iu^y&;I*W+vi%6jBm~E4TYVDE&f=DYi)hgOl91=Q7}iKvB!cVU|2&8G%kYu z)^T5LHT>&>WYwiE0)dbDGABsd4FJsgvvuq^uJ0PATtk#Ut0izWvM*BWV5?7hJ@JW6 zKq1!mRg|$_MzLAbLZ!b)SKO<*l~1v!s@Q7%ruoIUJ5_eu58DPQ=G1MegV9O1uQ zvn87G%RnZ!ZIw#krU{fR_Tw$=eM39p<*H09#zM|Y!w;?vfV%+$% z*42BG{4o;zz$BizfZ1wp+4BKXTa<-F(EoX%0=vW7l)$I*`1zzqE^svC!KDh%$B%zJ zpmz_ZPC1g}Z}Ho=71Av(PI?CuiNoyw5q&6fLvkWJ<AJKTY30z)j_s zy19=>-~Zmgkis=K@a_G&&_dI`QP*V=LbexiPuvZ^DQ+_52o+hdx&RFwL32qI~kMM1`Q}5 z*1}jk$CC)*G;R~Llua|Ksj__(mpu57IH}bID-))_-ZEIEI=gesxz}Hp z)@JIHq^LeIiHQxs<<+6uf_&2I4)P}9*I$@)TsS)*Tqw}A8_X~pH$VkTu%Yrc5D3k- zld}HtHQ{MTFux?qD4#z6Ik#z^r`Um`xf@wa+V%As zcUVI3LOugFn^XdeJuTCBujh_?j;j+`N@LMm#QU}ie~CcB%r7SOr|sg7qh*)_nqRzl z9TR;a-!An!6-?iu2VFvRPq|dU)0e zJy-kvT7Q06E$+B-nY9!fU{|@~jKIx%#@kuEYKKWj} z=xpvXQ@u@yE6RMa9874X&0k5San$>_lWHGZcypgQ#>TU=r1c6MRVx;8{X6*JhoeyY zK|&##rI!@m^)D@T*RX6@Y&bHZ<5iF2TMf-G8SCRabtBb2BIHj0#>#VN2ZZ*cXGnLN zZg2@cK6SHP^SZaCW}nas61COv{=Z<{g9=T9Wp4b0TcTiRZA3;hlr=JS_->cc623V` zHk4hVW5B}sjv7DoD);lO8g7+eO)LG_a+_3uYP!^ECR4dm{NxZ}lPXBbP1xm?*rW}X z9NVLhrzKI-j-);I$^IVARb$FKYeB~jbe(BpmdQ5*?=z%|&a*K4KId4M+NW?}zyOmw z`G7XC0}xzCtg~^5#=#y~Al<7m>y#tEc{K&u@Xo_oX@3h0nSR#3`MF+X7*dDPZ{w9Y zeZ?#P-tkl#qlBDD3c2h<*n~O(MR*YkXE(X6kPBa5AKYeK%k6bMc;wWF87bpB>)T&# zn{(QZd5)hq8i7UmNuPeS6dW^Sc^Ts{hMJE0a&Y1sqVd=T@P9=WR_)(uTEEd;C0Sxl zBLI5@R8K!~??c{E!F=EO@v}IR3MxiCo-XONkel4sRk`PO&C`urG%PuVAH=j=pJLSw zx26^1kGgr7bZYZDkq6wX5~;s!Gc`UhyDq*i_2_v|`uh`=wwKsfFqIy2rS-sG;)9PfXVvFeGL+?ZrhH^A*0vT8aFM#XuQ#G+_js zdc&qpzA=FrrWBDT{3OrJS*dMhiWk##F~+<7ULPIt#*FSEB){C`Zn zbzD^4_XawO$}1`cAT1~=4T6#*f&pCVV70+7FTI=iyV|3a$1z&1rU;UbIqt+03KA_BURnmFd&WxZK zT(f(C?RA}#t}wZS>iXFA|IPxGml9ZV+GltY9S&qE&U}d9o_J}VTMb`mjnW$x5$LX} zbC5p|uM^${Z#|u~8k2%*Y#joB$Fk-qVK#&Z7%o}xXZ_<$R^}H-fC`DiwE&jiauhDiv83d?nrm$&6$7WE$6M1p|AXI zlh1P*>1ZXA zGP`%i}?`s8`>%7+&xy`8*F#2OsPEW_$0|E z%)zAkJ^k^|;UPx?3lGBf)n8!UmfQroh30lEtyPh=+J5O1^NvNrpMfnkYGox2dws+? zu7>hRkb+Y>$acn%VfI?xZW=w0d}0#p-=?mO4Ai(|WX)1?Ud`zezu>8jdd`S%Pp^b7 zeM1-ysK_$qa-#p5!BthcNbZIlS@(5Qc8V^XOmX`7p>EkmH|2R zzI$lAPqx?i#!++v-0cgAAW7Xl$#V{4+Q&1VGR3%alyuf!KLum8p)+y*^SN`JZ=2TF zUDK}q-cyd%+hr|e%rTp9-z-%^IGr!x=%*k(7F%ktJj-ad9MK?I+_u(GhPs2a z=zlu&omjzU9p0F;SA7_VUFz=hT-msf9Qgi=2^0P`5Q|%^zwbIY>nSza93M96T|| zNOdMQ1Hz;VSNry24?pi{b%g)>WrMmG?<$5iE^|!B{neJLM|-E&!k*B*U}Q zenPb*80)|ZW4M*G61U?R>@HX`3wMM0l@;mfm{HI8crQ449GAz&AJs}Lj5YsVd2^t8 zycV`(GvIXpI7JKV_Eg9`IVVvzaun<#!zWdh>|k#T6YG@rZ^TZhR$w|wX*(~=oKXX| zTWND=0kzQ)v(_-1@-x?(6to4;Bkn*tS?9_W`8!WHq84si0HqDT-wVi?D5y1bZn+fr zED9YRp~x#1Q6b#dTOKHgQ+%o@V2yfF%`m^1Db&z6T8A8*lYIEZHlRzQ!G7My)H`CP zyKce6|LKqXN7#l66<(&>2p^S;*FZoB^r{$}O2)fqq}cX7-ekXQ(%=e7;4w)Jtf?AG zR^xpRC?bQ-JO*?cr1Byc;LFKTQ~t>T$HdqqpTB%IOND}Ew*Ju%DsyT%_%l%b&j+e0 zF;`!=fArWU=7vIfRv&-2xWR$79&ns~QJ)DZ8Z*!B4|-E-_vj|}sbnfu49d$2nHD%1 zxX>%ueW%k>?SeS{y9;goaT!mwlz44Rp>YgY>lqT%g%#&y2!%f2+kEjx=Pg8`ipvb1 zdtf)*eP4Ybx@}$(oaW_UrSo^e<`G<`9X!$ZbUbuMtA<@aR{dAOquE8%FaV^}A0;XN z1o?bzV=+VA9`j@pvC}QS^a(-p-@16f2%ofKwm=(Mc?wVUt%1(xHH&&94f#n)Q*?GZL_WNiJB~7loXuOO1J9Z5cXO(ey*N>Y>V^Y zE8!t|uN$3MqbdB!M1uQ}4Min7BWtMM5jiiCr_tQ#BOqo}y=OhGin%&dfEe;Juzwyx z3E#hMVE@;!0(aY=xx;CD5QTG5$6LF&pbHAGQf_6L&pkvrUYMhQc^8+qwn7cF{v0lr zl2SDL2oT-#fao@xF|mVf_#K~oa7v$kxG_ft5_EI64lWFss`?x=eRfIb0cz%5Ot~cy zBLFvY?ijLS@p?Rx>}1{99jkPm-_RKTIZ?F$4$Isq8m#5W-R1<_GxB_>wAI{gh8@CyIbf%^R@bvAsg8oUy$0HE7{;D zPq_&5f8JX|2o!&Jq-!i^YU=YCBTtke ze9`S_-jjS>$z-yZ=p%vn$tNy)BfaXgv1ibG1EUu~+@JUMPO-+)FXb&)&qwdKNlQY8 z0i+^wB9)zlIOW*!MslnPkPv< ze?$HLC;WfI-(E@;N_r+MWhLjv^NSno=@??d^B9JYgPuFsHK$W&8cP95fEr4x-tRI| zNDed0ONv%2>@-HNNTwR5R8<4E;a4`FAljr+Jo3@=R!0+Q16Nb1X8PO6FK8}@=qUeo zSy7_;J1p&6C41Shy}(o@C{C~3Z2f?~++bNtr_{Kj-h0~7zLgXFp+)fVM|13{|A-uz0#Ol4Z;LIq`sQCFs1nT!9H+ME!5 z(<8O%Nl8gn2JEBN@;|yY-2(&K=6b$hTwttUM8l6KRWwE>#Q(p7X7&kb>sK{37&wa* zpdE*{l46PivuxBC&$(>Y5!kolGAa!g3|u~#>960P+ZEib7;6@h0LIx9{Vlum<5LgZ z9{E;XdWUm|e`f1~!x-{Sm+&1O59@Y5agTB9?VXlFYFY?Un)Oi!QLP5rTqgav@=Gw` zUsUedo<)`(12>)6X&TH<5`uQY&2A_t_v^aY3=o*n`WA5D*}}X1X;(y!`m|`QtzPW9 zkmb@YlgCWv9QbZVd^EyTu$%V`l_(pUh)>8o@Y<P?K9(kk$Zk7VqHLH0%=J6-N zL7Z4UMsaQimF@tz$R!Gw@-{WW7f1{v+Uk^)YOA1NsK$`+st7{9E#L(4vjd@41FT=( z)TXHqiK`YXne{{g6Uth2uF6b4$}L|hOgfJC5zJ4v6(>BiqQ7RN=virLhZ5R2t3K)) znKm*hD$8!<6>G^Km)AE(2JPgqmuZq1k+^jvS(pEE) zcRb;5`{~~xGhkb+sTWaRrVg4FEA(v_E1=ZiP46((g3Sfo-};6}cJN=v+5p9n2NtOX zM@!E__!N>iy8L~8o~+f054Z+Lnm?(lUR#yS&fk2XC}xJhIOOdOP0#60_2;9p`rp1? z`1Ewy#xUL>%rs2y$_dKp0I{`o z^xXIt87?yprE?{3SDsJus!8fpy;#v4#30wB-tz_tEsY3?uVOM6^O_LF%;vdgp1e9* z%&95+i_@}}ea?87X^`sjWX^Cm-_bIh3R2mph@3Jn#h9rAqu3&=9h)EHlQ-8IuR2al zeKeqjUF%5DR8YIX_4-v)b;;v#jc{UT=*_K=y5TfG6dItGcz2QZjGMGl+G1qTrc z;SSf2A%gEHCA`*>AFOmCglzf|LRt6Fu4!Guc5}Ll)yp#554y^k@8IeDx7*baE1*eH zy*2p32JB_}KKkt%cTIWPE5%<0^nl5Ui{vK z9FoRPp!1GdUyLN~aJ!!#%jh&kvyFS?Y*Y-5y8h_Y`C-Cy*vC z8}D!r9PMA}7apvC)u>)URc@`;*l|_4w(avc)ZG$R)K>1PeTo|MNpv3-#k|VfZF}@# z`)9wf#QyxuZ=;8!xvMmrvz%NpaFKeJ4>04OtTFn&%$haMg@EvDVo03fHr0vTJemq^ zbGFmP+z_m0*;nZ*9PY|{I8Um704en<3i6s}9myh!qt%PPphkXj0?IwS@zrOI97 zDjkkp@!=}rAvHnO*Pp93x_<21=*wgoLta5l57X%GIBdB|D|dcA9G#1S7d~@8+swtm z@$&e+4_9ZGifgp8n2!YOg7w4U~2A%B`qQGycr(Z157On0e! zlWmC~_7@DklOgEL;7Y&5Wa1o#DaUaHySFxaF;N%a32{?XZndkaw`@IwS+xAs(^0Vx z!ii>neDb3R58w;9?+LP{3FWz3Xhvfs@0T!HKom85wsd|l_R7fkZ~;|K3MsZJXcMLUL!u;0_Mo%C!$id&BBQTRX66g;CB zPeZxW4q{=;zwk4xA#{oQI+!Akl6`+X$Nzu%*}*|k8h@%aBoBeaN7V<0srnCv9&1on zTg&e0mP+PVGM|#88z@y;Fwo3|^VMjlAC=gkpI0%rZ#>wJXA|0*)9P96mf*vJ;oR7X?}5fw%(#+E6q=zrBsdZ&xpSwt~~j_6XtJ`04h zZ5q#&Wq8jg>K)dTT-T5HWFhsO43QhwObZ?uF%UaljqY(#b*(K8*+muZtFtvnlNGo`xB7^XkzKzK{McLw$QKIyfX3+$oMRg`YF- zk~H*^LtJLei(t1lhs*y8q5qnjd_(0&N76Sd5xF9*N$&Y}EnQCZN@e_6*z3#*(=BbC zuE3*cCFVDFBvx*Gk{{jnU$$b9e|TerD%?~OBIRJ!t{L=L05K)V)=)@;_Lq|$ zfu zx9=ZpV{jFHD1w(YT;7PYzA1%*;{TIU75H&uNFNb#1GS^ozHU04k69efmV~FsH@H6#^I-U~>bm`aFVgPM ze;5@|NTpzQXJPSa{yKS`nCBI8veHA-VFMaSfc`m4+@IRrZV0M6B6C5ZA>>)5qW9sZ zZDPp>tZU@)6^a0w#icSZC?h^SJQt_!S!jDa#tEEj;_?pfNa|YNpl8s?II^F8uvoJ$ zH>P92w+KMq74D(cygc&6IGq6hj8)nqXe<`?nf4^MW&Ip^Fgxq^@IG6|J5u=Nys`IS zkHWzQk5?77hMxH@!E*K_41Lf;yaJ~ANmIK46MOD z!P$#1GkR=0m`It(e_hagX{A>KiMD2;9`3T(J;cy2iFS<)q4@aA(h{#SP<HdYOP}1?u7^puMzj{!dQ?P_X zMFiOuF+`2vwyVVCxd`HAaeZYd!OKUIyoU>z-T;d+$ zDZ8q+c);DbWdzH%<$~6}F$9m^z;+=^p*|+RoViYO^UgETw4_gGH!jiN!HI%4J2<>a zhL*C`VJ#_51eGHg>~#o<9A8VRE?B=!zpy|u-Rv951NlKa-757xqd{u$``+8G#DQ+oE_ho z>*g(kR!wiHjnmP;k04}ZYpk5)!>|2G2c5Ch#`oVeL=)b=iDklilfzg)9L?1!B|Z{X zFpR<9e({-1IBLma$bAHx>FK-O)a5a?5zSZEP^H1bXFw?2&~vUCuoq>@OpE=27*a)* zY`oE*&?OHOuHfbPi>Opnb9w>TTiA&g@%$@jTm zo@!DQnINHNgGK-9}$ILy7 zLpRxQ8ao~Ct=2ei*$Qpc80=e{#!n?+Qz~f-hW?)A)ve7LaIFo#wyd3NY;J{36NSk! zwj9oof=AZQzPN1VFhRj zxTqe`@WP~K+9m#yfbLRF;#)L~rMu7r$%6YUTy+g@N$ywbmUPUs|+w&oU{YUoYlX7{9xe5M!R+Omwr*f*gu|?>4fZSvkvY7M&TjJ5 z!Uet!0@nPr!hF3Yss$5I=$jHmVHT#(qA^YlYWG)qdk2r`osl3jLhO5j148c8l#&|PI!GfxzjgsDm9tUY?2;Z5~6cP+Q8tQ9xDhuStj}V zVrL=T%42@1BD$Z`ibGX=07eDVJGo)ZcseW&!-2Q6Zq5@l&h4k&~tQ`q;J2 z_PHGny2WxwZ`M8!eRs=kN|ku?BJqdP;mNOZw8j6+vjX=|vGUT#a)MdL(gh_x6b345 z!;GCy%2Y4ZM}&@(1;o1`B@`{$dRMAp$GQvq_0jFyrKL`T)dkJ%u=p;eEU`~2ahmL4!5+|D>aIsx-wCpD0;g`OmA5MgedKQY#PO;WdTk%45b^Gkm( zt(0hCG!Sk{Q7g2w3u>RA>VAp-GcG|kw9FgI^nu3tu!8O8sgrg#hy}3UPb+YOmoMiU zi2L+KmJFSg`{GZ^eHHCstR`yDs~#k2B${r#DWQSQr>fkv>e=}1%f-*Aopwiomi@3= zisz{2jMgyX40AGGPOiw<9J-KTDesrnAp$NWB>O2t>Cza9e_2zIjArz8ckG%Qm+a*_ zNuW}`cSsB<8IgV?AQ9lWH7YI!v;Dj6?8a@kg~6M5q`%a$USaFIrXP^0y;9pWxXi8A zTV!B%_kv3$1_^_$n>uToPLIXNXf~XLfl+D_Iuw4Do;>y^s}r>SWJV@uK0mWOo-wwq7RXYZ z2n|;4o(vBbNrRyt<*yFJ9!RNI7)82Q2VZGC3Ss&qKPp77^7$#nqKjkMut?t8qQe{p z`QbKzbHM4sR)d+GJcr8WLl=_twooL1YiuEry+$)eL(IJvT$L@9k-&Z{fdnfk*>2zi z`&NX>HK(p=)2G**ZW`bBpIydKls9Yrpi~A8b-7d1_N}gGU~(T1TQghf!O*O$k+{pt zfRi_W`rn&5QI<-El52kS|Ldl*Ry=yUIFyWC8l=ZvE2VkU=11JvJ6v8}m2chlE2Nsd zR^BY~WW}qA`hg*khw8P#+vi{%w+dY`<}_}$!92>)7!w#!IVr1+z5&gHc8ie8_P3D` zF0(#bo?7gkdRoVzmhdfKEr60fZ+~x-%8~a^?#Ey~EAOU*jGxjM92%qV@!6J>wX(`& zs*eaC<@?C5hX4-+{~7uFrUMBWM`*mTdZ;CX#()w$Y=*tsW%+DR*>Yb5#jJu9yDgYZ zrEn-p>CpHtM9itTudKUM z9v#nXGAc+T)v_L2wDJ?QtwZ%3`~GO0IE#@CDd{)Lwe-MA;Y10t(KjG|A@zD<`*Ef# z_vT^|mz{p${pxDLf(7!F(@L7y2USA#@@JXvy5FqOM_HF&V0}t|Y17UkJH=R+f~iY%_j=p zbq+1977h`^Ew`h&t!D@XI{Hor07PL+l!Y7#ZYs0ugcIm9IsuQ69G-8xL9I=XNdoz8 zg{QRWS6gvDuDmeo(!9XXXLC+gOJfm#SrXZEHq7CdAb9OI&GhJXn)Ufrty{R?OJ}da zR6mCEW#`%Wcuz&9=$zz(Dktvu7*vO-^jn06AU84YkqtSe=#5g0VgANgX$6{jj1&%| zmN#*#L4{9pODj|gBTV3wf`03!1R5|2#m>OWupW58F{mFqa9y}NMXy6y0-5^l0Bl`u%V{oHnS z%mbUZB(|#@hZ2Nzf$t0v@)4}>f-&577XD1iUU$t-OEK)a02*br9YJ3I77o2!kd`E+ z!+`ej`6DLThGGa$ZYOIn-#pqacJcW0(Wia0rQ-7gA2gox_p8)u19#mat&fa%1MQ1` zwym;*eqnGBW*Js=wV?HQt)QyfB6E+Hy0DCtSj2wuMS-{dG0G{l;Ba`@E%DW13?hkN z2PTG2C2p$c+V08JPY5tkSE7~R?R2u!_8hM4(z=VTa|FeYdJBwPV3A*Q(V53rP)-+Y zKms;stiHSvSCbeQ_W^L|jb>(WfNLiu>3vZ^Zmk&CKJ$=6!xylx80>YzVhn%9(#8iz z6W4AZchtB~JeT*su>}D*yQ*F&x3Z!R=BV^Pa&2QK)Mn5eA03@ei)-#%SSyTNo%%(s z=*BJ_^B61;=1!XL_^vFCZ{NIiIRI+%!Bm@`COc<~V*2(x&Cq`nJl9I@`-^6cw5`Vf zH!@Rrg=*T^zk?THsA5FlT}79438vRhca%|Gur#;d)?7&1pkL<+2?F1k)RtmI8!92e zr$sZGBeHIf^l`#>O&sg%TJUL%tMx|H_M&$eo?4z4eU#tMkS|ov<$r0QD(tO{l55fg zitJYCp5Q0JyRhGW@Qr;B!awflzh(c-=d4IMxYSA551W>^igR?j;8OUe7BVisL8qlW z*KGn^+;Mi;ms3UXdxI;Tkm{PRIQzM_Fs>&p`5HQ@W#bKsSOz&dTx(l<1b2-Q_U8jp z*k`iTSAa^!v*qHfWq~B5=KoMqfj>TV@IDv_dC(8}ZdhE(zyzOLg@%uE5+zgdc6!x^ zor%OJ`k(&m$Z9sq&d3v>5HF4u{Lfmw47Sj<%>7f0IMEv3bQ`3UX#Z)99I zWn0z|{8`8tJO>?R4yWjka%fxMf0J^yYH_;bOWqOnA6Vnp)|It#K!=RLFw-mnywDW7;4;xq2j8jW=ow>&=L_?;f>7m?obmlec0)hGRxjBv(BT&BO6VE50$9VcLn#hLL&fO8w?2aqbi> z?{h6NS;b&v*ZFAnt1MJ=f=TVq@PRvHr9NdROw|^$ITe#6>@CgSeDro+eA3yueuo2F zqB_w>uGGu2R?Xki7ABWJB`EYJRHKj^?MnSD$g*29Gv~-DzOSCsO34|odt6b!WMxE+ zomdI72bs_Z2jauSoI7@WwMAYiQZScOJ~1D;p{NV{V)eD~H?t(1POB`C(w6k`O3YwM z$|RF<6FJxm_*<80>+|Q;_SV+#o6IxWn5e=iF?ep%{a)9|&MUk%sJfNVw|!msy@gZ) z){3_B-6`1f^j3Vweh^@xZ~u>l%1KU~2WM?%ZAli9*tpk8atZosvR8^;B^p{czUHRN z?t_mX#Kv^Qt&{7vMlbL;b27i?eWP#L&5%DtRH0zZ2x^YReZ{SJxTV|?!S0vny}@tg zS6WlBft-YX>HF2Q=E<}=hBi=ihxL;_lJ&a?W)qxi1?JUB?qPQR<1g{!SmAP)0GIw9 zVwcD2x-3%)yJ1}V%g;Y=wiN*9xFiP89(mp@%G)SiDQmMyS;2g#{8^x3Qu0q=Xa0GDC(F{3s1bB5M{${cdW}2r#wqXXt613*v`8?;N$wR;ZET(TP`iAoluOi;bj=o$s#0 zVbiM!XViuATjpL^w)*$L&10Cw9A|e+4~APrE5KxESXpo%y@pGda%ZCcRvHPu0jrDRnUGq1;GGa2}3 zy@@r2B-ZHbiI6Tt)-*_qc(WM|9T_lsV^yNq$}pqMTcLzUaxcXXD_wgU>$C0a3Yb6> z@BWQsCf_7|9hA4UC~oFDH{iToSr51|N13h5+Msj^WVm0Y2Le@C23I9-GFNOOi3E>^ zrLVjfU;kO21AnM|aBeqbe&deV$!&8nn(!~7nogLX|WuFA&@YN0lcSl}s_(4qMQ{5xGxY!un` z(P;W)nY@nMTGq2;zh_{!5>F8WH>^{dSMWsNF1*74h1D0&Pi#+R8a*WW^y`1wVB z|C~NKLp2HJ(_#bTqnBOB*2kwz5h|f=Z;cziX#J`P@y(#jmd{t65SGr#O?!% zGh|;beS4LnHqB6q7-aB`!gG8vVFJ_eD+>1EO;z7clf(S(?AcbX@OOW?zjJxg*Q=K_ zeUf&!9x6W%Bd7o`lBjKpwKI&Wy^~$#ogpsGplIAxJ z%ogowt=v~_mt9O7)vFdYv&N40DcRx5z{aRf!TdNb9-!ed5$cG%FbD#-yM6dEPG-f%J&eB`|8g9Rl#t0SNjWx< zX0W@mTs7CeL@P!yyX(YeB%ZlmNCOHgW|m-9B{>eBKKqnZ^9u7gS?fzl9L<=He-h?a z$0ch)P)!bFV8A@xUkxtb->VlBclSK@t>qPg>t*_EzCD^QIN(WFken>=;_aC<02CUd zGJ^|xZVwo`=8ZexlU3CZJ)+%43HPyMyw14pIucJ}EhiCDIV$24f@yewCFxYJf3VXH z+ZkF8U^>Y5@uyJjJBJ50Skbq>5=ZSTag}ag>ijHThcw6oqoWbF*ukpPv9WU-txh}< z8$06o3H&q8HHXfC*wKnpKS}O%Pmc2rMEvu7IB_(D)qdKZUP>{KY$z6ie$;iU>dWRt zk$01c5{mV+j{Z%rI?k)9HLhE@ciUPLC`*{{WH1>a_1%vERi$U|lJ|kZ-mtPcIc%46 zv*bz^ZrPlq7UCxhRe?#}iP|S4`Qhm%kiknm>@;BREhdy_nH}qcFF6a>7Ao@ddczOa za@h?NOvMbAEeIF?Vr~ij*=-;us=ezN5IN7M`_bv4#fB9~bB#WGr67*eQ_TAKHZv(# zN=SKkHtFtoslX22ds@;rWJt8(iZTfD2D{d|#ELYnhS(yX3qveP_IZZ!M@Ze_@^NI$ zHNe)|r<8I_WH88UWTe#6oov|dizReD)#prsg7*muY@xK{lweiAK~mp})gO@VQS6*! zB5BY;D&^sZMa-1JSnhb!hERs6-Z5git$J&p(FQH=QKy8Ap2HU?A;R$-dd*c29p^4B za{r>QL_v7C z&h$o1N*C6OVO;alhL)LA=1?_UAi+(w7CD6eo`tdAN4@E^0rid+zC`h? zCX#2~79kauIJt6ry`RtiXa|aKJT67B|8bZic>nP|CM449pE(tF=DEtu*|6|CW6^60 z6=@FwOhlBHwUH$jc-x7)QOuaHJ-@};8!fKc<$X{W8L`DYIC$V5S5PX^|3I-)2W=65 zLwQ88zkmvJ?T$RP^p#q>)8Aqf-!UbB{qHN?p}AkrLSGQ}g8jdGBZ&Gv^GmT`k;aLCu#R zkx(FDw1rl)GNfI|#i3I5i~Vg>Mr^e?ex;{$f||nOUCzmx%^$H?kH2I%-$WrV;TU$6Vn3TJWsi$E*IeFAeYsxQ}-Ij+(X3Je(9=I`mLaJCb9jO;1C`_9h?bT(3rNxTwBB){C zY17;u3y41$5$<_)PA;R%)NoYG6Tirmz_M&$@QDGGX z7iZ`bQ$O&VW=GkM8x?SjUi!`)pr5_?N8ZGi(cNE{f9G}|*&c>U>Wr;LbYHic`7=I( zPItVGi0n|{Y!_Nd;{2syg&Kgt>T-DkH(00%k2W@1>J4I`<-XTIKCt!q3ue{J*Q!<= zK?(CSba+!CNi^NgcUVZQY>AaT$`^G#bX%2ESk-rNdMI=_o|VbRAHz#MJuK_utd=G( z1jeY1O{+{DU(d4ZZL&L(|KDr<+Q={4@pDBbcLU=2o%ZPG*cZB*^?J9a!(*!BqXwV5 zab0HMy+FII2BKkSSJxcUS$aOGUR3mOzv|^!7`)x-tzC%t^{C=574>Q_MlG1JwA|Ig zZ&mo61KrHavp`W&2q?%bzQ(DA<}oufj>Q0rYm|2^PMSe1S2lNROuG@zBvmjs75j(V=*<%OPNJKkD95MJ#8B+l;;%KNHE z7Z(?MDq5l;7t~Ba^M$mSu&sT@*d0GfI9NO8K_-fZ!~31Y)iBorhHHfxvJVWqx%UH7 z#4Ai+orb*;)aDE{;tZU@Foww7M}T?Ojpcf+ULb&bd;hj_gVgKckes7dFZcMW<<+f! zaKIdz&s;dh52zH5ILYoapbrq}tzHF_S5J%hzUCaM7$6KHKd_-vVX(4fPVbf353=D$J8ibK{o`H2cV4nsv8UvD^Q`eYaZ=M3feg;Ps>%(G1=$ zEOrs31NBH+w7HRJz!F;Et5rK+Eg3W>c@X1p_1cFgM{9)%y|?afk)V|9h+1#?K~9ko;VxBV-(1#DTwxeGQ(73C-in>$Ay3 zpU&k)+d)Z>3a1l!$72sm1y zPujnCP@C={^iJ$~X!UjKwT9%?&zJZ;Eov8M*P2a|kFC*bVplI+IIv%#l+ehbtX}w$ z#Np29cDq*duG%qI(xa_sva3$nOgbD*U9@D(C?XAKJMa>uBmR>d=Nu?O+NZlx25b1{ z2ua34rMwPhXuAs5SWo-!np`DYS$OIkAH&))GCH(tm zxt;bXpHQV^m$v$d0ZaB)|4mr$z@vM>sS73d->plwF`t3?&CO}>!uN%(q(R;RHEpbePyI&SkoUuck%5RqRk!SdfnBTbzL~kZy zsbnw;h2pO7&aTHhyzR=>GpDb-qbRK|BO-13CKa!8$}ehuh$?l>9brX$vdRjB@sI$S z?rMSat5hyvDoo-;jpvYzlq5^j&Zqzqwt;|2s3t=5F8*h-o3DJV9$_FI?i&F`>cj;NewbhIxfU+TDFzJWZ^jnP zU)DthH@u7IXb>&4Uwj#rAMK;w(&946?Lu&5{H{cZS^2@6{)-baup-Mjy z`lWn1D}|xzkvglFfudbwOrEmXbcK?<)*QNNcUINv5ncGcBN|-nyGfbAg=-VC3m!jw z_e~P^p!wI3snvf+qe*$jWC>Zbcvw~38qDXpB=Yna~eRTA8QuCkjESu5m z*;^$nutuln;Wx$~3T=7@5Q3AtAm~qDh!TMLnK~b_Q-)rq)b!y3C=C<08tbsn)*ii4 zE?Dccw|XAdTT@X$uP8Yc$I=10T6!k5)!=jIxFshUQc3HHm7*#6tmyf+FUi<=c~TG6 zo7S+yqzV-{9~G_qNNw0}B^UO(i2(iMXy^j6M!{IGxAVg(FMRKl)-zxIJ)k1#?Hs!r zog!FW>|%|K>=&G*f*Ch2DwzJY=@Nl-(bx~AT-tY~uO8^8c0!=h{SEGZdtOoOffm-dETDqCmYTY`G{Nf<}*K*?52+2DS`|On1mGx?{30ZlVkIS`#t~n z;KIV|oqwrULp;!{;q8`#!sV56%*&&o8nLTusvbS)TQxsJ zxlD!^l6ACFsY<9q085COcYz6kV}IlsI5rUXD)^_dAa+9p`f~X^KST%`hcfQ9Mq{IWt z<8~8*EwPlewj*intg405p|<$SqZbhF@FectN1G5|bqHAgzq0_d&awpt7ca^}Qji`C zUjZ>vBPS~B;b@#)JLF;9BoF&|*LG_EL%u*Pgy0IPKI_(<%$_A|BL#WPq3!!CY%a6Q zK+O@<*y4j9sdmH}a^b^VC?BW-J!J5}n+~#pr%$6c*y~cy!-@nS%$u9I?yr9~bF$!1 z2!zlxO2|aP(*USIlRlXzJL`s?R~7r=FhHca%)N7`0CJdpZGPyS)B@m}m5CYJ0Z zLB><%x@oTb`!JhL!J`Y1I^d}#X(MM~#(Qbh1jR>gyD=*<$anXI9Q+~lg9?7RWlq?> z7!B20A(EQ9CcZTFRoQ!h2mepbw?_bR3QBWpnw?_99xZw^9=oq{fagU>&#OQ0%KjWl z67w>IOnVUlF|)dWsSXOS2CGvmI07fHR#u^w)Oz7Pk06-{1QX_viQ3cLWl8dJxjKiK zhth!|tL~oWzkTe{v$}I}E-I5nLSM1qazO1R+c+lFl0p7n1IfipA?vE5kS;xRb1S5R zeK!}1@A{VL{xHP{f58**V;PspXsgR-VN!M+fyv_=ja!8*`@I=XQ1<(e^ed7Dy#;l$ z*s#Fe9g>DUHrXj?W4li@3ftPN%d6OW~urV3Hgdcrcoabc{Yu_^d|0= zI}lrvhNQ3fyt1MZMp`v+8Q_7={fY-aK#=C;d{lDqZpmnGCP0|E4RVG;D3!23>$(Q0 zba*=)0^vW)4^}n>Jh#4B)yQ$BR#?bJk~Jw%Q2^TefN984*@|cnDaZ@D@7<3(Tpk1d zXRBx|mh9^GUT%Yw%9r{sDwVBEdHyEnu);!1exNl~d_Lfeg6IHP)pyKCz-3@1-k93+KsRtEGcdd7Frp<1KJR3H3Z|V89W+THz2A_zy9QXMgc-<^v}!e$pU@Z73F-$3 zo`Mxw{!L;+%`xZi3PTniDlX;7bhJU*g7aZFg`txo}XyV7V`@ z`qq5gpC}aBneZ}gNC%$#6*4-v>!vwg561ZoKnTANsw$n`WtH+aeEXi(5fo5J{sw;S zOUsJr2&pfZKNm~y^2-$@q~kF(Syk;!jx*fC1wDo&A-?>{@y()L`FuT;0ck>A7Qe!h|7Aer zB@@JQ+)a69D}tnLF!v9bu4B{Szy~suAp(i&%m^`t ze1q={rrgkj0{4It^k}ro{=aDQ4BtL3@IVI?xBe)clgaf>6%w9>0CA=(`5{x+{)8!u z*48D?CQ#OIYI!HWzB*hh2|&qj2%4<;o_>-oRCq(BDA@Hv0ffN3a+m&&L$=*L%XoGW z>gZL$yGFQF_gkUG+DMC$@=DW<493+!>J!qmK96L!^tOJqW}<4tx<77O^#l@(b>1~7 zS%-{CH!gekdlxWg580!`rMLQ`N425PhwAyGrrS;BK{LOn`~Qxiy?vMEk%#Bi^+1x` z63j~JozRyhz^l`*d&eWBx#|u9{Ai7eYAX`TK=_2L&)M)Q;CFQ?^eh)Uz)`n?_hVE0 z5^8oQE^~0JSQTE>%H}fyfLE0_!zS`V;z;47&TtJ>g&;4{9&*L{IwJ;Wcp*YjB-$k- zF|YJOl0X(HxG$_Iqk`9~e+mBnKhD=`$8XE)2J7^LWM=kXuKKfLaERs{H!&mu#|zD&L-+1)U8#kmy3Tn5T;&+|bwJ-aqz$ zFo;K~^bzf5rg;)hSq;R$&n3QZb?yV<(^MDGF?s>xe+?NwcN9R{B1?@R;?B;QDS@-F z#j8&2l$C@R-oui)NV^b6d%Pe!s5W!Q#4V*ZOC%p*85<5uP*8@bc?%e{u?823N-!Qz z@5tW*46dPNB0VcvjF{)ENe`HW&`Pw}Rgiw9Q)Ad|$9Is!W0I{P5*got7?5UqVk!dgBA#sHf%5u&UQm!NkkeSKFbYst<;6U8+Lg9-g3RMZ3V#h z>E0l@C;{3)rC4M9|E4#P4AXwEQGvB1I&dlDcibuao8MRgB;bw-4ZB+=`Ox_E@FLh9 z0VhmsT#Hr?Xy#jZ`=XrIUJ_Gh6B7-XtBD+qO7uS(&;O3EGF|Yn^8+_2sCSBfN&T}Q ze;iQvtI(Opdpt;UyG%xwQ{0yRG&=It2xq6n!m5QYy=qpyDN=RpOF8qq z1w#E!@sLt7r9PMr6}SZGJWh8fy5#Y%@80reuUoW3jh7M6NK2O_)ONl=Fnz@qS|p!x zM`MGY->OG{{dWzp^PP-9vFv>~HDSaAfbQTfnp&IXx)f)MYWt?4%1{7hIG{Heg2-Rb|)kuLq?|fe_;B?}FAhJpY9|vpskq(4p?a zOtT46vC;^L0Ai^pIxtO$IUH|tZtYtN0QcuW_Y<30Z|>&!B)hzU(Sg=!<9H^>e2Ux| zAab%@-_(KT)PH0u5`=frd{`e!l!%nS0G9kt?y|W8z8@yw0s`SkY^D8ZFIwB^SEVk* zYI`#Il zfSjcVPAp;c0)0gNwuR}zI4kr$6sTgYVhE4ZH>cb;(?!}rUv<(bbQ|*^0Q>o@$u2t3 zNZ?XQc_HQ;UIA_aA@c7niUTa*;2nD)3s5`E34KuOh;GQxE;#obJ#~ceSuGUGRILC# z>ty-%yAwQTaO>!OV*S0D5%3Q8|L>k7ab^n>|6oX&z?CN^wbh%O$h%7@f3^~#3weTF z=)` zTG{GN;rOZrwc37X7&bl;EzzciWur=J9#wJpxx9SIVi8(*TzDN*fG5H%n%UREY!4VP z?e!G=MVBvh0`2$z1hkyn%1c{E{>?c;(Z>Ok_r*}_y#2WC+!cENX~+&$A9WJB3;mn3 z+>79u*?T~R5s|PPzxJ;%atfOfm@xlh?xgVH&ztfti$|W2H$#EOM^IP!4@v5(Zh{Ae zXK?jXh1aQJ6Gl$FA+%8ioi|s&L9`Vx1>7&V_?gluW1{$f+wX6VyFG%m#~exqc58BP zAtxrZM>e1}1Xog1YgceXwffE|Y+4Ww6YX1P&T;3SFaat6uJ4vT4{G5pdJ>@Xi2HXG zX(rQPndLn7acF6-qt2gp-)udH$V*WkNSmD|dR-;7l_jlg{^?Cu8l*S8;4M51p?BUb z;ozPv{4Ed+Js*+t+^50>eHMKSd;a^+JQQlGpq{<{Vo;-cEmybEyEz0*16<0{Y6`Ld zp}5mQ8g34;J-gpKMyK2iQ^(>UTLv!OTAI-<6RcXI(aDnL9{e!qh{XQtPL%xyV70&% zZ93^ViFL2!J{Ntp=zG6M_q;A>fwR} z=u9B_^8rM5cjxm2ED01&R-n@vMf(^fuFn3vX97V+aIIV<^C|pLt+qo6dnD*%U5}Yp z|Av`k_SrAXg0u;qWfTCCw{E`fD*vCBJXCC6w+CdwQk!tap$%k3i_q!pyI~E+TS#p1 zU#W@mFfwCiP5o~{xo{A4T6Mqf;K8CCJbb^IVyZpFDi;2aRlE-!?~`N*Dyf4!e0)c6 z$pYMp`nK0HCVEWb12PD(K5@`I6COQ!bd;N0_!tU|z<$ibVphMvW{QDdBRFPuNb3(J z{6E9sL2T$P%Brn_1B*i!NTijJ7ZUmgm&<-dYLn=*&<^xl)Zp(x(_v1C(+9U z)XnXWj0OFBAA9LdKnoq{`U~;&Ey-O(1bIU5MBTaB5f2T8NW$@x_w%n$;|6XWOqCp( zOP-31EODsrrlZO8P7rRPyGUx7q5r&=f>zc^Kw`sD?D@q1){YI>aRos4>$j%tn_fYZ zWr1jkq;=w~Kj7dT=m7|)7jGT{d}eZH3MoMsbi&h$8)Gdk6ia|Kw7{1ANq{LZ;kwrY zscHmlV8eh)t*Oma()y>dXuFIE(1SpUD z)7z{iz%22hS*kFlr~7miQo6nR=Lv*ZUuRgSOEP3TJac&{iKNTUK zLy9S!pi%~mmLNy`{_k^qwgn(W1};@9JK10Hb-CQY!QKbe2S}Y!@L{V^KnQs-LkfQJ z1K!pJ6_0hkm6t<&n*UFA-eO+NR)aNjhStn$N)q@<=n%#g{_y1U$-mq@7~$;|(yZs_ z>S{ZQa43+3W-uyQCJifvshF?WmQQiTdY&ZtNn}0X2UbDpG|} zg(nDw3RF1qfYMRYL<6dRgX_l)rS;AHx>{~`K1c!O;FXh7cwc9wGo%FFfQZm2et|W6 z1wOp+Ach)3RybDGt?A)^fh@n-48&{uU-kAw3y|iaK0=GiO_z?p3u-{+P}&Esf{$qr z!e?h=L}DSJK@S;|W(qmhUGx8fNI^&%tT`=^+pdyeGL3`-dB0=1%&%C-jx*=Bzi1*2 z7IH9zKcMI0`GnQw|CIrhIj-CwJjVBnc)vmnB|zZ%ew7iuk<1`GV#E~#8wM@B(5Xv5 zkip4N0R&uc!^a92K7cJhdd|ZjnnMwu!qeaz|AGrkO0w$Mj^)-Nu2i9}5I}?pJ?k zC_4Qp{XcyuWlXLSfEN>1>8s_-@EC}KdteYyHQA~AKUM?!N>0>{F$72&T%r^Ugt|Pg zHYR7Yuqt1H9-iaviX+~|wTu3T2b6P&Rmv!W1#2;x`}^qgVVF20Jk^l6PHT&vU<+sPKK8$xQ#uz`_u;LK?Uh zqL&ckTq!PQok%#idtA|fQ&ukf8j9yJ;GKiyjqB{@5Ab2lYR@N~eI+AXn_p-sPYr>K z90*+KDG-UpOgS%d|5JnCw2Y7C1L~GbbBoV|pqUIqYg{yDb^eN{to7)hkp3_+&OmTT z^cF;}7BTp)nb`cb^ttu4`oW{$^HYof4nozpAc+ANvtyN;@Elf{LaQl+x6u+a^;>W* zRslbSuH%LksupAe1DO)V14gbRM(*=n66*_`;k|6`M#wB%A~6#p?PzEX`)&TkPEmPi zViD7MiSDK}d$&MY6DLQdSeUZsP`bUA@E@wBYKk;?%6~p^Ep%9 zD<*-+rf-N**2Yp5S9J<3F*puG%C4t!*-@40!RzB>Fw=?iZb!5Gin+6`jnHgCu2H@* z=B$T?yI(+rnv;S`yj}S(^4pOg!ZS3TOf>kkA|)?lAlvhh%OS$eX%>Q_<&SXa)j3{%U6|mC?J?5gr`+>3OewUCv)J^E5f~XX zd7Nyus_}gN&vw1$8~-9rKXu;EDVclAG7y&faU(pY#UAQO>i2|3ooj**5KnJ zz|0vz?O6QKGo4h%RkCUT#tVetaUOKMXhDt7pk0<=23zFZ1anfn2=RdGhsh*6r5$PY z_5}Y*U3sxJGQZA#)i9IR2jK}vkfbd2eGHtWVHDC@{SNdeFg>FRuy|Q-xgY(yH*79K zX=m^-%7n7*)1kVWk)4Q0GI*jH%I&^%2K|n6(Vq{E2sA1+DQjB?qsrfaj*b>-#0$M{{;vS998p^ zgIWf|7aoRI$W(umlwlo=jz&gK7NEtr3lV^-k9!`Nzb<@(1g4;Swul0Zj-TOUsWg%j zmCaZ{RzCGEtZbGOZ?G0xHI2Q<9P{1(vPwW7J=j|it_n1)h$kd$`KfqmqGCJ?tvfN@o zBY|l=R3GaIcyD5T9#Y` zSK%c&LF681ThmA;vkFj)3e*ZN=pH2*wpD)+?&b$QNL9bHuA!4meTN6aaTVm7dO*jM z>tC=L`#-S9j}R{&*HGm)1;Tiqk|2QJmFl+d@Ut&bjiKfB=h5W=NFRT2NcAMwnq@;4 zw+Bl?;rR}xlCywoZCSo;YAYj|&z{kq$lBtea?DE;Xh-G7irS(^p~=t8O3XA8hn9C_e3LR4&{Tr(W9=?;c%$B)~&y zW3yMmB|t(l+166Y>prg-SMTtCzK~g~_>N6mg3r+*rvlk5=nYwbFxd|&Wyb$WWw=K7 z3Ie5}o#vpR@YCG1m5Je6F1vrF^dp{~=b)a@g~Vjz?H;tOOnm!~l;SJ*vrCTze^Q!U zJx)bC6^#v&JG#_B{t)qEi|rH{Q0>|B4vg7VYvY~ z^xyhD2J46R$9xXQl1@_EN5*P@GES*V-7j1Pd_9qcu`9^UAAw3pAhcYLuCC==DOt2b zy1-`DRZnwjZ>X{vLTU<(wtdXa5#au0mOFmsfb;tF(!jVZ+|ZV(<}buKWjl#DiMYXp_Wa zq3tt&m?;y4^QwiIENkfvJQl5E+0^kAdGUplH(L}`|LXXeB3i zDx&~PypcynsmZJ5wA1TlJrycX>6?)rC{a6;W^qmU%hInF$gd2HvbOBqTmmjZl;jP= z4^2?f8yR42sFPWtLXNnxI10V0rWtVD~O`P=yN0!Nf1O>qOVhG0bXEUHNS&Hf) zPwLBi#6wm*bX;vL_IaT6S)s}ClGpH@&jU=r5HXW};Y)l|N_zRkK#4@&ngVN$l!106 zb~|Rz>%0~CJTd(q)eSBHnnDe#Z;xnQ*_i6c)aI@I-9?b2R=g!cT439}m2tOB<>41` z5_}9V!byR2UCq7u6ZZlsq5@9k<=Un+>`#zj7jn)eg9M|*=lcF`EWkm5Ddf5BKmR$& z{E}y_asibK)IZ9C64*hq1w-e&EiR@)DXcQ(!jI(CpC|hxe<~qxoB-2c3Oxoj?BNO4 zs93mdYb6xd{fT2*h1*V*Lql@}@M7I@Y2~}4*;ZW}WOt=L9VbQwuu<@mgPlw&b%g@$ z7&kh+IMV=)6`uCUo^j1Epwz9G_m0Bm(sJtbDqERyJQQph^Efyt#LVv7vmh$`P?LZ$ zaQo9PVGdULIw;@m07>>gO4&R%Tpv{$`F=pQF|EJxGdN1bjlte;tV9Q;6JD$!R$-!; z8)mi891%5D$49|u$y@h(>|0dg=14g2Su#;1bRoggdU`!zR_c- zt*}vHH{ZK+0w4x)>x_#Y%ZkXnfSK`_zxD4?C@tVOV82;oG@trAEJeK=3HLu3_7vDT zuxq)1Q@kqN%tpwWWz$&hDo;#0HnslQkw{bN|a$*M(G^lnjRr4mEEX+)ej+MEDHJU*SZvh# zE4VFXvr1E8_YKu;z$bBmE;i86gPQ|TKZmX(*D`DCR5$3xwBlvbjm<5=6IGF9bTxs> zTz$~i2P;tqoV5&TL}fBPZ1$w)$9oG~jTRGSFGt=v;@#S7Gh?fNln^6a_|mW1a!YaJ zg?7&PSLvZ+FyOR;(y?{M{JG(#+((*@4T1Xe#Ok(?<2sesBQwrCP7O~H&t9hXatyr> z;P-iKz&YpF%=x{Ke4zbN=y;Sk4pJ|@YfAjF!GngIlb&@ZjM2;4|NIK;+Hwp~YW+^py3^`BElLYQ%3!jteHY+gz(@Ojm*UYH!4CCuf*zkA+m4eE( zD!Uz&e3kE}>*?4%ojLzg=i=1o*M&`fl&!#Uwq(Z0qD_L~Ed%?vA%5r%-k&=A++0dM znKt)mBHz1yD~q>@FeLWPowlw?*6RrNv9O$XoR9a;%QOKH`mt~gI6sdv%Q*XZjL6Z3 zH0nz+^O!5cKyW;J9I5}&k#7#VavUeVks;##HQH;`TRo4z()dt&>&E9X1c;ePN)%JD z(aIHRj1u?Rgv4}@gx4%fehVTTUEL(jTYAi{tE6%_1CuyYO&hK@Z6}#1Eg_1X3OzXy zn@j^>MKnMneIUQ(*FCOQfS$3U!KY!;bl<#yqbpOd>k$cvd9&t%V=<~diQDU zrpBFIUo^qjVp`s@kl#kt3vE3ZLC53at|CE^;a%aA`ytk)tl@e>6C-3Fe%`D5xmNc{ zCrqa$ULsIj+z9l4|JC6c@pTe z$x~H^47|NZ$qZqE-B}p0(+7y0$O+D|qL<<~mg^(Ld1W zsnnD0@^ad1&n9Wp8c%Jle5lcMpal)|AD2e_wv=x4d>HCetcf3!bXvtVB7DbU5ap|$ zs!onb8tTkWIIT_#G$jG-#_ww%0DSAeE44aGE8BeM>v~D1vXsP2y9o@g@Ai=| z$>2b0pYD+iImK&z=gk*(x1@@ywL?D7j{jEYMDs?ZC~|8CGB4V2@I89)YJr-AlaG$` zm981-ok@IV+#!zWpQl6aXV38in992D>`dP=oh3HJT=`f6W{MZN$DxjlP6ZAS{;CGa+I%Lub?VKs3EMM@T^t~DI~fR@>shXnTK z-3r3X9py7M37_jJRI<1i=a&+`TVT63+};E@H|>dVns=hqIoFrs_qgu!2=OuPa1s`= z3sAjUNEtL6^3{+?qgIj)+N>oz7*)D2?SDB?KNCCb6<}*$v3zvu+;j`m(3&@;JsV!@so1u*VnC;1(ApTYH{XL3(zmSl z-dr(aIYZyg?!nCzTi=`VkjeZmdJFFTCGlg5JS*g3(f7~q_Hi@|2mdhg3b%-I2PE07^SGEOq?6#?%-|l|gBB98n zk~@4S+>Vwa@eaL+UM?u!dT3>&uw!YEqXDLo7kyDHlwZ(yS-sWCOEp1Xk^KAMS?R~; z4R%N{^Q(|O1o5dXruCgSw*M>}&ablF^RBb61 zMU3mnSI$LEqOvq-8FTZQSDecx8CzOBdjFQ0|Ia~!U&3LHgRiCA0F}LBrm*+oB;L7$ z^66w+5?OQbAF^^9k$NU@q&n6ZfpC4{eUi7cQ7w1kN!cF)`g>LXs+4?}^mb+wF%mt| zdHFM{zj7d=w2G~q($w~sgJ`J>0Na${5Dnlh;@D+X=PzLVFx~?5ME%|bAAfh4C(@; z2=`wFEM$g@9Qj5HC(cQJsVM88&}go(sK!HmC6#wxpE0PqYe0`#TYFc8?T(@*^x=$0 zpG~p}ht+!aYO4;r$tHJxtDyF?1@4wtaw_wM?oPl9qA}Jhnrx23l{O+^R|9_npGycm z34!hEj-@`cQ6LCW*kX;Bw9iM0bM=1>%&y)ZrIdNk3&)B54-!FqHO)%-vimgDW4ER9g@ zT6cBfn!}fQni?U}%gaM^YSoK2ZQy-3=qX>MDO*fhC5Z846AX8v@%`;~D}b15?~kd6 z{EwuK%-1Sr=p~QUn2!|4iYPGTeMi2R&T3*Lt3S!)Qt+=P%I}Hq@2f2etUF3R-o#;Ijn8 ziZcI=v_eN_Y$|REOF(L&m9$USsO;AUJ8sb2y8P9JXZ(a79O{DbX!tjvuZFYW#f(p) z8WG*RG;=gM|8;!fp={4Mc#Jtpqph^oQP7~DJf81f%&a+G(?2}4FgK{04Sj!I+Bqav z?bqBRuuN6*! zTMt-*JL$Sp78X;XvG$J_TbP-xx2EEFNoOeg*m~M^vSRE~*TXVpAgSP$jpa-0E(~$i zT-NImoW(2Klyo~w#wvpK#149iI67?1Q05C3z)PnAFP#B-n?l#@xi7PPO|Ynj!AH-V zle9@V!(??QJ2>T-p=z@2HCz%9J~)b9>#a{+?)gkk@wJ;K(m$epwcsQchkhXu0Q^wD zELxl!X~Jj)EnZ=!_5FtVh()?3v&f5x6&){^UX=H-yD4NKS4m)J7#;0?s;v^z(;sR< zdppuAQ_7yVBwZ=1L*#7xb9bLy`#RhA&Oa;tD(-_S><1h9^=@KkN0jc&!)k`&cG0*f zXy!kXyvf4*8ZCpditLEST3f3dP3XH4@QANF5A)y{aQ6-r;-fp>+wNuY3{tTY&d--& z3Hs^F?C>%9cC(?r%S#wc&T?vy1(xv%i1<7+6?7s;<2c}Kbn(~1iPyg@87NP<0n73* z*Wq7${P=p)x6l#*W6~1e*(*v%nZEG zex>}}^7OI}ITz>{5!!f;rcr-7jPST{m&bT;>`xdw`FHOeiPaqi3zrc)M$oUZ5a7RV zK{r|d*Y0It<%vjeS>=85IbxMJBC`fK*4@g}rlOM`c{k=rHkQ=*8S4b+CD$nDV}X3l z;QEs-i@_vK0kJ(F2SOV&p{sK8U!F{KD1Ny)@PW;0B_sX>MMT4+4E8#pBU;9YsPkD_ zd+Eh~cTtF2Gy|PeIlM+MbVJ$6f8p+rG?U(w?rQ>WnQ-(FNd?J+1^!!4G!G&^ zIH@#BDGtlQj*y|xm)|sRZ}Jy9oaiFacXV6pK4!RUz1`}Yv+a8zZ1&7c`zu*^*#>1_ z&kLT)?`)q2+O1+wwi;| z*waqNrWN+|%v7EfM1oFFf1&X)b6vwGh9KVGEgb_?->18;9A?_lSkLte&+Y4Mq{Gj91n(~w6o>u-~uA_x_hr-O4M=sI;@gv%$ z%ke;y{;c7aLn#~Zcf4>=B`BD6WINgxg z0Wa8vffMO`+B2E#S%p>f$hhphj>fkEA0Y6CW(^mfAGYWne}Hh~k2!d;IKI(=(-0pP z5x>6GvO9grbn8^qZGS!1+TB%U#t9Qbw4G7?ha2FdE|tD63*oAvvWXJXVZOn#>05CX z3Za=ri2?675(BvSk@+Ud1qUzV8oySgyjo4R0qGK1bYWN^f8s^WHa|Jp-mL8%64;Wc z^46mZvtY$-vl<4Cl6udobIPPwcog>9?XHU&yP9abIlPQ0*Q#Zawh7HRieo0ZyErpZ zMowOu(M1*iN-ZPhW%732CroX^`sGnSwni%x0PDZ<2*-Y21meBg&|q*>frsMUL3?K^ z6WAMdLv};)?YTSW_(@{fI?xKwzhV0GAW2U?Z^Co?VuSa{A>>8Pw0WHE!1@}h$ljhV z6Sh2`Wqx|M8Ywq&7i^mEcDaaa{lmTH%jlT9$K2O%l@H;@qz|?IkUZ<5vwR!Tc}v>d zS{RuHK7e^>pyXIluTzaTr}S|e2mQS!DsI#L@DQ9S(Ucl!WHf;fVHVrl5EnV$ew6mn@A0G1OUwqq1(LJ3|k+jyb$U&K^5 zpH;|11OoexzbU$nB5Aq#+b5jZkXYfv7En7$EqSc_>N>>M1U!H}xAlzOh115%{^+am zas6Zf5iaJR&kG&4uB+%982^f87s-QJ9v5-5mfryQXRea4a?z??j<1r*cfkX8~DBXB!2RkjRLj1dJE3T zIhGG*_abKh^fnG{>KGVvmT2?rgqlWPh*Qhm=n8Z@`&Q$!7pGBQ!?Gs5+2?4^ueMi9 zX3r^<7f_if*y|T`^MHp&aN@Chp{AVnSE*`(*MT%5HgZ$J*P+z<=s7^h1@#8icuCwH zm4=oD(Fyby5S9;UXA3j|sQFJ2HoGnZt>dM4Mc-WI#L-FS)tBeG2k&GjPDM_8hPa&W z<-EVPVHMfe*VSe}k~+{{AWhyd&==jWTyQLkTA5@`JAeM+k9v6vGG4ANlc}vx^9~~E zOSEWxy=~-g{-ckMTOJ9jw=n~>_z25n+{p9OeRS@*t$7oaa=f*K;lRT#zKzbE8AE=s zXN$ue%OJj$XyC+{a8UhW+iWMu_E{HFo+Z{#+afE?tD0TI^dos$X<~iX=&q53ADBjN zSGow(r!4>J8#m!szx{#Ub_OOtYFy7RYJ?yiE^8G#W`{Bl# zOr@WJax=a*e!ntEx5NDy>AG!0(!z-6x3dG8?6{p+2>RUp#NMA~arvjEm6vHo=X z@XFzz5w&{`oy{K0^T|$PAFaz!g>~zRAkFB$D zXH>yj-&R5R&!XF790kwVvYYs2G*j9nWLctpHXt^u-Cd^BcQeq~uG|;N<(d^3BBWmB zXpm~Uya?KhvI6~;vJy`whKW}MIw}<$N*-2b(N2Z~ ztR1DeU4{i-{>NK5-l(OAe-~tn2nmqE8gbY5X|P;6YJrl4xj}&XnE3i+n`C&*bJIRl zM(}y^1dRK=*elUB)JZ4#yr>z!ZJRSLk%-Q;2 zzKlFP2g&E=P!WDL%El%q`6_SRbp$Wc{kZN$T*Kk5ey_$95w->i z7@d;7WbSh%L85z~$L6aLfke3d=Th&76Rf;QkVU<%rugR@5ZZTfvQNRB;!j-RL0%mk zs|mNGTMrBVmaM@q!j{8qv-)DloLi&2tJK$bbz+*W1B;`+{Z>~V&1=4HmD2fPg*W7+ zG3~>#+hs(YmZ}s*sIf#@7h-~*kJ2Y((*^=~w4gn$6tyM4!8Uuiuwzn&3!Js*H|KmK zppOaPp34D9#pBm+f34}8e0b;U!^y-Nwppj0DvinlhQyiau!uHSesj9gy^j?N;jK&9 zMZ@QY{4W9m@w@iAx~$!-(wFImdS(36)}h_wLj~5YkI$4JBd;e?=Y==VRy;gx=|X`- zT2PFB`JiFu*`z+g%&;|`06_P+<{pe}B#c{4{j=uad{d0h|bsL`;)I4Tq09peu z8z?79&6H_^Oy)mwSaGD|+z%7KL`2fJB_3Zu2?`MMH|xCowGpXydbp$eJlysAzI8~fo7;jCuoQMS zyc>2DW2~1}GI{yjhjrN5!si~vDfk)WoOZ1Qju#k`TB*Eu{P#Hmyj z#R?&P&WbFHdKIcc^WQ%D;gRUM=)GtS(XT4E4Ow7tJY197wv7r)JUvQR3UODD=w4Bx zSNXNR`3mu4CmR0^-Tj+*$(be4#C~h=D_hXS))#1kd}YNQUPzO7(UGAXAQA6PS65@W zpectr;xtN45k2vj4^50WycriCusQ{_1=kvj4kclfrLH#Tke8{hueF zB}UNn$DHe#O0H|Q3Co>7|76O@`lXuK8Oz)_+z*_Hxknj1dmgWXQK6g)##t8_$EpIF zWj45R8+Ub-9H60(wO>%3tv?38u_@F(Ft=)kFe~dK!(j{@F_AE#RnmONa6a|GWIExc z#Q6i>3vr)ADT_3pE|rV`R}}vlJ<;%~x9+f&`%RP(MerI3k6WP8FE@Sq#GQ_!M@!;_8-k}-);U{W51x~B%t@?*!KFt2n|nhNXHo< zEU5BSXtK$HE*3B`Z9=Y6FjN>H04YQX5)L`{fDXF2$5xT=F=MQquDZ~+VY?K<8a6Hs z_-9mS=<8&v25r`yLQ4ji@dUt(2m50^yBoE!FY|s5*)BIIPw1NgoD0$7`Z`q)sg?8d zSktKlIiC(+-YvRKf&D7LPb;Ux8+v*HQKel_Zcu&s2S4GBVb0~1FkFO#PY6nn2@rmh zRxQL}N^8xcaRz#8egt@$%~pwTIiZmB*Iq8+q7N zu=#`k^$qMsDf6@Pp+@gHUejmZx?puSpE)G0w9tV5?@|z|@)FoMS0>-Iwc# zl(mx1=@EDc!izp+#K1}Jw*-TFq16|9Q-~%IX8V3N?)k0S9hC7`xiMFZpw{b^rfOhGCDG`V}{ri&Q1wz-6?@vh^2nv&f*Vx65vmjZJiy%k5v9 zchyyUMqsC#P?#OQ$^t=V$X%^ino!$He8%0brXxDoxs|4vTdtvRS-`k4hT4u=EvaDx zpA+~>-FY22Xrs^yKDSRmR~VQw!P=q>8=UX%_c-=zazj)b-C-&(1a4e# zxH`I?6P%^lyIR{_@@&J9vSfyl z^#+aLRK(QcQ|tp0r4_t1DdB(k`=h3j17Au&X(`Rb_??ngMNQn*#h6uD4v_cS&pLwfGB%}~ zZmDH62~OO#c$dJ+i0J>4yq&PpU#@wz-!{3E3Jmdsb4A5W<^MBan%+_B0eHn15u$^WiPc zdqS(aD4DT^?{OzVSOSlY#R~MUf+i15IiB*(owsqaw!f)OTcEHu>0TalnNI4yNBmL#WbBxfWJ{}jR z)ZV7sksJThy^|qMPM=@MFnyclq*dM6$(d*^^P`;$h-vCL$DlRyFGDlH}3HhcNlq``oiPKkah09@1;W#5quS(&ft~0kMH2Oy^Y3?uZvaT z!Ic(3eOV{CHDa%CW}^*&Y6J-@o9}wu)QA3>gLvR`wNL*DW%0z21(JYrE)<-OE>6#T zf%RuWx6{P{T8Yf@$cwCdE~@EeHv?*rE5;@2)Mupi2Qt8LjOwKtLhC2Zh^ zG@Hm0i#4Cpm5dNjKy{oz?*&!iqumdX!{yPcV(={{q<3?nTJC_bhx_?XU^D885HpgC zMlK?B!Rije>CIFoUCuioH~QzLAr2`S$9rHS8_~>?XF$%PlW-X7dF6Ceq21eoBC<`2 zI(sLc?z7-}m+tb_IO}HQdZ1O7)n3rCZDh96`u45!T?KyeJ%185DXR06j4qp@1?$7c zk5g6U13wF8a9#q})DK*Z*(5DOh#KoA2L{}nL--|MF>ei9Ai zl}!r-PzHEGbvZ!ppl1wI0LaeEE>}3kZA}hd)a?&u$En9TZnV`poz|F|Zc3Mq@a{Yc z5V+A(_L3IRbI7ZJHv_-k^{db-gUsg`PxHS+mBHN9`s}v0*J$BPXyVh~_Zu|+MMU^Y?7!#|zLxgt?U?xS-L`m9`fZ_iidS z-Sg>CT%+a0lNZO_pc5q?Qvr%UMVev;C?X&Hdu3gsz5!CN57*sKa%&DdX)6o3b`B@C zD{3A9=3e@CLLD_zpg*GDSte@S&-Jn;@PK&%)CwJrV8^RO#{sC&kmL?0jXkf7e{oT^ zik*)*EZ6`htQfq$!p!1FhI03Wyyj18$KoLM_MM#yn?4Px*&58|m68-*?i}{@-3I*t z88d;`B`<=Hsa0nPhv>;fZ9rlQD!HI3!eir6;gR7j<&Cp}$}fL)Gs8WP9gK^+t2!}t zmJ`+cLd}<7jE7IK2N@so)?Mh3pMa(L?Mp4Hn;-WHv7=^@9)8nx_0e{I$4jGml7&H` zpw+;7r?+*ry7~P@Vnn}5@?Zjp3<-C6vZ1AtZJr`sypX7b~*ek`(}d5hYPu7cU>Nijr}%ma<_f!T@;$vd|bAdB08!?vC}s2&20) zA+%ToJWR>YDEdAqH@gknT?c?R1*zYfe%23_hXkiq_+H9hXTX zoV2Dg0l$yv*C)@-o6G~^+NFn8k2pgv_x9EI{?_DMbW5uQU(S}60K9!y?Nc^q_|~2C zLWTCpQhJSpzL+xTw>q2XRdFW(;bw(Y&BVJC2SfoUZELyj&xVAA4XX(Nz5Wuef0kT< zqn^*VA-DTFK_iYp0g%|Q_^v?R%z}AR6p^ELq}V$WWz3>DF1xGGNRf;QYr4n#&Pc@3 zOATh)^KY*>gS^J_y`>@F=LK?o$jSe8c|JRA=f%}Ef{tsXnf!|v6?4KA(XtOE3tUI- z7)6(+w%$N!Gjc#UgY=ts9e~mDdsdy8r|gJbpUlPzrTfh9@0_*>{+fNt1buw?-U7>`j01E{kt7@RZ z6hw>UdTug(!1nb5BF?MIyc%T9{q8UjPp^vGN#8#ryM4=kuqV&SkR4(E`b;4zsMUU^5hUrCD>tQ^DoZ`B83f~ zQO#sjPoQ^~55;amMlV8BIle<+It!ZAesi@6;WyFFW_U``im=C@6g2z$U=`nnu6uKG z8B~LhtW>^|aSuEJ9crTIK2Ha%SUUN1Cmj)@vx*P?#hbr;E0IJNXQIS!rr-P9;u(Ye zIu-TiK(4rfjh0>wH=Bgec>V-m$t-8umF|IUL{R0KHHOuflQyH5^O&?tG_Qg!fQ<%J z{$?rvo|hZXY5!Hgjo5*v(`r(Y@VRv;ZgP`o@?Zccr<9P5B-wv)z1N@{jfOK8gKl2$v2wJGOqyunns2fC)-J^RYrLmD%L95q} zQL_yhAMrbg)yk8Br_!hz6igY}S&3Khz(RSjy(!fMn)Sa2mJsB73u;VI;bSwxLQYO- z0bxYw^Gt&8neOUAf_3lA6YJX*y00#yuLf}nQptT7XaJkI0~>9ww}Z>U$FD1^E!R&M>BYRO#{?jfpP`rD(7pV0m9H^#m%8uW1w30lA9txV=^dE~uL z3mUjyg(S9AuIf*|hU}DSGZrdgdwfWuhNen?lMi$%!6v}*1jbP<3O(@>D+LL*jJypg zFEt8mHve3t1U;eFoLy-DzH!!6dQfOb6MiN8kB(t&@doUsE-om3==}zu^;}oULQG-S zKRW8~ENzs&KkpSLP{45`^J$PAd^t6hec3!g!M{Br7V7@B@}(|gUM1c5U6FYbWCUtG zL?!Ch=k2l&C-N==OS5_zS)ov>d6k`g1$Y&Qm^vfa{fWT-YRO1~hnqy9jo?ciua@vm zh-J~#ZU{qP=v(`Thq1uejyztG%QjhdV&1pB!K6nt&(`QyQr`Cnu0a5hVCDPxW zZf^d$Ab%5hIft^nxt}t(GYVWZPe7?Vbn+rEE@5VOpZ4vHA76heV+?vF7v^t%)73p! z(9!~M#Vqmyf;X+gj;8UCc+yTXJ0BzK7!X(g!?0v1!D+{XgL#-LdM#VGo@7C5{ zTtpOBXYqb;SL8Y zalzBJZ%|4;?WGa{ijCbc$dOe@UNDeIZ*_aPVBC*KIKuwTzo3MD=KikDQz~(d1&|}p&=Y^$+M)Z%qnGmlRFEk+OHb;}0=}QJ;S$3U zeFi35xg_3Co8FhhxjE30!8|;*OszgM=EvKjuvNbCmG1jjdWNl5!!tKl=5qNpLknc$9${geTSCEbP3#H$GdlnZQfJ1Y8^tDVz_SDbh3Ti^2 zI3!pVv&hS2QySZ@1vE2bC0MDeknBPERY89isL#jMqW9u2!MEkoO8Kd5KQFxfLI(v? zf;6IhE->+ExKAf0`Q5-^;9<{LH52B};+KNHYI;xzS^(U0c9x5v!=5UWN&uTr3SLpXG1`EEW`It*qg(ZrXV!}FJPn^(N22Uw&8 z&>}R*3+(OcJ&03@d{2vlvb6sR3W+J_An!$qaMYhD%i(76qx@DsDrwZQDqvoI^t2s7 z{aM$MnhN$tab=SR`fli6^zmxR862ySWXO|F(&V`d+vwx!L`of zH+%0lp7(j{Jssqygk(NGoE?E{kQ_D}dkjb^Rr`ue{}^^58rS)<$=7N*k&I>-w^J}@4NXm+2rv&2bu7;B&>#0}QM8_* zKy-2pdq8*rNf-F5ZMLLqRkdN$ML7Q3@~P;`l=R<8#chgv?sZtLyDWzcfJzbi-T_d} zXcF$b9E<4jpj{fXO4sZeN|6(L3?zvJSh&6BZjgv!?dS3M)2Dg$Y;k$wGxz}IUoWz( zdTwR@cv+b9`wSkcx%8{{AK#QCLFd4-VbEUsVH*IdR8>QG24t@BD{I-sP%Tgp0Sa|1 z`-oNPRHCn1DQoHP2tIp-l8h|Yg4`zqUR2J0P)ki!yb=@HSKvZ|G)H)xp`UE$Vo$Mg z{kKW>hE;1pF)3hsLFLQp{q>0(vbZ^F8Nm^#xBY{*%KW#Y(c6;l2Z5kcm6C3En#ts6 zwLux^Tp^fPj<*5CtazJ^w+&d(QL*~~a9K1uzyDXK{qLdlrw%ki?r#pu@v?m*@N(FE zn&TSDTJ~b$Siz-)g>Dz;(+(t;i*9dP`iXBQiKfTvDs%VKWN#lmbl5YS=S{ep%;$+| zgpN+Fh$QACAMd`;g&9#t>e@~ah*L5_g<&12)gHe5t1!ayo4neEYuE9`(OV3@H~LJb z@k~a>*;*^Tu4aOR-sBOFb6*{vcdN&=xM4T6kR%c0a=o~jBtz6-a=Y~b7%|uX;Ik!U zlhw8%lFxk-*!G{cpx>cr3kAa0iH-El>d0bIHcIXQQ^S&HcWkejcb@?BT4R_H6b=%C z8NDb=)vStUSiR8v7k4?Q``x|+PtxiogO15b%pUVRhF;QYUGuP-?IVJ-AsHsa*c39` zFwph})4snDR9O7yH0xY+QQYyiUKnCt?RRVYdu5oVcBVkaL{gLYV&~HH{g9%j+}u}N zTfeRj;wPCJ`1~fK-_~;o#%v6W?FDKuKUl<#!y_-1tzMtIgbuw1`;IQ6LtBsMa0MMl z)W;C9;oWSb5!hoOeP@5coP0>}i+2TSlBNPtnq)5FQO_jfq!|N4DFVzX1HOv^LGyyY z8MN)74b7R)>*u(>fKb~Uihw4kC{Q`Dw2-(7NE!uemX(w);1V>+Z5}|}m>725CIcP_ zs(gjB0K4cNKn2ArxQqZ}FaC z@swQ#&~hEAt{;g9-%E7x$=F<|j}3U5)&es|3!RKLq$}Gpl4B3EnsKA;UV^SD7$57` z>fffhj(Zd-k#mG&p*Ab_8JejWsLkoGwEWR3QCWs}yTQcHR1RxI+2HVHH88x8#2jHd z5Rj6kN8_BRQL__YF7NU@pv3rP#>J$Sv!Y8RZo%XPvx8>k4MDk~Z*1#7&7R976sTZP zU$c^Z8|Kbg(YN|2Q6F#*q@Hk|T4H|sr`h{`L&;1zeKI=69msvDCpc)s+2SCVmim`b z_#5_$um`0GhXi5~3b9K2QBRTK(}wb&@Q;0v&3F$b&1vB-Yro4JAL zw`Qf8bSBWn+@J3^?O^+DTp5}~IwpfE*5z?JN=}-RGaMp~CSP2krRDp$)_{H>BwQPh-_p+W{b^F5jb1ZH$+L z{yZDyB{{{`SyUXZ$Q%yp;FY0_mtGH8Sde+>UV0kZg>ja&>c0OdBv(sppefD@{6(5w z46$Wk1JdJGNgbJH09A&_ziJ^3>QTAU%b-U|lYlQ;=FjXSgpl(ZXBDy5kk=S?AnX5y zC)(KJ=gLS`=DxDn`gV1YrE=~41ZBVCv&38spr#Y}FNjH&J|LG6aUewIZ`hk#x`k!Z z?f{U>s#Cq?W^NypHEMID8~)R&-;$IptbRhh|F-}1D58?;>o%8MSUG#jodFIN@tfpa zk~I@VrKz1_P|p-^IHISkx8vmSV5&}ovZ_Lnh=AI;-$oj%kSkJxg9I=O(qc+1$az1T zjI&99eYPd?VDyq|yWPOOzT)bSH{4Z;x^!pURm!=pZGYRa6*PIl?cts%>19jxPtJpP zT`k`YA77Zt;66F-%n|9AjDMAZPPgq_6Y_n_GWtF?uXcP*LtxiZgIHBe&F(+!cR*w! z-GvyRmp;{Am|B4-OW6GT(TB&M4aeiU?lMD;na9=!pif!PPQJ-=KOM=F3oWe3!vUo2 z3*5&19gkCv7~dar%i=t+G3Zm0JDtouX#?+;iH9aZz--XwtiN7>C6rcL8F4C z!%KSLNAlE7|j*aApv;A17lJf0LJ$GjvEi>ehpJtNgNa#wbIRipg8O02Y(z&#)ry}GSn_5(-cvF%fDe&_9 z(?H>`yOD{RE31pP(S_va5*v>~vR8ytme6ZsQUu>Tr~$c6m?k>|hYW2i>dj@OHev6H0gSrpO4p_~x zG(}kuYQF-7XQW@Z{>)aSn1LVm`$II&-`|N8oU!OlFNUt9ctF3ncukfFXE(6@ABV`jf@@lcUls$VGaNA)=X|Ge|cZ`xgz3m5|a=w~=+~ z&6@T0U>a2`Lev7^>14E7qW1xBYw`ZguO}OLqU}lQc!*M232^s%cP=x~&PeZVvLot5 zf2y$|PY5a?fv}mCY@%|uVc2f*n;YG+6(p4G%6Muep~R0R8t(SoN~wMEk5cmTx7D0D zP+f;;nDzW@H26Lg2QGacA&rv|%YcW^#Z@rVdZC0}RUy5T%5}Nq>JS*_F=i%F=J( zR_i61$M|jK3m7mybEU7e9yZ=}HSb9YCtNgsd&zDFrtnzXv8X*fZUnakPUBLD;VthihX@dJyMn3krlB~=$k~(K9>;$M?+31*;rWPNY(8E#^bB~p5v%j{s zmPQ`0uXFJ056iut*-7`VR$F;96)2Io`_cL-eP>ijhi*!dj(7Cn!%JLse{OQg*0pv_w1qyMf(mNHr$kF_tQ*-{jjI2%=d3vu z5#y5w^^s>dn{r$|y=LU*`d)rLP4#EH$b$~l3}#dlfl)>Y37$0rp;Zv|OoV=OeTEqx z<_iEt|JN2xnl_9TdP`|C2CxFs%rs_8cyhX)a<<4}C!Wf*vi3Fe6cbZ#_xrB~U6wv^ z;}&0Bjd#hES6ATXrIj_zyE{3AoOGkAO|*aq2B<9+5XuBg2u3-zGe`&mp&ms>ure!w zActZaFj=}#7%C^fD_$Q^bGk!HgjJ$L=&_3Eqg^ty zkorBaFPedo#|zGYDyT(lIf6{b9AVbiymmV?nM@KO-R(T`n;5X-V+>d4QEwoY@!hr~ zW=ec;_C2X6vgM}L$iuP$yIZ-2ZtbJi_X=O7?Z;++KL2s2amSJ*`Z%Iq)oWDHV>)nK z6H1Y~9k|UCIpnO+Ef$uH7j%z!vro#yBMpzSz&PafJoKtq`q?yvj`LyNfJfD!Y|z8L z?d76D39Bnzu{1)6C)@CeWWFE<`FT9xv8o?YA$k@nnA=nxI1>r*Hc$ET2-k!DjYQJC~B_Mn{8C)+_ybtb%H?QtGwt>e`*Jm0TBtYbC8%E}1cnG#=dz8+_G?esOYJ}R;P zRL!nesF86vpP$%7J!0ED;h3V=uES41IBrX20+7<1T6yyZ4oEmo^2QwqlL+!^ zb~6z?F>HV+p{_gqTPS)TNhC?y&H{=jo+(2L_SiSr?)l{dXXk-}sqR1Bg68*rouA_; zsHw)pR4ABFm}v^Dr84)taWroH5FSKTv%APjhjtoLdFa?SlIQlC?|n40`*lKS%pI9w z(*oIn{gl0KP$U!%MHyT2RMG>2^z${i0Pso&fw&OJP9hHLR3h#aCDpg8oVQ%(jfG^W z;-#hzs+_sm$Ak|LXH~T6v+ZP_p3kzD7S}(`$C=9(=2n@iQV+{b0lS$r_nzwEgXIlhp;IRN-eIklQ}U# zAd%ax&}u`xLlg_tu_Z8q95J~Kd{`v~7)9Jc*d;{%ITI(($Zi`XK);;8V=EYXv}{wb z5Wb(9x;=XY$HTu$<>wxA*&!y0G#I-y`qli@Hf3I>zJEjWd_+ZyI*jkMoT12ak0&=z zMLj=r{&R$u03ie-fmeWy*=Js90NEs;1kq1ri^JZ9)!Rjn4~P51cfs`Gu6$WA!OUE= zUu~izCa=I?oIbis|1tU(YdnWCq ztaK(v-+EZfMR8f@PD+jrZD7zd>wpE1A6yHja8U|t}RU?q?Zn)fQ>o8xTECt!~tWQfT<#bnNo)x%fsOH+UWc!^oW z1E#wa4!Hf6>5Fo7tDU{lD@p1Kcwo!@z*bPo;JhAc|KZ~#vBB!qoaDBx?qa;AYe!aa z7{=Zo=hQU3v#Kz6sY<#EEu#keS(`E)pA80&iR-%AFaGD)jzwCmsLF5TY zk=7bhPui*zu&+FCTbbv>FjxHi@XP4;c+h$k+#j!_ZvKYqG_?1z$}<3jQe=YZSNYDD z;6*Wh&#rUWuF`i&Ybz;h<#DMOr%vs`=!KV5#|i!qv+#TX{DIN*AjUC)xgUs|c-RTn^V7SuNGI9E1f+tE*~TzsaHMT)W7}>=|x84Knk(zUO>GUl=>&agsOb z(FPNtUr9dbsl0Ut64|a}!b2La+>5XU)qG-pbHso(n?+9hh5B<7&X-MD7F6WnEr;uk zL`R#dKiEN7fSC^Vn&sc#nvQGbtM+@9H#Kf6ZFVc{err9sL*;Hey5=%=h<7J$hH{aw zAx!f0{cOI!Mj7EzASrs|Gf$6~+S%wbu0hV*3Pbo**uLXShzOj8oRI1S7!~coL8dB6 zTa~A&SQQ~fpH-f{A*bJ_Owdxb+;9HMT&sLM8fjsocT(`XRgQbZWJp_Voa0i^htg9AK z^QYX1XA#tJS_AH!M}N!lX6ve@j$cdp1!@DAc69~a;2%R~jC<`uY{V>dn$K@Op7`=z zoR}D1{CrxoR-38Nt#xHI6*5H{S<_K5S+t0qsnMa^F{=a-d~m*fG_dcGd;~G;M0^Nl zhTnMgk2{Ekq!4qGAv{9nmO1-3(&Zc{IohPmbLvr~K3s<4stW*R@Iu|KL}FE>>@Gu8 zy4x2b;;ycfH+l!Ao<=YW-0Xy=X5~8V9HA`K zB6+EJ%NZvTdKE9NY`+ouq5gQS9t5VrbP1AVY+36#?a(5Gb`CnAXFE^W6vpeOt0|jG z61z4rK=!8Sz&s@w9U{nZ*Brb{sV8^~kipDGJpBK(A>5@fPAn*=UEcp4!R9Ohp&#zv z>dZYg_4#O(MZ1bem;^eZ#xb!sS}TVgL7) zS=19YNaf$~YL^R*?o4FFCx=CkodoXwv*|gi{DJ>$vuYK8iY$gg(Y5LJDD&O5ovB?| za+21U$ODc8;>2bl$BfqZ-Ob@I3$hE=Lnl4<@Y-_#YHB*HdLjVDPV29=+qXN%;eG3hsDk z;{WHz557K({-#Z^EZjN@Hl$?5j(i0VSZaE`g)0Zv7S&ABby((N<&rL7-iN%@QJull zf|rt^3SuI}VaMCn8~J36$S3nl$Q5znt?9n|7(JiTHt5}(H%!C0c+0O>#%u1;s+hkX zUZh@=+>LBP4C5w${0V4v0s^@dQ{;46gS4&Z_+M4)rE%`jA6`;G2aB|6@$`ruGy~I@s22_C9`7EE7 zjk@FBiIeqrGns_$ke&T4z0X#PfkLF(l4}=03Lxp0OY5ca>3Y2PArDEBKL9Zo7?`EM zPo-Xa8HK`~4?DWKk?rMu7dO*%b5+#fXIb@p9sq^~;0qgZkxhqw zE>~2ryg89D=wIpWwm7;i3XEl>-gq3O7>}o-l^@J))%w#P<4G{XHZ}q_!6aFba4jI8 z2Ap9fxSKPkd)tzfSG@^LuQ$txw%3!??DjFx{CWI%HWYCj>7P==d^%{gt=)A4=~NmDL0zY98x7KxRsI;-~<-XNr_{IhjG`15}yqKlSMF3@0O zTF*#JnycgZWmaI+HZGafOz7u$n?S^!^{f^RJTRhw#uHH7Qn1>zpth4Vl{X_970#FRnwDp33eE4Z09+02Vu~7F> z?5V+)kcjkhhMEynSSTj+bMINeI8FQWjCMv0I0gVxpTDEa6n5Hwxv>UFVo{ z2THlAhvRSzf$=Mwm4nN5k00{t*AyNGwuur0DTq%r%bKm(aDoKczy8fRf3V7^o_%nk zH@VuzUZg$(U0!mAi*MLgR!>k!uYoATArDo$83ohld$PB3BjhCSYBw~BKw_%r5p>8!CXLB(Y zHt+<~Wk4hHuF)a4DU>7jjzTsA1zV{Pm(>TbtO^2%)9}`&2T-726gN#QKl*Za=uv6k zf#6W9CJ~NEkG3)qotMpE)-P?WEjUFF4PMP1|H40GuIF#s{~Z<5G3Hfy^fF*slK?V& zKNDeB{-;GM42!p$Wl@gT-O5LPr%X6Rj4kV>tD@9DL?AppWKZIB9VTIqRKrPlbrI%; zH{-e%WRHP-TTtg$oTQDalfd6FtH0=#PMNbfpAd>|ANIF25kR5S^O7v^6`5ej5C)=1Opmz$P98-usc3QqbR0mG3kMci>dL zDvKM9^R$mXNrOs`mG(q^9-yBx+{j4aUwz3G@tDW%f<0mh`R+XtS>5p*F7K%oL`CoP z&XrPEQu7yW5g5hhy-S2RM0E{Ar-{!?SliWJ|FyMrd(WOd$+H_TgG_<*g$p6)QCKXO zJu>jv9g#&W(1ZLdVu>9W3D$D>VG|+#CkLV}onV(@mZ(zXgGyx)s44G*T2TqBMLqdm z_dFs1c3y)qj2HF0MBq)Mbe$z~D%ifI_UQ$gt^;aGonnfOJ(!h6@(?4=s{LjNr zZKAo)rZrx&%5ix7k=$1MPKnUUdh9T~_l4BJmj0lSVzG8As&+#KxC;2}n_mn*Zab5? zu>!}w_Lc8;gdN!bSTFpXzHYe?T7bn?>my8=7FF)YN>+fP4H0W>Aho?j=ZOlZ4{ak% zB{kg__24X?p*CWt6*=u9^q(V`z7I1maA{NluaL2gkDiHM*~~;V(qzrsCYvH)TWr^1 zJWwRdLy{D(Yqid)2@&TCPZfROBYz^>Bdhuf3s9 z9|Eo@;A-l@vAPEGb}LbpQ4q2b^_1G%Z{x;k1j=lbah#<(^-B%Nmp8ciMY=)0=}m32 zSGU3fe-3+Bvrs1Ve2)?l;tO6P5Ac_|)XEJ$FQvf`6Vg;Hu6Y5p0F;xG-2Rp5Ct!_Tt|UXyqnf#EC4QP&pOXjH}Ck zBkh9Wr5j|#BvF}`Pu`)pQ8%h~^8?%vFwZ07Mv5CxD0!9@kDj@V)$d}}5nSX~Qnbq` z`6`ywf~6L$dgKT?L>qs1D)DyFx}b?eR4TvwTe9inVjAYfi~UW!i=;4=g?A{!ijt40 zlfd7!L|U$?l%}%*libCN1ZNhUv^rbwUdr{2I$6= zz)LazV_sHiKu#IFxmcC*6}Q|Jj;-IN$D{dUGyS=q*@xC%-*}L7ZQ@A0Hv3X5-oT*X zeq$2D<%YU^8s^W-HA&ZxBB(-)h{F(s9s~F#oaN_{G$%urT~1$31C`n2>1&UO#iLAr zsb-!>C}{FI8$?BtWCuTlh%l8!Ow&)7%ZYZf;@bZ2{m0jX`FLGsI|Nc8b0j1_r9@C{czkl)1Gh|K>VyENihnPz71?@9W)OT8(Rdr;0O#gup{baH z`BsCdTMd7*7w5-dAYe3agB<(C5!6!hvW>X{H!cba6x7oLWgxFX%53UC!X67ze`NG8 zZ=$)V-Df=Kd-vbGY}}S8ZUp9F9oX^nfJ{w1fawxhM6Y6wz+H<3$1j$pd{rc6aaB#c z(3i$2YrEUXmZ+2OV3bMSH8vrWHx%7YGDE#E{Z?M`WVISn+Wa38cF`Z%?8+g{)eBy= zG;ziTrv#ObBdFkswIYI;#KCNM!lD7I$(3|1*;{dQ{WLpGxI7|#p==UNM1sDOcf@G5yL);`d@r^X$aYxRD_Fo?N<~F z*uO)D`Tr|Y2I@Z5PO^FrIpUb51MIoq2)T%OG(JQt!tro>`{l%0`5WvV8@|;}dNsvm z#>7)FG)m~bdiMLU(2uFU{9LzRc@#U$WDjeAePUPU^beCi9aT#Y^soEl?dnQ zIXx>5yXZ2VY7*{R_44C=U`z$?5F@#zf0IRT%1)O!3&>oyP10U>YMSu&b(90&s+cmD zyh|fE#eJTEms1#FV<2%=ajq(hC;njKsv&q1rAEZ}3&@$}eyAM&44yZR+GM|B1wbHZ zGCFO$TUal?Fn8!KQQl|*D3X-1`;To)?f72cLe)Q$8K!2^Y9C4li1I`9#SID1ydxdx|W8Qy;?STXi2wVb&*W(Rc=RxDj!#ts;RngNR z-d7(HudKCEG#H5f1RT~VIR3Wcw9ZM3$`X1t$%Z zO(wlS7tp`fKgkRP1jS#3keXO`Moo~jcrjzr5uXpt2loi{N3k+Nm31==uQ_n@j%$(pf)0g$%r$UgV*#_s<2DZSw3@zEeDQ`>kh0eiL~RCTkh<6kp~~ zK;UqY&#)29f=Qq4-U(uXk&{yIwU9|e8uIN7p?J>vs*~Z6kNKp8u~UjwqB3hi1rFav z+Q7W(E&!DRbuDFsbpQ*rb?mtdnIGlTdRmq8oGJNER1;5+cjGvrZ1zWe$8sL_M4=}# zOZPqF4BE5oI(PtLBK!MUrG}egyi1F&Y6cHW+2qgIOQ}XbpK+aMaIJ{+gveD=;d{pn zOkSi4%ZA!(+Fso6*nLJ&#l_IuHzRu&ECd~qc|KALO&<#uuTj>zeFx{R%_-V1zT3Eq zSMn22HEx!6q;dzV&p6K-`can& z&@{~DY3o7j(A#{c61)P*keM^ASf^wKjfS9AYzTa=>H zbmRmik3^uElj;?->PU10%@X8$E+xTW0k5drmHXp;IA(WruDJi zb{zYAmfg~QJYXC}=m7CE`Pjvbz=A1&vowv?7Q!S+T%nKH_btwWxQylpM#t|mlG7?u zR!TGKVyD(AtAN_zkjqEgwTI-qyK46KH;qh2|25#}v4njAGSGCr?9U3qM8!gCC$pY7 zEcK*%j6O7~T7Rcmb^fmCLpv_#MLh1>ng-!dnnn~zG3&?^sB(6OYrl#y67c=;d>zXfi8Rqv( zo;46e$99k{z!N#{={iPVHsFw?$wDCQtv(;}Ou&DePZ;x3F$hQ0<)#O*&}4O*cK-<= zbfY{a96B~Gg+0=Y#}Rvu(JV!qjl?ZlfB(EHl6ACp+ceid&bKhBVK8XdNr-6uOm9IA z*iA4dQZ>cs16{3|)~^$R3bG7%G|T0;=)h`ee|Uwi+goUZ1+VB(JV1&@igP!52w@90 zM#>t9nZflwF=!2=mp@=`XW#loWjzYipZr7XZ0y~nKA^!J!QCE7&OZn6fg}$Qu^RJw zIeb6=)!6J^JwF_hcy}8>rDM(a>Prm-5auPKvj0BY6N*Dl$<--~jcEWPm6Mm>vz0C) zJU>KNC3teVj1GU^NXhi3Y!|CRS33TjBP?ovqvDWBQ&q<;hy5$RIn+o|n#kX-M)Tg! zd*uuW))EpKq5@*==EU=vkRn-`khqYMfa!dkg2)F_eZXawA$IeUPm*RER?GDW%1V@+ z|4x1=kem78_^WsaXX!-s?^ne2f1aTMc{l$+HY}3Hefya=wG`Y}a@CcZyPaLRDX*U; z#@G^!a#|B-?AvUpUO+@~J=4N(ZTMSAyFX&tD5QB0iSi~l(e@t!2w&ruC~$=iSF7yP zNg{OA4dw-1X%Kh87DYSV?g&(`!r)U{+e#K1n&B7p_>4yy>I8{l?-fJ0eTqwI^#T}` zHkk?r&@4?heNbI}sVq(WQFFOWd4=@1(x11epDJfJVu7%Wc?tQ&y0#ciaCLDQ0!(z= zKGntxlWZYZjT8V;BBV>VC4#ih1db0GR+yH{OQ*Q|T>84)0;kDS<IuL4c(ehpw-MbGb+3bE zmv~lj>GRrJGhWe0@sr02)~0hXEXYAIsU<-P$jm@ql7Rbw=uVu2DNd)q;rx!u+rNu< z8F;bhTXv<%2(x)?&p}B6xmxEXSr=*>Sjtj>@%6_< zDChsaT8?q686c_joC)U#qd(^>D&THH!G#RtcAYufb*S!7F0YYBsK~CZlGuM{uqa{A z(;5kW<8dH;)Sk5X4w^F3`KcS`ul>>8ovb3Ig`?_~F0asX{Ht(Xkg5oFGP_CL6#bA4 zrn%ujDv6PqmYpZao1ULr3FCWx9yq1k5pUSAZPq#&9RljGkU|IjP)>E@d!0)9#MZ^# zLM)(uIh{Ie;E5WOiR%N!x{OgDWOkE-kI!|HY_XgyBaYN4r|c?5!ug&Rj=5J82eq0` zg)2*KEeK0Ji8>I>VX-Sh!#o`>zT&ogtrEB@M)&gg$lE9D0~~}pW)+tq0%HlP2pd%= zVLH9$9>07&ca)(pSAb@J*kRvjF+q#9;|*q;A#*@#daGWQ6vv5^7oNM1IJn)U{ou4< z4iQ7aY$&!AVJ{pwc!+L_zBfqCPqQO#nw$W-dFx0?XNz!jpJbD_%tdHj2?KyS*S5$@ zTBRHt-g;gikrojz;iK_6ul``3C4$@Hw3sr6n11;F`_gC6++962AxY!5dY80eb?zju zH7kjvB&Cfi*LoP{-F~35;a${6NElO%B50OShRjMmL)tdG1A=cglD+v&q8x;0oMj&H~%8WA=q#cFcGfaT>|3KD)oI8C02h(4=LbcZBzdFJc z$n^RNP{!^*9C00Fq`0}mNRe(XT$_7nR6Wx8CweK>(V{=bp5&|2wME=}ChBRR`9U$C zHWL&ID$naG-yN$D9)(U1>PFewFRva}8oUf_>KxAO3z*EQy-s*q|Bqqn=wIuJqykF5 zhMbSB`{7pFmv@O9oG0T^6=7#DJ1zm^u3LA(M_H6O*rUszy1^#PI{Cqs&)IIFij1Zn0Pcm$Xbjuq1djq+F<3dp=JZYU%QEf zvhBWkKA7J%6&~6ST!nnUOA-`Gl2VND7wL>%_r$c>W1<@4n+({vSG&jhB%h!tF1v@b zh(3`r5r*b)sI(TozYM{g#?bRp7-WfBVt)5SYCb{NZ2{&j{0eli-G#s=j2E~Y=$gJ+ zZUAeHPIy!WC`iHB{!g1Hi`>VjLyU>9=B#<8>b{%+z$Xf7bS=cd^-o8XWKdMet-ais zmbG&piLTDmQ;{vWUzyo1Tg*Y7qg_IVO03_0)iLL{2pxrC&_}x##P7t)X;HOXCRw>_ z8S~7YZr=CdB407Cm?SPt0#ukR5-u!@aCO zZWx#IZPG2qBObet6R@` z#$;n8hb7QGLs?!R@_-Psw}{}ZU6P26f8Afw>Up8WROQQQO>AqYuw9UVgv2?Rw#SfA z@kz`YZH(|!a;pS9doB1v|Bep0QV^J|#}_3xoplLV^ZKxhJ}V(>A;|5n9p4S4O5N%O zj7N{2bUt3Lof|MS#e#qB5%exnSLlz(lNf^KatxkuQ z(!aYW7TXJa=JuD0n#pMW^sd1H{)6tcfH^&0Km3lC&(clP_5f#Bk&PS=7S|i|iY_D4 z%Xc7B!4br8urEoF2Pvl|23MV8NzWp+{G&Ehasd>sA3}g7&;6ju;02C^s001c*sMk+ zmbfbqzxfCGl<1QQ{N4*=`uyLX@kGp&cuV$7RI*NI9E!*@#Sv6exX6)zbksEg?rUbv zA1M|*UsOO%5;@G+WQw1Y{msHFb!zv;Z%sVZvbL)Kax##uAzJ4(CE5FjTQrgQk#)Nj z6d6753O?is&D?3o7Pg$==xs9R^^*}z0M(qBl?JLHkl$qym3psULPimR_!hn?$Q>dw zP8dk4a|ggA3sRS^MGtN0wL@juPU6Tivsk_&D=!}cwzQj$M8K)-Bjw}+`g@ECvpa|= zK&tG8o-rD<__x|u$z%gXJgR|h!(tSeh&Us~;T!Ap9 zS*S?ykfM6TKYkT{C6p|juh@rU!4SaN`b!BIm9CcODpveIvkL`d*J?~#e*M@4lL{Ug zzCY^o*BNVZtF)iaA>Ft?G7Ae{Pz>dA^Qu&QDO6lz0iakyxDaW${LOihyRGc1ESwv> z*MqCkWs(I7MaXvux7PM`7So{-PN4i_Z4FU@ zsrc*1E-3Q+-`5i9-OiYE?-xoB%#H(I_$83cx#e`^2eUhaLg#oAqCW9dWi@t-GHUp6 zd7LxQ|It;UX)@y#VxZ|Ud@ftVZF}ejfSYzPzKmbr?H;vDoat7_iqa^2H;1#h zbn56t7uQ?y-hbVX?Yz2!rhR)4&aMAE(}ycJ%^?qt9jk09ChlWON9SoVB3bepA8boP ztk#BLS?f7@92L>hwTqYwd^(65k^pHc5QISlCAlPf9^W7v?yD;C=X-V;MN?F!;QlatPksuXBC}RdM3x_dWxi3Bdhutx z`NQMPHl^Jj{~Xu5*&hykJ1r4@23@5;Z!&N^Lg;>HYeVz-fQ2SaSRj5#hpDRQX)mWg zHl^Hv$L}tI=VvCl{w}nVJt0))X^{gSX1M@xiR}1f0MIK`g98oK0wV%W)MS%|3Eyn(DN`H)2Baf;pPRAgj^l zyR!o6bm*7hv9DLYlRW15#X+-3>+eG{|_Qu*E*`5?2F(Q(yfY6pS_+okG38DOV>9+xql*xhL~b|hF6A~)!_ufzo> zgCY&JKC}9O{hySun+o(R4?SKfPVXcs90D{zG?gF_OZ%Kn8b#I9<1B%=jvAX6BiRSm zjQ0yZWS9|5R)T_PfEuvmKRUa+ts4MM2*7bD+kMoKQ(weu%MqGGJ{ld!%;RzS*+5E{J=7#=Ir>%Jq#WA!~;0>>w~LqsU?i@w>RcToa!u zsQCx6#STj9{i^2D1_@CheTpE+tAV9a%W~A&Fb*;JmC8$p^Z zyKbIM#x-~<{34*3xwwK>J8ze&Z!1(uaWCUtYf$q;iQC8aP)m-q zFX%=^^wM!t^qOO3DzHu}`R>dGA8M&A@n3g_q|3y;?VAqT7P)|<)?jxxn1jU@4uAa< zluPOH``6~3pj|c~sG=G8d7NawR|Q2?viPm_r22X$nqk3(uhpdtma@d-_dMiYzB_X0 zF&j;d1h)l2yF>x8+M1*S+VsDhzw8=O+6# z=SoLyu8n8mnfn)$B*BICQYuyM)0?kStY5r;g!a0FW&l~jgk%l`^=XRsJ&2nQ6f5na zLC84e|C&>((#97j&h+>ci}6w(5e60%n*Qh%P3#~`QDxEPAJtRG@7_&BT$+-Z zORAFG_!ds9&*;KveRa`ye!vqP%1LV!d_qP;(<&IwEm=>2z*G@N+{Q+HGoAK0LXSE61 z5l;eoZNdgFG88{y5FMzPfC?pGxS8I6SI_(5HVg}{9*Jz?HZS9uU&d1GXQp5%(mCEB zW6n*viCE;nPF1-QstR@6n<~;Wb9d^C&~dve(0}6bG?r7R=evA+f?4X2$UOysFQaTJ zdTm9Vo#kup=vcq*pFk|ghHTOx?F>t-$EDa1*?EW9pzleq9$c+9uLLUkK(VXi;IJ|i zi08w@)W{@iz8TK&irEe+#@LEaZXDun#7xubmh(g??LH&xjWK9@bHynXR-~C$k--k^ z$v4;_fy&I#bgbCHh>=E6^?P<~zs4wska+M)eVeO@=diHLLsW%xkM%9Z`II}2XtuAF zl13V}o`a(FQ5UmoX1-lUK;SZ-SUU+RU=m$O@Hk>p-Qahr+dx2(8F`xlJg8yd>4PSU zlU`TLjZHc-NrRZ#Woo`jt~3%1Nld?^7qXfeI3!nZo3j(tvrH4_)PX{*4m=>KV3zo zYTP6is&rZyLQ)B3KBI>zbYmuSHRTJ%*h- zf$Dd5qfM;?3X%U~$|n1p&z#KgP;4>Dae5iwNqHVO)PfJbPR`7U7#m!X+s_1Sf2FF- z88An5?|4+Nl^<1#<>myvliU2bMVF)o^CP(%MfW2X7%tG%v%9Poya_;g5~ib#Ca6`& z&}1^piIG3lLgM5Ln^SPFIBQ#%i>d-wgUlWxppmOGB^%yq>Y)4;S;GLt!ISApKd)Wi zTRGo{ntwu)H6N~RWh2y`RaDAt$?zzP`MN{krJsWA&nfdJD_$|uklUmtqTOf2Ai*QX zT!Z>Usv+6*j&cwfGEshJBG+)ajyWp>{p`MI{(Vs6W@q-D3YiPfkLbeklw8JS0?JVI zcLyqi*KHdje!}}UgS8;9SumL>+ui!+opoYW3PtF}{Yz_* zjh>pjkq441bLqh1&W0i-VdXUB5iCW#=Zqi#HWuA>Yu~MSZM6QH^xoy;?dp+_^scZw znp=mS1>We9hxkN+DMsQ}JGzdcD6YZIvmkN>zX1uN& z(ERVtG1ixm*Xousx+lI45<+$s4SKD z4PtuYhv2p$%xl%RFB-VE{`Qw@aW34orLxCkH+sC8w-YsfXvo`k{x3xpu(9ma6x3a%-tB{8_k* zPp%zp_l2C{`aAGPlUKdhy?aCaftZefh>X*36)e|b-TS$4d)*#@TXIVU`_)c=^Q*gv z|1@rCau%c&N-gKGh+IKZm9W>0pqz@0BP?yvak}?lM80ps-PW!H2DIqkXMZuUaP(bK zY~ehcRj&ft4vWYFiodmp!6flNy7z3Q>OR?rQPdH=!?0SF@S`WiVWMi^5L``>1 zE81sCQza(r$sMoF=u)CuR#=x@T&K-|N%q=?KS&7BTo|}OZv{20w<|jaX}@x z%@lhLc2_gzu%0*h=Jz(X)p`(fXQaw>-+-hw@3sPCC)LF_uY zy~m*A0vr_sO(I&(b}Ve7FA|bK-3DwjGT%+1B88i2rMAxHN7mDvQ(4b$ni#{j`X~DA zIJ>{2;{fZMpwB2kFxnDNhg5lJ{U57L$@s8Qy_cPSXeA$9yJ9~BlCE4;IBkAUT~ZI{ zN&qPsiz57i(QXu_CJz+F2U(Gl5PllZQBK$mi)&F9r=z{jDo?l2xDQ*#_^W;9eeYQS zr|oiLj`2t*&!<+=m6@6}-~ErB--$H>@% zH^&9E=*8vKU){?KSUX>B{!M26yKCva@an_h*8S_D!s0j}$!xNMxh!uhZF_^K_oUt1 z%hU*fnhs4 z)l8cf5Ds=vsoJz;(ng4;`hL8TPuTwrZ90)sr6;I@uPR{CDjH<49y%X79>0*S?_ULG z5xc8E;Tp|LDfGLYj_z`jgbTKN^GQ&-00>_fn)$nzd2w@qjd|XDS)Z%PU*G)x9(esG z13v|IN~jVZUL){oFY6Kg(YeRlP>sXJPg7>Mm+5{UEoGrn&t#W}-u3apq2tlt1N1H=O@;2+VBIj51izNeq= zuL{j#j-SRa*=F!-`o%s+mIO)MQlq+t#LXp@qsC{KY66WknDI3rLeH1zHSTvw=VEjE z+JLh0_UqqmaEtZ*0cFBL?%!45+WG?ggy6<;jAqq*^kZMO1l*qvS+)0Js(_;J_Ps$e zJ+x!6_Ziubk#mojxYJFt3gpk>A2zj0XYwo~~~Ys+X+TDW>x!}kbY#FShdn^o!}q7SutdQRAR zjJGb^TR-a@PG-AzsifC^sGZ>7Z$6e~ElYL8I*rTK(c2p`Qv8EpSwE)r3eOI`NFsKi ziol=~%(R-`LD<76KD~VK13Er0p#gB6#^a2jGv&h(Ev-q*k(;s2f2{mkc%eCD-V6C- z2<{*gbGkoG9&;P{j6ZL{xVoZFCCi8al;33Q5rF8+SZi>CxJ%B7Zx-Mmr{72E4*l%5 zO)yoeehfwz$oQB4be-C;_CEt+BhqR$bMK2%x;A9?OEny2@K&RG1wn?i2$-(y+UGWe zA~Ccc3O*RTiWffutSk2XBmV7k>JslZXSAgBF0y+DRuHU)xIH;)`>!igv)hPdNI)}p z4f^jLLM^TzbM^xfNZqCt7g@4-Rgw=H z+4tcNwHM<=6=0XXdDz6?TmLePBs@!PEu{ZU@ZZw4YaGmsbu*Iyp#l``m?bH6aCjY) z3$i?1vu6Qk+v+v_&e%yUM~UYau3cdSKLI@d)MgJy*_JPP4Qn@|njs&39NB5?Ihku@ z!f*iYcH38Z|D~%Fn;RBy-6}>*e~wtjGId*b9vdj1Ngn+)vUd3W@UEfy zy{8=Tfu+!UiZym-!)sdn!2XMeN@{iE{eTigjPU^>v7Q8DUT(NIUhp~yS9M(_)64MODk3otO-IdJs_^a|kxo);{?ywL^gZ+|L|2fJYewdJII`vonCQc`b|Kp5RIcHlkr~(Nogw!#7r1wdw?T>IJL7O5bUs-+2n{L2DDg(EMv~TUmj8@ z<%}q2>$b%t>v7$qf@tVxGL74rm~u#5SNVB zf3)7OK`&ORp8+EsnQ-7u7}h{S=)FE2LGSr-Z;EVNs{4H&H1?th(S z-hnE?M<>nECtWvWwI7wG83mPWkhAuuaafMQ=MZZ7#Q~7JDZ^tc3atqP1xb>%VJ`uS zSA!92+*}oF4x)OibZ`>hrauIab$h-@bslzA@vfrGaN>mtZ(Wq8qNx~FI5~)#VlhXL zpN887A|)N%{B#YI#-b@kKRbp|EVfgD&sDi8v# zVFoMWyZ&F4IbG5}M!AZhpQV5&tlOL9v6sb=ns`NF+*~E;z69liCA#=0KaMf|mdHzM zjgyd(r`Vo}nupaHKOEw7yK7 z4=a)E?X^61BK+09!Jru86r8hpvKE=h$Uoo&v2wz7;<@_>NGt{NLhn;hp;Qq;meO#c z|5*>h9+GWbI_9$I-(q_(OaU6Q2Ob2XZ*`s6Otk0T+2T7lb?rZ+wiTJGL(%z;YOp-? z9?<=&+!)P$T358?G_#E|n8+wrYk^IxzzTCkG~zHd4S>fqMqUzty~)3vUh$v7O*gJ| zwi7kQ57Fl!D3)+_y21h|F6qQmp%Pj{`-sl|{LW|SFhFL1lHRs@zbO|X4yVRibQUNP^RykzK`s<~ZW*(< zR&4+Vy-qLsDP`(!TGn1)U@|R-$R-I2!WkF(UebSd%F}o`ERd{l%d>~NZlq3z8Vwa5 zHjUe~HOZ5RMDpiw4jCbs0g5jqjq`sCVRzmw?%%yCZ|7vJn&^jTX(K4g`Ms^P`1-IJ z%Xp^Do4QrvyW(Ie`3p3)4jmb+U@6tKRIU8W%tJ*+#2WLz;2HAIQGfDgxV_r0{1$0V zsy8|F9xs6L+68;*Llc7z(fb`Nm|r&W?6WTW6sU-FmqhTNIGMfI(z4>GkM}@ogACyR z%gfoiKb;z)k9vqG{4pCN+BIf5{`+zt*duQQk?^e9O}*pA>5;J(TWrm>h8m6uFED5o zD$-D^LYVMbxcGxBQ(zH|PBsZp@eGWBbA&xq`SL0K*)x>s^Yh4C@jdj&uhvkQAIN+o zQ)ZQ$wuz1sGcVJ+oo4qP?s>9LYxm=^AUyoLO0yz%H+Man;&rT7`xN@wWS(q`+dZBE z*aq^O4Pkea#mwe3s_k!~Ty@ljr~_f|$`6TKI@l(Q>o&^Fdx-NN%WZW+*zHAw5b2CO z-Zz(*Fds2oY}^F2J*?+K?e<*BcDs2eU+uS#X3(G6ReX;jVQgw4$XLUAW_PN4+jiQT zQw7sQ*^eR9PZqFWD(?Wl42|&g2m5qGm1W>!bIE-(pNRT{Aq6kwo-u6z8{j=Ho1F zE^xK5f@upXLj8U*j`b>JHa5eFpSe%Rr|uSyt>56&Dq09)j~Cn>R2z&WA89ElM7=}Z zqw1A>nD>V8_5Qm;(iw8*t~O;^H(X6W;uiW8_CW=HM$zu{*8%iepD2c6*-UNX#X+C3 z=;-NkLcKpop4t)T8R9T#P>}48xw0N#V*v4MvF;Bi^gf=GwO=-*6qqB!2n9c34t1+HMoH1>1tzWv(GB>kI*nzA>4(^gAvrk1X0rkj2OB@VqmUVkj9ak`; zZzziorWbPELIcD4B)9R(*H~rV#$nymQ6N|+$l;9!4j*bvSS8=h!HEWQi@(OHJolr}ImFLsa!2 z8+bm(FCo@s@FA&VAf8sf9)V+z#IaCW4QC+gZNzs-G7P#2582BPdGIUW&D1k~d(2mT zqCUb6e`6sZ_F8snQR#2cUZRs(UQnLx`m=6Zhh;KJ&if_5S1cNmt!#*bie_x{URI!3 z^5fX92Yq?>+j4hFW>78@i`~H_DCDSt80@XV1XbIpK)LMi=fLV5pLIxj6Fox=jyKKq ze^;+l()e~ltV)Ymk~2AAp3KH6JSHY)19vU_(R5nQujf^7e%2W{Y$Kg|N0y$pBR!*) z&AWB2DBKuHlpW?2ZxrH0gH(1f^?^rF6G!5HmCRGYwc{Q4>qQ9(3y4IBI-eL-gM{^r zFX%}Ef#=;jubTtvJ+iQ3w~Z@s@bW`O9D1)oq7%1Xy2YHQ=(b-E@;jQXwG_n*W`X%l zD5gxGbP9}K@xuI!12%2Js^t{_^$f$DuBQWCkN4?42B}xIJK$N7=i4YNRN}!q)_Y(M zUI1z8e{CN0!_mG}@K$>|<>^+Lzic!u&#U*WXpj^>6Y~u4z_{_(U_#AI{M3yU&WN=m z9#~9mhDQYG&oXI*?>D+^sy5VhxFHTG3`!B^IvDFZipp<55g(*K%7||-V=dpFkP7%P zzSd{Q^T@yYH|v!n$8&!U9*M!TfBl5RgTo4v7$pV6XH_U-j9v9e?sxa=>!***z#?}$ zvlabJIgt2nV~IbMD{67Bo!-~fCI`0=n%pJsraPFy6k-RlMVrRr|4&9X%kss=d!#;D zUc&7>+wh3aWzivvQkrsu%$a8()NB*56Lvjq5}K@ELqEYx7s67lD|Y(U!G4A?;No_n z!oCEgi+TD(1(h&<9vxqXS)cwEOGn3H#ahbuh=B4BV@P<$-+T2TEI1FnSsJx8;qDvW zgLMmb=<0Q7O*so5ZxepZ(#{-r^Ao0AUcp@o?uz*kN4)69&F^f!ml~KXwJXEtOxj9_ zrpA|N{IVjlSu6z+D2TY{r*fFFTY~%C$ArBGG&8Q^Ia1*at$G+pZF1@5`zu0=`@B*t zEd*JMYRXqA1uEiu`0}MwOErI)C5khXKr9MJ-W|{2_KB(@V+3>&kdB?SX$G6sG>`x1 zQf++!{a1)<{&$so#9#z$3X(5=gDWS|dWb@s86z9qcth1birYYV>XR}B7aPc`3oWBK zHADqvxZRQD6Ge0>!gJU+{+HG{1qpfCg9yAP7yh7PNcrV2PBnZMk z28XBbyRQvxTQJ{-7xsPD=!iJ+hz9rebpUZd_{{Zujf0SI{Gy*8==rEZMw&o^h>E#I z06Y8C!ZTwXr(-OqgoN4&4s7URuwpzEHsF|2st+tVI zOw{A2;tTCU%5_kwN~Re#H-!<|U6}#z6vtdY1$@53ZvknI*uDW%&%2F(V8r@*DRGx> z4L=U$xxk1hjcD&F`C8|aeu>aJjPim_uYP@*apz!mYR(P|&jt9;IhWC% zGAQcn9t1!weqsi735lpxfR$^%$B=!1=h5^3uU>H`R{(MAQ#PBO;X>nmMi>Pdp$(oT zPn{zaySW_m`h!?N>_Yf(%#6L5CJN74wS{O-vaQ`rO`KU(H4HMJrj!z1gB7xw>o7v6k?g=br;>RBy04-jT zPAW9$K3~ON?NGtCA>Uf=dl{@tvNSqxx(Jej2*?UlUG`|v4*leZ6}hZBUHi8G8YM*{ zlU}g95Ds$Yz+T5IPcJh#RJ>I|o1ChBQ`}C3q@5rq@3p|d@YBND#IX^5l-laV&8r+O zkd%H1n@HNa5#*;J#s3nqRIXL=@EFYL`k0DEk;2UYb4{$Ta0Q?!vJ84P7GijZ3=3D4 zKcTfFz%;Bh;F?kX=si3~Cz`SZ>P7eGvzfb}N7uo%b)6`*0Acf+#GG)say;$+T(Pze zAKk}1byx>W{6z7;64!8Ofw(_^*LW0vGjz#lZ(?TJC>fiu9J9mACz8)F??@s8 zP+*31A&jEJdH(toXZ2sh2QCU4SaMqTZ?cHQgz&4IB|KP62Zd)oU;)x?7(X_p%$Vu- z-?Jpe^ROYEZxyC&jCo6o?Z%(n4Dfv1G`^(OPX&TCQo9SC(YvrD34T8h&fT&LWM-Es z`{IS(mO9JrGJakmco$mSp_KIFb?|wyWQDrBPYcvCm`v6y7%CLwcjOfSZN1nyZA2&7 zjPYfE*CKfEOt^^p!Ey`1eyCp%r7!|Qw}oFeWR|kWH;h;&uaBbcCpWv=i56XdbJka zV$s%;;l!w|x|x^v8^`$%0ow+YF^TrZ^~GQMk8%~iW_>2SH$<(FB6yFlzNrpj5;?<0 z_8qIsonfz8D(T*lqkR&0-}-}9y_H4kkEq$8PD4`*YtHaMh)!h_R1Q%}ctd%_4Gwvb zS*naH@YZZ+YYtkI(I!5=ZL?=>#p=ZqE}AQn4fiv0=rrXRy%H!K&ob|Z1pqu6@=lCw zXogz9mvPrnEB|e)42R*QWWYNdu^@7Cq3@UbH(T5^fhy!W3CrkJA&6 zMyqN_!T0bvJVw{G0>nkq86g2m>3wMy!VCr)D|Anue zKFQZ2pa`S?{u3gHf~+V9xDd+@B;|C9ttR&AkrTI(E1VCBfUcGl%6TuYG z`AoO(tlpx=p-nnFR$JY(1E*&`>Kj^>w1C%Vs6fmYgDUk_!V0@}7c$a>W($Ou3KCYX z74>@R3S=64%F0U)&I(>0JF*P$sXuqt2Uce%lu~!6D|JnUH0_dE>m$3g*wvk*BU`Gs z8D|(_w*LZBsp@Ga_&Nx_2gbOX)ojz|YMj2Sy`Qg+D*38=%*MQI%ddD=Iyay;z6Oxb zG8l@$_hAz=$n9u)@=@~bYf6``RBbno(xu^vGFcwBx7LYOjeL7UEQ?_%19V>uQ4)$K zGcrWWXlK-xM5vaJIBr|8KyW(FI;G$>pKmNS)>R7%ndumxC-iNR!M=zNMmV6u0fzLN zxmx)@FF+Uj%Wrvc2$8>MT=e_yK3y*u?J4t)|EWvmQfY9jYQ%>+|2d~TE%07}W?2|f zH&yM0&&stF5(wW>i;5h`V1sYd{rrF>e{oWj%yWBg_^x%+vb5elNwOQCF?VLLm%h0x$Z?K`bTm5i{sVs01+=`cX0wbOO%x zXpf3H*`>3+N~r{G*R>N~qS8Ji-A$j9aqW>rauA-LD1ZCEI6D4c9KFmA-&5zY^}X=w zroY1#kATY5pEGd!A3fG=g zT)E^JM?K{);j@x`wk7dqw5P)f1ED$~o5&=b(bdsj8Zo+0_c&97*u*+n=@c(5VI#QP zxbv52XYpP`G*l9wT2OP&v~ws)ufnLH^Cgw5W$T}bEm3Pqyv4qk)T^!_uN{byM3U$& z@UhceRx2uptl!)vY*tb;DfTCDId^TU17b8>lE=7y_4Az#%=m`)dTENh;L&=CR!Ary-f6W$B|)nH5_VfuHH*9@pkhMov{nY#nKsNa!M%OKk*x z1tq&9&f`OhrLRsmLiD1Tek~OXQGvUyaCLhX=-V?I$2}YA2jg7d!@>_fMi2pJkQgsY z;e?Zu3FGRpsM`S%oM=>n`D!@2MVTg&wzn(CcS`Zwo{>6+E9p3?{W`ITYtPF@-nwN$ zZLbUIh&~lkD*dsbS_kLMv*5G8fJlP~J&a>OJ{@zgGO5-dh%C!C%na$!_AiEGO3rOr z{ty}6n|&iC)^X)VZlSugZNqA&6E8ly+3VdGlf{U5itxsrO&foF`?=G5>!LFG zuRR~JE~qMgp}jWAZgNuEPa+WaCzf{EO~J1 zyJld>_1kpwvURX^l68;HF2+>)PT@4fTTAEnpExv(xL$$`V(KjYCWcc=9;R8+SqUF# zCn>HOh($75|G+yGy$z-r_nbM`&C~-jx8Ez!R4Nt(-?XEd zqPaavZ`9dX($KKac#?DXWl!UHfcNSJ-Q`cY+^Xo_Fu@B#&SOQgo_n6ca%Yc!T=DOH zWgJa#%3}mU%0`+%OgajvI{BNr>kT6YSWxao`0;!-xXQUaXE>@FxOongZ3#uQ-Wsw- z)#ceD3>UxWBB6CRr|qy&bEY5fY}J1%$1Fa)RRt-uUAZT!4mbeT@%;Pi${xn)fQbX* zlzFvIDS=bQ*_mCfNNWZ-7pFgYhUNz38e#aPa9WlZ!=?FR{Nb%;CkAq5EBJ-=*n;<3qh%&5nwC^-k;tZtGSwoXY8FKh* zXW~Nk7`zn?hohO04PQhX-I*jTfA~JGs^ihsso3;xJsxH74ui!`{U^nf*>3QjANWVq z>;6w!-5t|A*bA>8N90Y_w-_zr`@aDP++3G{>3Jmv3hds+xdH+$Co>x?s+TrEbIGD) znQ3OZ6jf^lvw2n14aK_4J(nz}N&7OLm%&}N36G*dpWRA@F%HQqz$c<0KBhw6qxC1d z%ZJ(#F3|57o(=_V7i6rdOVID4K05Zxwh_pfW|N9={=1qej#*VV<@3_n05vzV+f^ zVj{U)oS0?Al(}#w&mahnK(?!p_x#cj_CT&c9)c5+^j7#N%vs}abzS2pl|FO6I0vN! zuTKE@cZs!G7syk^Nk%a}A3^NlT!ed!SkR_&j<^_AcN!o)8*4-?&>^z3-mp|F-mt_X zKArk_W8$e*tgS-wt#LdEl{o(_Z0n{KtJ#z6d}4~Kmb^ERM`ZDNr6A5Cg$jF!t+f&~ zEh*@T8#E3DP5Z~7bC?L;DlML!yL$10uy3i&&QFQy@J*&ty*3KF6w`Dqz+7~~M~XAJ z=KMxcW`< z?bBK67AyTDNq3e`N$wAuPl%#;HjFvm+WO-}n$FJ>DI&@`*b6n)$0t;w z2;y~9Z*pE%YNDFFN&(Kp&94M8@Wl33Kn{LKad=hYcF5js_sZKqxJdD)oZ)aI)M4#T z#r&6;^^v!aF4s-WM_**Od`H4T<1Q77U)hE~WE9JxxKkUGN@8uM@;0r6PO zi0mV+SSd%XX1GzoQHqOj$ItkfeUZOx6nelk$IZ&^^b-~X1&$X8A0}MnzQj%>IPsqS zjq7V12Y#c|3Tv{y7&4Vqmi-OV0t@tg65dV^enjEt1^@MZ-bCv%=S(K)0xCqULN!sc zB+lB9iL&<==5~arH{%#B4Heba)qpv5a=QKmV3VI$;*N})KQZF)==?r*(%&5nx{3Ik zG`VMs2lhQH4De@OO^+dwc_`9daRnXWC&l(A!R0$|Z06u)VX3QqFKP6?j)sGIr|gaL zqin%fd#_m+>-yKi55$b|0AnGZqk4l^HT!p=&TEMb7FVS1wr^x{f4s6+7qP#)J-(r< zJbrfX6M>}h{G;A}h#!yZ#6Vq^uboJ$j$qyN4xMH+b5kF>>_208UL9D^F?-~hQc#+~ zL~q!a){dBd*D1PjO(1Yjz-lnvLJ}FUoU^33Cae%?Rb`7W_zyc#7Ttd;HqayPZkJId z@gtzQY@{CsSA+%IqK&-_6p_t##F33k)d3u}9J^mEklGF;Geq5Zd*S4J7?tPrZt1Y+ zEUy7O=NBCR7nTML7XVEHbAR@accY*F z&BcDI{q&EV(RgQfWp(%CZb))oGa-rN7d&h7x5Ve};TLl0uoAB-W>|9VyV9k>;&7qa z)LCc+b|1=Qpc`FbJE~d-MJ<)Z&@DrBN8F7rD_N(+Xk{k9>|RCgQ(+F>rLECiS{~_Q zeOB{{OpQs^$gLQlh7!;}J&H11GhA z+=;Djm7}S$JaNx6hP0hgh@^Nh!oqhGUC$KP|6Xsx_T}48b42W}zJV(#3uadDYoybI zr5W;x{%dOO4Db2(_IYLvE}K)72}>o1M$x_<4mGXEhOoa5IeIF#Sb)L*S)3$k< zEZGh(=>Sqn!1nq6XcDCH^1Z_LQ%HT4x?F$vSMm{W=dFkB!6Z1~VQ{=Mwz^3Lh6LXj~qw<~8 z?0~1E^scrd_&|sY9mml{8buxAOo$)Qy{HRqK?>?dsi1YW)89dFZPSE^`fL(t1yut8S{+##rNgqn7 z@?R>|(0modz|tXHLI%~yTF1oe84|#|*Jh|>QPc)l6F{fV&KzI&lc#4(9mm>1{n{L> z;d!^DlTEl~QABSy1Fz7n)yafryl6+_ zx0nm2Gutr1`+ktz7ysBIOTkH&d0nTI#t990YcT>v*-x2RW?H2%B@`GwTX@dKoGT+0 z_Gps7Pv(V(sy|0ewz+Ugmgi-fFty|p3)c)?T%VMRp6os>84*6KReWL09vFF41R|>D@A&TmSG~ZY%moF`m7bF>mX5)%X5z z#%~0E=8M(>-3}dFZ{kd;L>qm}YKp2&SlaEDz53o80!e#8_%-!1-pue_FW zbq%@n70S`CQV-^+v8EN zVR-g6!E8NT(V_OsK`U+q!yAx+7wNywRxJdorUS%Zwv7w>JMAfl7k|Glw zFePW}XS#_Tn=|H-sD7GJORLVCqg6OYhgewY(J&=BA7}%E zZ>L~T$~`f5gu3_|Y2k45V;;%m<=7zUV1DI#0TTu`{F044V6snz`KRe_ni!&qAtc_d6K%EP3>zs zcN(hvFvd5IY15gX*+@q)U%JuEtmMmPNP1;%Bg;GGy~x{&h+t4j(VoAIslL%NeiEF9 zTl5hNUdXd2wf+OUV(jFJf{jr?eFZ`bDf6#2Rk)`inZ_KK+v@@a3vnbjdRV|aH3C2) z8$>0VYi)fVqtsH@uB)=13-fWxVFwZobwPwtZ~ z2@_9QmB)tQaS6~)Z2qDY%-AiwJdvn6+Pv~rNd6b#+pomIgR2q@NK><0rF!G~<2{XQqvgfk?yZ8y7KvrHHxj^kbh-&MJB zFWQR_Sr)r>jXN_%r#OzswMD_*1|l(HD%;~izIxIFo%(Mx+sX+`HEXI&Nzax6Iz^P- zfR2-mQ?XmZDCtu5GK3|3%g(sW1~#bp44jcvmO_&@_fAkR2TehF(`7xvQw3=H#yqwN zT9HF%F8S(5ZD1RrE6)NQOnJ5Y?PaL_3~`I0xa%+E!@quE0-Am?i{Zfr0lW1g{_pnGLxI~Wy*^vncT-PiB&>p@m?c@*z)}A zuKae+w?uFE`1%XHn?7(~o`e}0$faVrS~#rWj8hHP<>ckGkJiOvc!h6?i`pL}uuo&f zw3N%Zpt@ZyZ-U!Wbfux*iZUplBH6bTtQ$oGvGmnbV5&1|ci1?ckxz`YngSg?F&%$o zE=YtT_}7Z%Mf8hTiR^^4E2cYMk_W;hGsdxN~&M>qGK%3Vp z9$efkBNyf%nU->09o8GJXP(r{+6Lybu43r`X9HdNEB|Ud&iBr}zp!9I1JvGBg-ICB z7!i4c)ZG;dJI@lpLec-%vI^+}kLcBlIn#O-tg*T5t(Doy1DBJZ)viaO+c6un(jpiZ zT3fwRGDFSQCz9KrdBVOHdeL^zVcu0Cd-SA9*zjr5ZDAy5Uep)f;tnrt1)t#CVJq^XZTP~oeBW4qN2V1I-dil4q0#DvXQ?4$sMwhxg^qH55*N|?dSbVSR8e{5 zIn3QcJ=$KZB^6jeKs@Sg3wSoW_MF~XoE;04?Z1xvm?Y^){}8wJKM@K2CFc>=`Y-9@ zCacO*kLi0m^Ji&Ae*RyK#RwK9e;iFqnp@CfG=DWN3*tdQjtz$vUhf`7)@ai(!Sl@-8}1 zOQ!=EfXT}3`#M&Q!B?+J-HHV-O~Sar-H&X@URDTrz-XkQ@TN+51_LU?&t)f5@>^yf( z!lzixc!}eF3B`|Du#Z|hqG}$6X0a*zxfeKmd-fC^qrlGzmepi-VI&9}w}WFWee#l~phKi+ z6JBj=i~M}`uMPNsojiIECWBPv6UNEw0=4%Tq@LTFd@edZ?pcW+>)IovKpDnUeMKM~ zP5Y)m@pA-E``UTZl}*paf1Oc{FA8H$$T)bZT0VZb#Vk2!%Ty#WS$vYW#R~Cj6|0II zojfZlwedXWc09M=?iTbKf zcDRwtp6{{rnl;p4IDKeTiYcHn7yVpH{Lv%*f*FBBE8uvU-Gk)sbqijNWDeC&T(v^Rt{rSf{O%p(5U!|Wy+I03sv&&B>3zPzqKjTT#%7(AcB2; zcOX=x91}Ia(ua;P?~sZOvWGa<63Nm<*3kI>E(_tmt8GrUU(%FUVKOe6e9@)7NB85g zgzQwp+&Ap;^3Xqyw6OGSMO&;J-Ki{-=hKF1CLbN#KCUU>f!7l8M2jzvohqMHsZV)a zyxaKYSjvdrKHkWhGCe?=8uTas~CypFQ?ID7(+1E8O zt23`IYiQUDC5!UfflX)cd&c$r*$?V@POLF2JpoM?tU+&?MpyPRU2sSEwEp?{(?r%v zny6sS(hf~Vm5!dXj0>z8A~`cmoaGtKWKZkdzX3mns5woKu=qT5cZfN_ifOPSKkuSt ztg@wd)pKe(nEK^uckG)2Iho8u}(Qk|=GQ6v;NW|IZ5$zO|Ui!H2&o zuI=1EVCp4Iu+}$p;&)5Pa{DXNl>$M9$dEjB~NXZ@Du`VHwm$etRdnhMQZy_vZ3R1+9C zvl@)F>QvTP3(5sV+ZeA{c$zfd!VV4PCJ;XbAw@^sTxo9NHgQ6udI7&_+-h?@741*2 z{Rd8psyvPk^Q)F?Ji2=dLqe;w{?>&jWA+IeI62i$@V$oW)M*b%egd|*!0+wEkPK5w zaF!eKqsnEpqrCqD7ZMJu`{0!`5H`Y>ae8qV1-vCwQ0>nNM_n2z6ug0~`4yGQaVNW& zXxv5f>2ugp#GzBcqeaqz!;X^@cl!(*{G{QM%^!a%De(s@$*MSdU@a^OxiZu=)7VI* zA_OJEXN9{ZG?~3Pt_1h!dSxa~MD|ei-VN;crxHNa*_7!Ib@J&+U?{dqyG!h?$tXP$nlm}?>ZMV0euOm^oVut zU!ekDa+{WAer4A?Bb?T)zzur`3@Io|wh5Mj(_JtqG$vF)le>VY?pgtV{Pccy z>#`;Ui6U;Eg(!ak+JE?NAaAJj(aqbV@aLh;nv% z{S#Q(#-3be>)~k~{hnX5m~$hUJTH^QF@8_Z@Ju+~G>gnPl;PJat>mO2=nb?(i%VKh zx*zhXlM=(MtPTc0kjNSbkAF2eiuPC(s63yyAJ`@j+ww#aD1FD)cx!VnGpJJKoZ#=F z8o$YI9gR^aO-9S1Cif;O90$)8ga(iVKa6k|URsC_;SV zN6PCytod&0$)i*4@LcNO%-fu&S)@jCItBaI?Zy5uUqnvbHeL<_=!a0)uTb}Wf3vP*U18$AQN*ZksUSr?Pkui^a4HM{-Jy@=^P*TI zTTbnp36>{L+4=E5OYPuHRG(98r4L^X&N^uSNU@`IR6Fcbi%4mhrpOqS)gS-5Y%>$@ zL@lqgjgcIhE?=nR6StF-pcinT~9)3^!57WI4BiyaK8qm40Ig`BLRk2Qhq6vv9KLdK- zJJm2l*z>4g$UhQ?3V7l}XjCa(m8a%_R=a6;@0!RT$L?GH(9P{YxJnC6LW7ODsSN>c z(M_EBV`8at^=0vp;uMFE^)y8?0?C`c%Zk4Nf z?~CrBz}lp96phucu+g0DS2glJw6xt&^Umn=w>)473;;nlD=a|$WRqc-Nt~d{)~*ii ziw6%TpXzRG^!Bs+9ir~xL;cqwiwaS2nN{b2S|{p?a)wP*_xQ<4=l{ikF%>`Uj|z{l5k+q)*AJs5Z;s4F16FPr^*G+fpXl z9K){VH9UUN-7&)c!q;CHGiLp39TTP$1_DIpek3utu#FWIVy_73bIaFW%i}3@mWPr+ z9q8V|`}IaoAXDjBCU7|O#7ml!yb&^bRTjMdqe2<~O}}R-RklTZtcyC;aHMo7DzxT! z`g5S-iR<1yVGYrE={tO1bRdWCoH?!-{~uLv9TbK4y^o`)yZ{Mlr4&J01Vmsd1ySkl z6a27wvm*qXb&-`YXVTS!_O*lsB?3E>Z+2rtr|@}T-ZiT$4O==xa=H?h@Eau}4megXFG1Pr$M|D8 z=)$6^Gdzj#k>OFNV|V|5i%w%740KFMQg;EmxwaAdth?sdwc^CUCEU)@G}oV=Gdg_N z$wpJT`s!IgnFTIQ_2h9hbIq^G&)Rwek`j6}$Q>EO6s8O*TlVAVjdcd4t1&L8qIjvu z8qf{~I#9ie^w^hKLo7uQgYo!Y_{ZvDpJtM?QOBfma;{3sII3&ewB&P#rSW9{5#A%s z?brRU>mO4S<`CA?Xsve$5pb#c$vxruYeB>SN8yA*GwvDSx?bp@%u2`l!MF#VegBlDIJP+)5jL={fR~o*bF~YEA%KO>T=b`kt|M$g> z`toxd#FqKq)~0xYP@X+AVT^xW{Y9m?t=eif0X~f*6$YP}I=1BTRkQ#xRX(qdh}hCx&1iin@7*foPM>WGPi4h5e0vkT?EL7%`2_r+J3sTmKDB7HWaf>4u#jt_3#|PjT{9dQY)-IF1X8q&F8ev4Jb(WZU=>?;7v; z$3o@Eu}W6~Z(fmJxc^4PbN(0Ai5ZUKdFZ2D-|J~c$DZ;StVEP)QNsVGJR`3bh_A|DcA+I*F4@2qIRF|l1Em`sgGPr zVumB~XzCe(&ie18LIiH}Y5?Wh3RMg01^|Vx#Q}^!AQ94m2+V%Z&1gpOM!d<~w=LbZ z2N@5v5VQZUy8vH;c{rca*rRv*5f$k8o~%50EhPUYsfi=kf@Beb^@0?vc@$eZR3a^Hycl=R_P4a8ifa2Zz9 z>Y=eoO`4_sq~1uW`S`8mw9DmM!%2mq^@L~hrqUtg%8dm zSyM1Mg;$E|%amp71@p1(?~i}Jr~`xlFcQmlbI~?N7FK0!y_ZZd&T-IvX4SO-nVG>c ziDPfe3mG4jP&E;GsE&0TLtgEbpLKC6RUGf_Op|v--bCZ`*zY3>!T@JrwGWeku?le# zo@@!6?MKI>&==C%6oqu&Ro?_Kd6oVZRD}!9u3EK+G2P1{HWN7~8?XZ%NAFZwBE9!k z7`dnaiHUE7&>ivLRE{QZxZSU*UFWQi%P>7PoXa=DX7g)c^-5$5%jTSYKnwHyZ9v}| zJ>a3OEUR_EiRZOR<}ub`e~AD^Q%^BfSJ)mp$^~?n4|H!c{`7$qm@Um)H_6^5{_lzO z`ZwS3`YSqMdvf@56Eh2|30+5L?9x7lPsD3R)1LjXk6sOPa1(f!?Y)~xSv7fs$k3Te zhyNLu)k9ol^G@v5^4ryh%xW%p9}H_fk}WS(sdnv1Pt}Ar8Z=9T!xg55L6m;PGA<|m zhxPBn=B6bVQ8WRKNXQkvBbn37>q37s;>vTd!|LuJaa=Ey5`bR~Xy!gp?5QeOh({71 zPcQuJ!M*4_yxP_Z1&QWxK>2&X@bOpor?iU2NlqoFDZE&^qXKjz9!44NO(Ec(_4NP+ z@~KaCEgY*Tu?SnIW>&%H8q9yVc^~8!7edZKr)m=v2jlA>CICt5)N9!Y&kCot?-FXA zk9C+L&iBX4>(9s_w)~D%d_hS|b!tq>S5y3DG8By<x81zk9UdCibjW@(d8Dfkyn_8Eq~U>SFk`CF{vMkjwok=PFhRFxzOvzK`+MLNT5)Y_4XCDEP2)5mf&vq8*B= zuxJle<|WUvy7#=aeFDg-ihpuA_f0->hQtjINL_cEJ>X@S?;^1CnKPymFy2hk^L#S8 zT0Y@gA#AUtlxm>&K{0%gTViWBc=m6{hty{5J0{b@|6hk7#^ept;5FvMdXLeB9n`@9 zx#|jIeElLhZqgSf5FVYqtjsC?RC@Ghct)zJsM7Ej#>+d12(QvikB{EF)#Qmbh|Wq- zHrg0G@G!cJsm+;kp1Zske;hY!v4pU0lKcY#0P3e6B<_O8#6_<@T3t|ry|R=>55F1F zYstSdNbb10@A#96`uv|;Jyfz*pq3gii$&Gk*AJ|2UfJRH|6Z9cO}V!Ho!-m@kJFS+ z{PtoWCf>V(Gp*PHU9t{y#8i|n7%>4x?Pp4PFw4!H3wAC*z;K2!W`KH89?YrE%cne7 zCwA@M4*Ws8b^aTo+r-ggD`!ide+1qN5iaRIVHJ<)wf_jf-C0`J1d4_7lb{)gMRtCK>^_`&^DPt|(>)A4QL3eZ4 zNr#pmJxBQm!~}1gPRsFI3tCScNU{@$Bn<3$!2muDe^1JO1KR%-^5w zr%tBXthJdZ47U@bN|-e54IH7Sfq%P z-iYvxPJQGoL5mR=G(IDx>e_E%$aj%a!d(%--J*-bx| zp7WJ|02$oC15~jiqqS|g(fP^FYMhcvtyR{#5F(XW+xn*s^xf3ru~GsW9-`tKQ974D_egY)85^w>&<++kXjw8Z?Z*S>p1GrcQBIm_AY}2 zN5*>A^2wEY!uqx{%r0x#Ovio}W4|T)%OC3!vvxPgmE252Co|mH+I)n6vIz$3RePv$ z_aTOnPsk>B_1K9mTXnha=10Gi;G58e_HD4Ux#h7_JZp3( zW~D!>%d9$oxg!-Lz3mp0E%F1Nm4hM|@@0iHrdb;XNj!DP2iJm}QkUgEq^c@AS>^UY zy{h4;a=lXI+Qj-)f7^J1{{um}PM&mb691U>-R@OzOHU(5_Veo`Y`5g2_gHtY>KC_?`kxHYU)E zYlU3$Qp(a)TjwA+%VwShT;Loh^cj0}%0@zI>?>-l*GWuLbQWfBnJ>O4A;;X)7w1}P z);qi-M)K0_*nAab0^o$~r~Ec=kL~7iknb?y z{QEK)EAjPHoxKh+hOfU^524VMhcH~UWiv8ZvzVsUa@ASa=yb2v<(S~1##-*{{$Om% z=BP0L$Yx!uCEtw9XgodiR;D}>`ozkTOYO(Oipf%k8y`kEJ(U)NYQ@U=*4~%z%PAG<7C;&{k3c9rQ`YXZw`W>S=+3R zuLwT%ufK{o1~ns>KGQD0z0neBM8Zx*=Azl)PBgwENz9{I-MF#h^0@*?5oK}{k)$Dh zkTg|=UTO{F;V{^RV`@91*~h7zDapo2rA&Y7RaV-5T^^Hvg)a4nJ=h7vvoyC}E175; zN$Oep7KISZd@(`nxMICc=djixIb21intl7fI0x>9zCbDC^S^6DH3&Y8`v?A%N$DfB zy+IkMVM>Zbai!Z;3nwjX0*7UaFnn@92Y%H_UxX1s{T{~a7xL^IX zG(9>p>@R}*`b$2c-PnCRnMzHjpMVnoAdLTUeRsDAi^%#-dx3GugRvs+n{a6^e zG4HSo)@XuDE{X=e1@gn4|Gtpq#~d%%7zP4)7yjqqLpR4y-m+N+8<)Jgc=nXxk;_9| z9CzJ}I`J@--6SZVq<{V|Y@G?2gmk#C5jC*Bja?pk#@9JNCRRa85JkxUbg=ZpJJC&W ztBjPL&VysWBX3~xbWu9r-JX(E$*GdvoLJdR3ue^7i2I}Ge6W&Xw;?}WDWRHt+vz^5 zB5UwSyx2rIIiAjxLj#nCO}gFN@dPs8ai%Hxcg&oh$P~c^OEx#6CEoc6nWFVpIn8aE z`jq+CM8$xY%X}uWFnDj)O)7O58dy_wj~1%xQ=L#)<26>9fJsP^r;MK0$8^)$)iuvRF=9A!25jABB$^=3{)|cg{T_B_rQV{T8DU@rE&(lMQ%Vs0;?LFyL zrboJvy(-aypC@`1pZM>`@QRzACG-BR+w?runLKUDNN+z;V_&6ku*b~F@jCKKstbHC z?Q{LoMD?|xz8qFKU!GTI**lC-8pQHEc{Gpil1QYB+0f>HGVQnIdp7_4m>f0jF4Eqz$&|S@Z);K zO5!Hk-!j=%52E&VgnA3yM9+8Q>v>>zowO&qNf#B7khey@I`GG;GKSkRrP;5Y4?>EA zSQ^u$!Le|PL|ETWjQlDRvzn#}a_SufWFK|z*4lrLt zG>g;cH-~1nf<@!u+1aG5ib@?y3x&tqaTQwv;k(Xr&e8Q+BZp#Xww@fX^|g6ZSW^P9 zF)&#ErzWfQU3DhiYc2w?M-CYT4pDuWRFTKFtrVybRTf^~MFKMw4Y|8vcO|VcB4i~h z>`T?>*H0{i+ATR@_x$0*p0neU8W}Ntg(R{{vFb_R9YR_BMqIuktUc*qF78RAX|4=CSaI*R=EGW<>W6RzG8yk2W}M|w&meR23%Ufj=TNax$wW*7 z+gg=!_|;VxN(gtQB44@XN4|Y5fpDW#4-b$yK~=A1<|_^O^_;SObg=!}d(6fey0d;F zBh=rO>baD*;>^r08PF#azRwZe?26p|@mpp3kJeU77#B5RL&~W)mZRnV2W(3Z2=CPc zjELqh2$GDtGqdJJ0@$ka%wT(0!>|19HRtgiJRx9T;zO*Vq#|Z3!6W>1+oy~e=^Jd_nOd{w! zJWEHPl&fdiisp{W`M-FMzv=jC(ix#4(SnZ`^w_jE*Uz>2`|`8MjeMlrFn{^(}hY4j7IB6lk`Jl=e1{hhx{`Q=Yy1-s`Hw&yW!{AZRZ=3d)41X9`^Zx#ferW zm)1-)S5*iKaYi20Jy;Cu)S3DF&2v(30a|3-U_xG)LFUMIG^Fy-zJW#YY1h0Xa&E1F zreevr+l&c7dlF`}>mxO8H`*vW(%sUDhm>4A+S7*Hd|mDq4xAqMoodx<$!y}OG>B7C z=|ALq9Av9B7nsj;T;Q#eiLu|n@#TjW#3MW=0d+C(E&Q(eug6UDD{W@A(p4kSz}9_6 zB7xqmbi**-{^~)n86NK3jGWlr9KywQmi2^C)pJLO4{V!An=sw!E>iAyMeTTZJI@E` z6z5vxoD#>c7GAR^!`N?*_<3nduEL?$b&4^aVCM{VNzL<`e=mdilPxjnu|E^5nqTqN za5j>p(mEXB^tyBKd~JOp!8t8HbLQuUXzZ|I+M2eH3 z=QqnPnk(5`Ze=+;c8&*s;wO`5fpRx+NVZ-qVAG2G`MFWnM#to6hBs<`z}_eyJ`r7dTMWX+x6`P+7ou%pdV7x_Ha3m>Gd2gq3F&l zPZJ34AKr=XaK}rjKx(4us_l+LKGXzuY%dlGzb9UHWYhykk7tg(pA;~>7s$t-=H(ox z3+)8JVl_{q`@GURkyA%sIAA5xtLkjk!YmFdop`0J%n#M-n*jz-RVrRZK$G=X?bg+P z6>WZp5qQdeIV(Zj_a;(2{^EO#pN^Zb zPIEpT;;67XlJ?WNQB%HTP0lqwL*^Q{bugk8?ay>%9;MdN%pshO_n=;vrLphB+{xwb zflK9W+aE-v`4GjBoNRu@Fhf5ozkk69E8mpm@~3zCP*+dEpxD-SJ*9`>r~_Rt%O5-^ z%dd}$hU?yRJK$;yFGZi_C>}gVn4VCli;<+*^KR)Z_sU$UB=xPzO%#a+i^A}OtVHGd zJjCfuA@AR)C(CvB$Z?z@9V`sLeqo89t-T6E5Q4Lwy+aSk0Yg;xmbQevCKF7YUF&y@ z6w45TuWbe^Nkj8J&#q?o!$rjNHFK>?nd%>}*2_rX-(5)ytz6Kuyt_U?LuY~&eIXHP#<28RYFr*3(6 z_ehg$LS;cdDFE&=+3{Mg2^#QSf>F0H4*Qpbh27VWEU{!~Gkr3JAK_p}AIAP~^av27 z{%C`Qh%Wb9ov7T-g`Wd=7nCNE{uDmtTHj$cLD}&xVZn@&)sn2fatZ0VI*@pY+%*w=~{v1)fXQQpI#Qg!k;@$7jUR zJ;UH~S%KsGybSYY8g4>#|H~N{1sA6mhRKO%>k>BR9rblFmRU zzs*%?WYD}^7_JG2iow}3AEVpS{zW-+cQ^@5!aq{N@3`Bl(yac{VM!BeB>fZPtB8Er z+X)wO@obLS8S*rGrF#Z@w7kkbL_6Tng>mVx3h8)&*AjN9ca!)Ip`jNDM7P) z2P-{2^L8>Ow^)Mu!qaS9vqUa<8M?5q3h5(RyNoy-PJ?JOUZ}|Y>fBJQcYWCm)PQ%lNZUOASIzcG?G)zRh;z)_=O}oo-i+*#U$#*dSpe13!L9? zaKfa1u4k!k>$f*wq>2io!}0CK?U@Hb+dS0C@~X9-j^*J|8vBy^AnG(d38WP~#Y4>M zL%IwJgKg76%%YKe*KHlc*@vaj==E z4s~ngB0SzTb#onbl<@%;* zT({qlpN@{+FQxg@<82C*%&}cE2vNTeM7OX)O+6$|j%dpxi1GOpmTm2SUgt9tlW%VO z`)1X-mEM?LYmmo6pA70Zf(_&L;RTVp%Y&0WT$`>gT@5{SIvvnP|JF3XWrq`KDa|CjxCd(h7bViUOz$apKJJjwH@Os zyH$7Cqvz~99tz}LUq1iGiXYjio|pZSm@)A!48P%&ZtK0VaB#VNwO*(bdPj$SJYazR zmOt=Sfy@X_k`?W`)m1oxq_rm&=8@AE_c`uhKkLKqF&;c;5dzr{8KwD|_E}~LwMdXX z6flc)uwrnqkT}Zo*F07`^>f(-S;tefXy`WzS~V2QQeNh4o5Y<_a`0F8L1};$A>lNl z)d|%+c>b(8oR(L}x%`-mq4d2Fl@4Muc>D=Xf=T&NXiRnlgY3LLJ1R^!I-naS6^i!}cgbqd|*;GMw3a#Y6Qvgf`Avl61E6~Z5Ra_}PtC(I1-K#8W>UP2y<-Om61uF8s65c|jXG zE2?$>8A}Mk_wxM-A=cciJ$7&P0}7IysyA_MQ3#6grX#MwRcBqq<{*!L?}!(*i}B^z znQ#zk7HM_*li7?$(a+BMYM$mD&;Zf(nvb3qpFCEE(x%k-KNjX}cm*nFg&bI(bEk`e z|2pcX+|&K4>}WF!kQ-eUW!xvNpg$$9{cCuO=RRbxun*vMz4G>y1a27C**rb|l^TSf zk_%?o2(nFvIno)K6q-vZPsznmlh8l|6GRVXDEA`E?5a(SbDn15Zi$)=u~LqOy^E=( zsYJiXL@6*80V9sTb%$*JL9f&8H?w;m6Z%+AzNlQdw2;fU`0U;R8VF?2`hm=ne@nNu zq1aW{KTZ=g^94e%Z#aGNbM{ZX@M$>^khre!`LN!EdfX*zy3o2opr|Gq5t>)qKF>+| zdv~0(*Q5Tb2zrz+V$Gh#99q4Ar6ejpbM8|dy<~n7qPvm(G*hrqXRZGjx^5R{sZXu@6$O;#gzx5?= zwv!~8SgY1~=e%boc*HFMFu^~&J~;d9*#2|yY~dlsMx2F1KeOlFAb1FI(-s`HyVO~V zctHW4yOea8>0k#92E`2->f64QP2s7KTvp)FWs_8fYHS~HZ(rxmMBtmt3rktc`eYCq8P~hPpS$i-m}7e2 zg3qSjC8IL)18hsFog2@>d@5y1FJRC9raenr({qfHVi#>KUZ9sfVch1umI(FB`1))d z6;!zcycPsmAOz#{n5UbnTKQVG6cAiDq89^hL&I@8pErZ?IKdRs{#_#wK)ht>)y&8F z4vQew5&+b-FQH_uCny2719W8Kz#WGz3c25rJ|+xj#*uMFs5$#0T4xnBa;5NOZ7pZ6 zD$o&0I@*K0);&$D7~yqW&I(R%OEBfVx`Wo$ziti|p?Z81(Sx>a`E|es3w9(7@1S%t z;syVz)C?jZ-iA4E`JVt6_N&g2e&*N$&Mf(cDS0OB5oP&-Gd|EGHekT<`ju#3!W!>b zd8~&gA0_XYjK1-#V9H~CYCMQAsQ4_9AvzJ-hP5aOM?wq$s|j{ z2{T9X^-h*F{mJx0P1$pF`}NIN16UwBE}6+Qlsj6|ii_G5z3i{fKt; zLC6v^SP7&!m_Tgo?xuK;g`Rz8?r9|ScTArGj*HdQNz+nY(uiApbG{i7t2V4(N&i7f zL|!^~RAR}6q@$(j-ick4f!gTu6NS5j`@4O=sR<-ONDe%G1K-V&1a^)`p8-rPRBvW< zNV8>(yz#AUVIlwgo)=Q8x(RW{BUeF6%PyVxN&U!7$*M2N-Ka(OWGhbwTW^>uNJh&l za+Y)yyle9;R-rGTE-0FRX3R0#Dd>Ud5k7`J!$}ff(${1ZHiP`Yo|!r{d&^Px%V?CF z`uS_xeL<@|0W;uBy9m(?1d<;>=fdnK=Av~xuf7YEMtb-n1i|(2-7=EkE0f%eNteSH zIqaaUos5roMU8!C1VbIj_jgsHY*(LIz%|I;8PTQ)CC7z~Z))O0k44AtFRN z2X}+n-A8bKvaTj|f=6UiFT_g6pdvi@;t9VWkMz!h1C`zT| z=>s$zX)t6KpbtLQm1|P5nw}xARyilDN8V3cmEfgy({*vP(#y(~EX}55x=lJWe^%L( zmM#{C11#Hdc81{D_q5zU+DsXo5mzz2a6xgg?o~(KBHJJY#j{T<>3yXAI)_(r_eRq0 zwGPi%+{iPrhZ6A2uN$dhFKm(7{`EX7&fDQ=J2UqmX?}#YmX)YRU%mN`Kk$|`ByCk2 z4lk=ZTVUIlVF)JN*{_;M?*q_*hJNv*BKe zZFx+ee$zTTmdnx9VO8r)KgXOF&+oHe%HIzA-!J5eynum2Bu1fM=qqQ?)UM;s+SF-- zhg259FJoLVA>*qAa-!(NWK8ozxvMK{&&`YUgGB3+BH>t?*+~On^OzXsM~0KlC!ryd z2PyNN1Rlzt8lZuLoV*_4eX08nP8nG$VXdog9-Pg{>rKi@`@Coz*k5lU8cCZtVx{-I z_`w>GVfl(02yh};j&p776c-Wn^p3o{AxT~63L|?lAdEZOMwB~{Bj}lqrTVo{C z03YC(Opkc2rW0&=NC=NY?lBJyva6=HK?5X1ZAyh_DJhtKV@W(vFSefzosQm9+P8P} zluU*bc8|^%-Wit*^ozTOjhzg+QYs5Wz#BgHQ4BeaLtEe0n@v{%l(q8#{q|Nhj@n@d zqyrFh2w)GsX0VBwV3-<2Hjj_@bgTQQYsgk1=Onf_>m~ui06feBIWO!U4vBYq=@cJP z&{YtEDy>D5ekM3Rp>J~Us$f!*Q!D3}V3FbFs2-Vy736*6k@-U{Mmu<|e{F(_5=R&I z@>9juPLJQnv3%sVGdktt2j=Hr8$j=YUX$s`y%ATvxgVk@U$%yCZss^FL>?MGX5XtP zUFGXJ>HbhIU6sgeH)%I!J*Ik)_(Ccviz_)iTw1y0@CF>F1x+g z^)|N&zEJB7O-yyA+~rQ z?Yh5*G!RRAlCi2hEWMEUu?w;o9 z0QrSl3y0Q?L4PU2i=B7;x$9l|vM{ITrm342_5hyGErAqG^di*uO2=ZxO|vbi-$HXVGxk#Px`#-u%*dCJx*~Y z0(X{o0#uw$o($m)|4#Q522O*ne6t4`PvoZQs3J3t#8b%asb0cThBBj@)y#Ndk%p7@ zy$0Sn2e#&H;mDpqC-VXXNqU7{=<0KaN69*q^j#}ionNQcoprG{g}^hs@JfIH(U32x z$unVOH`|TNN1Dtgmag-n-{>VXKo9%=;wKratNr87h%OtruYOyMNdc|&4z&4kauZSo_sVHB?!W_^pmz)xi0CTgpf!tRw>b5YG?kJ5j}R?(Hk;8FO0S# zuKGKAWXsr(J!Hk$Xl+;qhFV%$7{!@>14BY}L&qfH>8IS)v#cpJTf7QlQsDiC3V055 zjI6DQVX8_K6x3pU+C6IrDAf}kZ34#O@tpq3iEV4wow6p)eQ>57W7~H`iU~DkIn(%^n<^gFjMv zffQx%$gqzNJLS8DLMvc->@F8Xt*(g67zpn3o^u=d!RnudV~eJ*?QBxiAE^f4g1?ar zObJm-Uro<-qY@aJdfwI5omBXCydZU!+P56p0L51td6{t8bQr&N<8!l19Oxvys3RfFyON;;1 z?`>6*3Q+fDgdVJNI0*d>b*C!6CqNO;s3mQ3eT2I=N3eazTYne~0s<3pHWRJmY3>_) zhtm}zZywH9Dx()AQ(Z7=XET|L`0&DPO9o4KMTW*LmRH0G(uJ+TgXa+phX(kiiRVuZ z(}L(Rh2&^xq?WVRL{GQv(N9$@t()~76a?|rVfMwt7iqrHGSs&-pq%PX34!Lj`UN-h5 z<(>@wpjE)s%>_$UWSEjZKboPY)CK}Hki9|cT*%8kLv#8qJVai~WX`9x2=BMsj)0@G zt@F$9s7-kEQu6lV%eMWn^rp|_<%Xa3@gE>h05(_u@8+hQQ)2rovILL~%?918;!-JS z(eVZqLSOjcUXHXRs<}^Q;3KetRH6>USW2$%rpVu| zUvQGkfBbBr5js!notNNr3UG;lLa`*fP&;B%&J)n9tWOl;V8xAw~R81XJst+;)OI&u#8Hv z&536q%p8wn%o=sU+~7|43S)usL8XM#U;Hdepv)zFF06TYQ8O8@x@5?EAk{Bht3k&3 zU?2hcWzSxiNpIT2X7U!=l}p|X%LxnL3yO0dgIMft_SL)ejcitv0|pD=)X`VxYcvgn zSL?{V0(cnt58&Y-dtU0#O8Ks82IguV1(qb-)%oJ0G8Q0PSJmxv0r510yv@?Abnchr z5kTf92IVvjFANp@cm7xXRjtXVPx}tRT`-4{>azxJ?&3ulc(rN5aPi-v&798h7hlj6 z5y>0W6-K=ii-`>L!mZ|=IT4Z6;5B3ah?eXKaScFdTqbAOPKfQUF+MTcGE9G(9*c2Pc&M7;J%yduTBlUfePCJ|ja9F%_xsR?%K-v6Kj#lQxG z|Gna)X}6X4kOhLvuCgWnaYynaNn_wy43gr00XC}{X}(MU?n#}{+H~Q^`HL?32QG5D z&hF=P+p8{}{BVW9jgG2Wo@5ekulq6kT+wzy{jA%Z-^Dk-XgTs!axZdVpI0DpK2~SH zT5C2q!>4b9({fQarOq_!sKm$2{Eb4SAXCLuQTJ8?CG|!D{`gCBV=_mMErqUpwyY{I z5VA})>=)a1Xq%~iKPO#*u22O_s8E-iBxss(4-Z^hM{ezx&Dq*$Ut!1dM4){pKJ zoIF**t@Bb`c&WPWGM3qxAKRt`H0Et;zA z9KGYNosspsKWG8!Wb3xC8r=2VF>QtM)w+1@e`RK|#d^nL&eF>KC2J_kZL5ny#PvGB zv|ng@!15hL>&nOLP1~neB1#ceD*kV!O+Y~AB#87Ik~a$~dpf*BeLmx*R!B!0lyP!i z=+GT35PISGQ`|Luk;*5;deXG8X*k_Wxynlu!5f3GurA1RAk4u-{u$nmirINtcF+04 zowsI{9_ISfVF;>cN0!s|6EN82Wb2eqyPb6yQ+hLS8*DeIUO8+a&tz(*a&JbD*~0S; zGlKKlIL!@gRvBaGsXM=BI~(0{FT6+t7Vgwe)s3mBtQ;ZLc_knKHJ(>;mHtm$4U~PX zAoogyt(a!2X2FA9r^}&?O;$MP6(EzITvQ*IycWylqdbuJl-2pIxrQ5%*1z7~O*L{MBXKCdbU}m+%zm64Fl}P~= zls7a)xAscvEN5Svw?XkkFNr<}O`o^5N)Of!4G5ER$%b%XC&23xRn zZ90nX=?y;c>F?K zvrZ1#t(lpMf0J4&r63mh?^i@nY#=y~eAkUo8#wE3cdj_vx9nWh8EK%B-WuGT1~@p$ zOwiK@pULrAD5Ukj9~Jn6&S7xGi(@dWas!S9h(ldAXG#T^Dp@a{=e4@`)_p7Ui?=2H zp@LaoA;?`Q@-ICIKJ*OK>GuEp=NDr34hym(-Tp$Rzt?;)>+J4qJi_|hBS%J)9TMM| zJ|IH!MK|YJ*qU}(!Mx7k`m6U*k2@wKku}rIF)^IK>sw@0<;#x6EO~mhU+)wi z*|(dBXd}c^0bf^#OW!8_*?vs8~wWRw|{s*B3EJKGN>l8*rk_e z+>wK5o*$&c{?H;1bFIyn!-!UEw%iB$Rn9}V`3kowPu}HFujX_LiP=L!xu{p8lq)2g zENYWUL?mO=#p?w2udMV~fx$`%tC`91C>)ULxi@i$T?*ZER!l7!QB!(LWTSo2+d1xg zSLB13Ol}8$ME5n-`m52+aDJ-g>Ekx&v8j?83UgLjQkYp#01tEpd>XGL`k^#>nW!4= z2`cdJEx`PUunLoCOimuA4K>c@?S&KrZ4U+5?u#BP%LSj0O=!FWsu%E<%pf25JLruu zr;WBElcQ5rN*n^E1Kg$QjUhe?BXfzvhF1r%;vAb&zi?E1bta=$l}VQ)#oVRQ0O>gz zAT=b6)h-`a88 zC2*q`vY&I_K70bV3GB6{-;A8!{DYt;^lS=uoc~Z`k%6FE)<0_CIImsOww9qal5i4N z)AtekThifL^DWfh@xIBoT#Jg`&6c{^GPc8GS>g>(uO(mPFLWrqe6uhWtT3>22dmKH zEI2kd&0S26r&){0-|2wcY*ms3+KtUqdNnb8uJ?YS5cL2XI!4$vHo=aSZ@u~q#ctC)ci4i@sII7!;pQv#uV=6kLFpA?nU z)`l*>TW*NNZ%wQSGG;R!Y`c9gm!!Vss(FA^3=?*B;y$T3Vgkk<#`FQ8Dd|%$ z%0{&@pju&CEFkRKie~UU3kxjO!bcX{67R@T@A7V#!fj|0M7FXmeyayRf=m2K@e~kx zP97tx@Z4o)uh_q4$W0$UuUw%S*Vd)ms?j4D?C4Z3#BKrc`C?`{R^Jv2b2=8aN?>(n z3Ge?9F5X=Ri_E9ur3TO_@n*#>hOB;n1b6)~2=}uj`Duw6Jgu(vj>$nX?q}wY7y(D; zWZh~Z8ux!ty0D0uJgoS%l9SG*x(5%oVgl{%cj;lWIoQ`C9`n&5Or3}8zKsPS9xpfO zsZG@)8Zp|PjYag5r%8PLB9Lc+$g+MZ+R2{qimUCOmL|&$IiDOui1wcnGV~^qJ77mC zxUjz%)#dR*#u_8tlZz*H<3vZuO&8Hm-JYr@oKOv6u;kI{kyv4Odt-Vv(SUA4zfjr;yg3^W-nAY@gf z9GWM5!G6xwf8YltF_}Na-_9m7JH|^USH1rUqTNs;C}^v&;h2_S@C>Bx2+<}A0~dNo zuvG5Qx^q^h*ys5x1W@(>U6}-z!9h&9sh8SSPzqLE%i&P)U(ViNzG>d=-y=W7PUs0X z{aiXVz`2-9lBKB9Ztr>%j#d^gURwqDRJ0a$K=R&)E~%P$ zB5`X~U613*=}JU{!>xa50@u4x2&2q0 zKG+ZEjXUarIw%nC`>4)2>kSr#`H9JA&z?P}rc~!Of)GopSsmOR1$=X`L4k^gdO2$I zXVi}vXS@%b1Dg4d{Mze4;}Ji?64{+o+tTx0^8z zqva=}atcZxRyUJ(p{H+lrfh)vJ7_rp zjcX5QZEjVK4*b?GSv?_Oa+n%Dw_lhz8fRnjBA`V%S1_E~hq_tdO3?}H4Xk0{M25ual@S!wO=s)MI4a@!t8 zwuj}orj?x3&z&?68PWJ^6}%(U`y0a`IPTi*|DMfkx1{gX?s-R@>+cur_!wUYlgt6$ z_ywn^Wem0DpxE}`B7|)J*Ra0PJ_7k@FSp*%caMY95Yo3;WO~V^9rR*0yW(>&$M3^( z%V6PGY4@^n> zsQ~J9h}I1xSNq?=_G@mT#s*KkUH7GTqfZP0W`MYvJ zFy9|=y;65&%H5Wseq;f7JJRjW+5mh1&~jBjQFr?(By@3}0?&N#I3xRfQ5W){15=np zC;Dot@H-#L{^Dr!>_y-EG%L`(BM$QJ7jpI)1R^$1%JE8}D+0222OW2&NGM;S%U9ay zdN~G;et(|HRteiiPhj0epOWNEZVN#LrBjfQmXelII;EwNu4U0kW#vlt_4I|N;;O1?v{q-orU|k|L^Y$esX5c%v|R>*Ewfa!@T?+uWKqD1sm$6 zkFUx9&T4V>WdNcM$d@)n1wxrI$q;_v0CpMdNQ~xRR$dLOjk>+@8b}Lnl?!-Hyhc8y zHZnZrssRvjhyw^MU`EI5NfGohf<$Tjsfdqs)y-uJMzIbqBBK|ly^QYq18X}u9kidmUypUttaqFFrsMcpu!QR*OcVU3R zR1E#LKc7nWINg>nWe4hV-doecW>G81kYAcDE+?yO=oZcu!>-{Q;^IJy)4ko!MeT%l z#IH6gM03t9HiQ>z6HO?(iy;J{MCgNRCxzGfTi8a4Tn;_K)<+{JHTMhobj4i~wC=dK zDjx4&tHpICVw`4PCj*zlt{IKN?^Ot^U_ygCMDn6h5fOLTyouOAh3a3YCY)G+fN%kY zIupjOQzj_9f&tIc-ePyJTrN0Kzjekj=N#^uw z>lC^-C5I@(HWmPC621u4;y^`GlnaRD-|ks#063~3Tvz8o;k|v^P2&$QQqjSjcSd=PbU5jwaje@HE#7(pS$Yf zkhFHk%tBT)RC=5A392;*8KOJJJogSHs$D&a2eu3R=gGqpu$05J`Wg!Is`8KxqCHj* zD;hhE^RP}=@qq&m3?NTjP!kv?$C$YE$Igx8IZV`5E=O3^O3pkzec0y@Zd>1l6M3@#s(<#6gU68e4!mHrxL%9F?ah%_wyqm4B?ukC@^ zyAkLLfGtfPZZ18ggQq8{<+bf-zu6n-2BcJaXZaEqpC|-hKB-p5Nq?EbI!m!_bsl-M zd%m-%OZkC{d(TIM?3Z53*b~^Fg{mwlz7p&jy4vI>o;BU)!Ow@(S)aGQq1bz&sqcBt zs1&C#X}_yRH6w|8?k|>uo4s%W2-KTvQTESxJ1yV!wDCLl;V=BM-AxzP(PaaN490U+bqiexyyu zNSg2%BwBMKotnja4#k?>T~(~>L9p|SAM$(6bNxVw-aWCv{BqE#LLlL7Cj|=y$#wlL zQJXPJs@TiMPH~?i)vWsT1@RyzcYc%T6!}pW z)aUUo0~77X6KCDv@lc41<&w_J!P%-7$ET`sAVdVO=-Er!&jxPMnu3#s%cwD9x2}b?m(~dK;QQ(I*8y2m26%=-ElatUV10DE+g&SJM04T;MpAi`2kW;p=H(!g_mSi z%`{FzpT3YR_S7AE{FZSsc=U`MurKVSJ|(%)nVJ$hPN!Y*j`~ncjbPB*ubL_2V9UJ# ziaJr{(OEY>C?e)wb*~9*Csaf&@{FF2NHyX4yOt>au zaN~Y|F8)gOOuyp7aPb`5JE1ZrJc}c!V7ivUDOr0ln}SNXVpaQCXG-0g+88+OUr%rL z?~_~VN;uz%+_`quv_-&|7aCNACbK0RDG$&-z`t8#a8*53e>5O3KnFd~bUFQC#*f1| z1UP>3x0pJnT9{dD5~zAwDP@+V+G;Qqx0Iu6wp$ZiQ=O-9#`;SmN5^0`z2|}CKj*WX zu@A>JRI`-sL^_k1u`C&-IJO{1d;ZQjTxgA!jvq3eq(%xm95`(EUT6MPscz0c@wm&u zJ(%PeehT5#`4c8Y zh(Ol2(WplF=qso}Z$F5-oj9>)RLuvGe9c0BS5DPVsPw$-i2a&WGv1KuuOXYYhlME< z1sX1`6yuiK_JCWMCCT4G`(aOCy75K%TlVTGUmnSnLu8Z9=U6AU6?5Y4g0?MC420W|%PGv47LOx(p>-0W>+gzyjpt6S;X z<2KW2CjpY_i7p;vVeBxsq_?tkL)Zr$@*XpfZcNAcujwYQp(E~aDy(|uT=AFkn5l5Z zD2lrhw1Dc8caNBa^7E6s1Lg(WMqa%Z=eIcUAvA>*pULDCb=%Ha233^cAEF*l*|i%v zL|zHc?j&+fBsrD2$Z9F+)Qm0V|8Wxi_58kGaLu!_UtgrnpS*o0aYqx8U4|!$*Wg~H zE-n^F5b54u1i z6}t2$6WW${lV}_z#`5ZL4v{0o6y|0bvx9P$DBX87m+_6|6lV}Lsf{2)hM#tCByI2@ z`XVhu7Ig5Z8y3E*4f2;r-SRW(3W+i4`Rvpit6QZ4Fu|Bf=ovWgXdE#sZhKIe`s%Kc zaEVf1mP{V&S^Q}QDO$v!qeA0~T*14iBYyEEP5I9CTdzcP76uz}W7M%F+{aej-{h%1 zBxE(euii(H#V{%DgiJa}b9^*)N~(w^{pZc5^71tapyOSQ-dUf4E?t4&_# zx%G+^182PTFO_=dPi?N~GoluDOxbiis?O}})I1YdU{(>m^r$j0G>+sk@A*JsX2yoU zk#6M`6c;r1?KIxy+@MGfjFJOcLb&!>RkfwcukQ{Sr;iqf5gn#{= zC&Laf72fC7;B&UZU}gfCSX~!(HNx(3W0UR?>v(dp>$|@)bc$GolWXT36@2c<8(UfnQWaiiXi()^#pU*0LAUlG(42La|Gfp=7 z_$_xo7qnxNL=bRG!Xt;cD@U;>u%gx~Y=$c5GIqGB`WB(BPk3e+oiH7qv0tPeR15t4 zc2$}}J+f*m*%uNeI7VjeTvO`XZxO)dLxm+ueEMYz5FE6LFxDpU=L7Rl{ofiEgZJtJ^a( zx>XDC5S(hb!mS}jYxL%mhZ3Qlu3jK5=pg-;ROpQCihX~(rS?&EVOV1MU`trTn}GuB z04eeE9P}1yZD+ivyE0l?&r{h8#@^-vUCn^}E~=VdByL^T^Jg&9RszzU?_#O62Wz=b>vs-O%|N7L z_x+V}gABrF=oyNZ8ruChw3f zIkXQw8DwJ)`#FNyDai+QTH%>ojLM@^Sq*6_sZ)Z3Ls{@S!K#IekIppjMkp0*e70za zcn>jic+V}hXbS{mIi~cn z;8oXfOhlG_esX%npY(mK0JG0{ShX9x%&(3!$pHSD@? z|9%~)B`mfQv=0EgXq8y*s2$8vW}aF9rl&Jvix-x2iw~#J=JD3@c$lvp=W($G=Crkd zcvbF$k_(rtL$?42QX6xB`Yky%8X5&=%SD`$%EKQjUrpG(1~|7v7`;*w#{VgHY~Nmx zVm5)3TAP7M{Sd-yMFEGaU$1PKhnUr+xti-zURY#FU{r=RYF-rNwi%4oyG`6-HH01z zC~*bY%$V9OL@_?=S?t`l0()cwFjz_-LxjPIwGtM^Qg)=Iz>&05_NNf6oc2j>+@>24(on+rZuX1$;69LTj9Yq1jsXwW<*x;!=yTAU7xWxIWfMk%7#5 z45Q?bgp%zmX4KReK3qcw8v2r(5yad(*9r0YOLsV*ZiS~(5IpQF4cPvhT7VJf7ic2u zNqA%)VyIObwDC*xA@g zB_)r%bB}*fJ0q~7Sn^-M{yH^(IB%2I_)5>x{7VxrB^%>v#nZyY$~=6FEHC$&i-c>9 z*;avRz`<;qG~3H*iA?JQov|Of!cQzJQSF6j=StVVnE5B-h_L+$^*QDX`+%2hddRm)PN7Dmo zVS&&UFZ1cxq3Urkzxi^3QGbmYh?xCz4^kU9uL~z@Mx$qT-opc`d*8Pn=Ye|35CLkM zqLMGMU1U;n0+^z?q;G6_ zTNhJB${k!z%U4iuNr$B?0L}9GN$TFC6k#cwdkX(RXojR<%!Xe8?5RB}4;B`lDlcA5 ztFF(gOn2Cai2ZsYfvh`P%Tbpa@<(SoWw&^a=f-~*yGXi^brqNnmx|Ta;uL=A zwN_i}2sC@KYF_O5(>cOgd1(KDQF3XTdtzY$LNkpyDV6=;O7kqg*zcPSfSX7xm&Klh zs)GlHdK0|wRtROk@)|R!Iq&J;XxsreRe7c5%I zOtGCEezRO?lo|9{pnaX<^CmgG&TIf(zbpkkp^&)RrntYW%j;EL&Q<3Cgmf|O!F}ri zx1poz%Au+ZgY=M!Vhb!m0|ALDJDQyhBwOv;bo-=KUuLNBBwLdnLdPlXWtPOAeSV@^ zRwLOOscYLy+h4tr2Il@nQWI^R`t_;ELu{I!P4Ea0TAn{^9{7VM7gQ9icv3E$tcIvq z=ilnYr&`y!;<(gAqYPyzLJzgy=Js%&gJ)0kFi**=UYHNRQCSDNjFs35o}38{O`jAq$z}42 zTi)KM6vK5y)$&Fpgr&q)Ft~cA_=|kASs9Uqku#tIABM$5QN-W_L#4?Yyb@{arh8Ia zLXTgg6Qi@KU;G_J4{YAQFs%3GHqTfFI&bTp2T@lYpaod<4W_#A;&{?)d1iEumVSiN=aF^yhV|uxp`#LLq0xEA4L;3^987QbmzLwE=CPfk`PBJdfJ@SSpnKV%zc8y#{%)y= zm0)RWt?ADa0yyFMMa6P_h7DILh@KTwgC}q)%6K;p565z-OvV4=ec~u&HW#bu+?b`V zt;w@+O9$XgjPjGLzsKpNtLZG&us^Iuq=-`+?8NumQIM6uqiymv%@x--_0}E48NR1c zxKJ}fTZm-y;wv70_;H0(5oZ8^Zf>q#MbF<#_RZgpUp=O^St>kvn5#mH8H5wu20wt0 zQd3ub%*^+lTb<5m`3P%zBOty6inJy|VjxI()m0w_rvTVtk@fj%VbnR4o^vz0vOk+U zQTCd^y~44T4-P!5f$^F1k)RpVWpR`AYjadM>>|mJ)3^-;u-HbHE1>lf(wpsbEGdQ( z{$QR9cyU46?1FlMmI9d1%@%B#?FZWnvX22~Lm2yEx%BU1U}>#Pi0ALEmzhnC<)}@i zHDs5dkQ^zs!{93QRl&=&6NE{MFl4jxuJmFe>&@&eGp?afWYB5WZYu3!Zgai+Y3(n& zkA#fugr9SzD}`>M$ptE%4^t@>!3VW5PRN3vOQbgaWIdXi%$LK5O78X~PAD^eyt z(}+gW(1#l*b$wm&2A)T65Z}r^yJa7I&PUHmtR4VX%ex`YXrqp^cJ|f6p5pqBJ7f6{ zX5L~9UU@ifC19r`EMlUj+oL@)lyMMHkq2&V6_|9DhkE z#!E3<3Pc6s5;3XU+CPBMH7`C2V$UhIDt7M!ps0%Ydvad6pz|uxNR5)d`FBrV*r(PY z4Wg9l##--^Ey_I0_D8x7y{Y7M?PIY?OwIWcS6-=A#PpWTbT1LMS}#K^g9lblS6ekn>(5}S>JK5TyB z@JCzj9+@sFfu+I@!N0oVlYdR5-zwgsPk;v!B{n@LJEK5|mMLvb6ks|g9>^!k6{uz% z^(HUzs!%+{c;7lTFo1U!3LLat_`?fI5^{1T|L(lRmP>(&kc;Clwrv?zMqq}2e&2*k z5z19B1lO)E<{SV_oOtT#b!j5c{E+*m-KA)hDVsR#vq5d?v*TCEn8xslt!e1}JGaBd zTp&qYbzbGIO{G=649vLIhIRDf)b%fyPp?uMvJ>agwH*`d+9*h#I}~bb35+V>SY%k4 z51;!2r))hM+TTRSUvp5zm_Wxj5}HodI4{ec@$gs)K;X;fdJ`};GzN?FvQvS}xv&#u zT<>|^u(% z{<7O@_a1sv#sWmSfb=4s#MEDwn|k?)DH{mNq$qB7Q_4*n5q@4Id{OmxP_mf*ew|rR z?t_bNXc+)$YfdUJWZG=7+cIK$t3%jI06W6kAs%!RFw{C^=5+7*JA;WaK8`V1vL4`b ztIGHTe(AeJf72X`r96dNJ669UotL#_(K}JBKy_(ZitQ!!KY0=K?VWJ=Iy;T!*(8HyB8+&&J-+RLhQTN(&kb%-HJ;?v2On}OAFUjTgRfVztFpRR1gq@z(6ZS|LTo1Y_7U!XWPbx>d0^XA`j zrdwV2myg1-e}`777?;E0^KD2rub=3Qg6c8(6{%-$r5;jmxQJK5^fG@sN5y2@KOnp( zds;b0V&pySB)$rRbbyKdIJseelq^oHd`JtJYaMVIEzenb^@I2&#Tpc!vj^Wn%M4lx z*=r;ZmAEQD4~wpc1pH&2XS2a%&0M>9u4lZHM=fY2ZkQ2=%YYH19;~fw!iS#i`@{jJ z;V=?AxHu6ORcWcMo|-P6#+hg|{dq0@W337~MvzhKLElEhe&95K#_Lv8SxhjzLGGrUC?hH?tDJ|;E)P#$r_rD#8YBgtD)p$$PRt6C zbXpD(LKF2=Nj&MfYYz0Oyfzjd^=#g=zlHO#7l*Fow+)uuBWk9)R6Vb6D{k%1G1h8? z-prY5e`dNaa0$({{O(}N<_O{8to3oR#5PrsQ^kh%25Pylv4Gfx(`d0VEbeo>HK6Zk zKU&{j6J|;yYfXM=k#L3O^r*Vm#K6v2n5T@G*YK>mwE!ua+z&nW$DV2Vx^u!1e#bcYAD@Ay<=7AC`d84Ox4(yhLNCvY5WdYt15vu_`t zz|<8_z9-+Z>t4rsd_QPjcv+8RJ6z91Kr-J7n4S~4%L?XJDheI0xE5DF1pVtC+V|k2 zCywlj?Svra>dv>yPRWl&#=uU+U>K9~=xFG8xa>_hibyLlL#DGy#VHek#}P8i6~}Fe z447G_i>{uv?GJkrTA!%=QtrIJ7X1$;7BNa-=8a?SG~iF*w;c~;z?HM;%%*U8mo^OezQZjSfwddC?8gwF@nj7s%K!5&&|!IfAp zE}w%oD@p0~Tyeih2M<3q6!YeMNh07SyV3I=p;@L08OooSRNBH7ddtw2mrVO>t{|+! za51hpPDZPl;BK-2&c{p;Pj;fBti^*O6-GUC}hc`Qdgh>IS?UJSXSmIEjo*b)qU&DsXe0 zd5Pg~`qnjCG^CTl`3T!Hx5TOM>lX8b+y=MqkYmqnwO(KG5@;1-VO&##mgHRUpB-fB_XNN2S@l=>hX z?%qPDylJ2hIazfb=xecMO}{0Iq27m+5)I`-&z^d~L6TJMEqaLs{G%?UdtvAIyr)Te2g4Qn zn!22#7!6It)l6<>Iqly<;2v$9J&s9lG1@j>+l1A1%R$2W8n+g2P7@mIQ2AmF7t+?q zGp}KBPxF_P0wD{95>>|uf~ zWMpP$TD_dtFKa$-EO-!pVU`dU$V0~Jzfnm}gHY42N@x2M=h?6CMCbT*K}KQ8T1kfX zIT6m%TJe>pSHa6(5LhhTjd2Lxy-X35@Sb%OU$VET;uBy2yM;vv%|)pxe<-V_oPP4d z-r?w5vTCa3eOzxzU!+DZEnLu=bCQ0DDU}RyYo&nk8ybfHtxS{J5H!FseK8S3s9Jk0 zWxS)=(y-Ag#^O9Q)c5Y0&e!k5diQ8#RysR7(YbXrn<9(66Asi20^>bD^pysuEgJNh4-gH<)d&?#&ALpxbpUBhjrD-RiVkmZ1B-x{0<(dHRLcd&=y z03n(j>~ql7cv+nI3V{sUOQS+HMh=De$N?iB9nI=4=f{D=m4z3%I9@Yi;lYFj>r=jB zVy|zV?j(H?AW$zv$(|@trG1d+1@NFrr^h2{jxYVZjc(!@ikrZJO|;?Ii+84Fi*8RG zTqLj4`q|=2WW*Cb@t})>rWB;Sh-pw@f<7sAxP;L_fJdYEplsFce-hK*+x4USHvhqv zsD|W+Y|WUm2StfkRMyZA*O=v2N=OL*9#AcB1~XLEsj_!3E@)yOSV&IC#K4QCqr$;4 z*Y{Cs;{n7M{ZW>r-YPa=5WqR9iWOH~J13QoH%brh!_+a7wD#rW^Z3!4VX-j%cw#cIA@Z4(rYa?n zCOE}c7PY%Se9|K1EH$eCiPw^K%WD}kCp&x518F(&DGdB(OEvt3XkLU!lB7Qs;Urch zmgzLSbhx+M3i!<=Vx#TtcSMHI-vE;KQ1se!qRD-0tQV@7px6#&JsYnQ*;3kZj}H_Q z6-!x#dK5I7*JJ!c*7_9YCevCNl~6cU;W(oYwwGfhS$mX#hk5NXuG*zv5;#&ZLnSNC zc}|5|lgjj$to_7Yoo5NHLcj5gwSf43lvl)3+ed-)qUvbXCPHU23s#o)R8T71f^JtJYA> zauL4bns3DOi};#+!cm4TxqD%{V{+^|`MD7Hi_1y7M&559E2x|E?_IHP(9q-KbTI1; zDWA$B7zRaK(u+!5Zf(hlTulrEGKyQc7zL4P1`)c0f zBCFl-73Fl>#S*M7A?KBeI`+=gdD2bniWd>;SVNuj?VE9snEwlsHC@){mKGcmbd=5e z?sk|mo5>ViW!$EQJrCnwSB&O!!}k$L%S*Aa2Qedo(ZVe3-y)s{;XeIcQG(l%fI_{z&<3< zy`;$^Y~+IoCMVTLnMD`mSWo54DK&0e?Q5(KKY|9qHmqp_ z`dH%(+8lw@9AH`#Hur9EJd~x96oTF?3<9~T9~26So$Lzl6KmRHF=Rf8aTpR5H>97r zA@~M;Y&?X<-;rpMr9FBw`fBD$@rmCxO;2$5yahrD5ZvUiY|u3>PnLdV(^=l0 z^a#hn*PmgK3ESscdG`ooyj(ow6{Ph-*r*3uCpmgmQoVW0)EP}#YGH5&+e%W9ZK}Mg zm2}dA?vryXOG&iI`^1z3>Y$$BC(y-@q}uB|2nAnQE{5>l3&{;qI{Oe*qQKBSo$A@Q zo)+C+H5b`vGXW_rbLK~W;=@l+_hvPhP}|`8r@L|f1s*Bf2H9}LJht#uQy`XboC>WE zY=mk@S!VvO{mwp5e_7%asR%)dMPiel9YJ>3W5vt&-GgpckbqmZ27AO%5}7wVvc{Qi zHD*PX1bID}@oK5T;)^wh4>syi28_}BURL<^E$gwy^xo5qDKZT8UV^LycBV&=8Pd?y z1-I0cJdvTELT#SWIzG2CAWw~i0G+gUoe#kd%ojoANpfvRi(gj4cC8fN^)O-;l#IEw zgiKQ3=T=(Rzm$g_$7^X5`$z{8MK;k>HxU{k|1g|-9gn84x^}O0Sfk&zH7~c>u(GBv z&;o5>ZrXq{rx4^Ri1;oY&f~?jzX~tnz#QnDgU{kE2>6{u)AmO8SVdM8obB zkcgd94=OO58m7%SiuZCOLf z=ke@&>IGrcP?r$5jYcZI1MVkkW~{WkF7=@aX0e@Vtpd13Yb6M zTcDC`V?kaH##S1U)!^yDhYF%=?}AlL{klo6>UsFxFS76A&1HgLZCH44ITvF`@ipcS z07_>E96~?>LryIGoxf4E6Ew-5chH%w_v4x2^jPDcs$~sgLo(8KB3VfAN5QcJonX*$Jo`z zK3u)uo*OM7?;FIc_yxl|YkwF0ifZUbDeIS~&&XV^3T@lEgn2gE_7MYf4Rp{tOZTQN zde^MFIJKRTberT(b|CND?PKDC1e|ftdU1}1KI`0RRdnXv=_>x;@AliX(myr+s3LM3 zZR($)mY*^MljR`3ao-Vs+|_C0E7*$L`t7Tn8ob|r`L=&>o~4;2TZ6iP#KEV~Utv9s zOQ&jR^`N%-8{c$d1!Mo^pu9b529QP2 zu5b-s7BX>_^+WbNKJ5ZhKMJWk zOLysgD|+Zo%oL#leN0=TLgjE#V0Ga)@u;l-1Cd)-Zt3|i5|bX;f!wo#8OSKv_uWS# z5?|^wcdQ}sdMfR)MGxvX(!x#l4sAubgmm$mS!F>_4jzEG%G>2ScG;D=_R!j4L4++M zA;&~H-&Lmu?RLV34MQUPi=oA{)&f#>x%GJhF=-ShpHXx;VJT(qcgb}aFEalx9d?yJ z*VZaVG`%|7-&kjL=M|1y=^IGplw?RW5+hm2fQ=NMj39C3>~-WqPx?KBuMReqkyP!4 zJii?C61M-QoA$=lFCR`~-T#rQ$pd9}a&U+jM?V+l20Bup^ga!fjO^DMduw%utlX|X z#`cTv8-Af_M{x}!ul4oFWVW=!YDHt9nUF~D;B`*D9g_%LZ)XT+&LrU9n6D5q} zl%IRoj?Na*bVWWhdh`fAHZl}h(|o0Ay$>Tu3Sm!*RoX~rFDa;oj@^TvUIcC>qbc`K zwKY&UI!u8_?TSnG{O?71Bfajv91^eRlLjZ9!*F}Y-Tlz7qE=QO_)7&s$^QsRX%xT%B(|VQnm~6U0ZngiAu!n~qP&?^4voLvA9-hNE1C)Y81*iSZgub< zyKYO`x2kbd2UN5~?6UP=3cd&eT3`NWgi-scFnCxC`=nr@z{WubL!BpZP{HuyGbLwn z>SbKLjmV3IAiF_G!R<#`P`GUSIgu!pqoWhnRO8N=4A>3h+yAa*bTjRJF%ZDtYFVl6 zPoxpl-))vz;>DL)!<3ZkBvQ$bVLWnSU{9AB9yh;GYBNg>hO+L)zm`rjtUD>>slPOd z51gQu*y2FeDVM2)So6p@2kxE4Dk13wLKgIo3=xLxP)VrcfiraCZbigxwk1=RH8&%p zH+#pDF4o?%jvD%d{X2MZq4+Hf0pT3wQS3a9pWf(4J@0qMAro0T#m2uIyzf0VJW5I&=LqjZT2$j3?2@ez-)j(drh_eAmo38{HrtTjvpN!So?>!uk= z1Vp=ggT%9eTf^#K9%#{NMjWf+Vw1D!JH zk^iSXikK~c2H}7w(xGIq)QvmZsHG!WHH5b;u>?)87`kXhC(p7W+d=va zl7P637C?$sedIv6Tal!MHM%sNL!6C(c#vnSbC|Vh$kgvA#S;cMD-BnSL+vv7q%PBo z@ZBaGn5ycJL(E~G>NShKT6w;;=ceK5f|gaJY=#P3*7+BqsC7+dtCf>Wvc%hYH~o<- zcwC*y*bKt0)Ye5#auz&%c`+}<=mlK)c;nhXA1-DU7pDY^F39$4b{ ze<>mO4`qAn+>G*)(#E#8ChbO7!&sz~@fCtNi*%-oWdk}Fg6JM`2o>u zwkT4AxGwQrj%2h~!fYaIz9nKP_}w{ITe&K1RQR$Cv@&&2j6j}6Uirg0tkARvoBwBQ z!L^A+UVM z)JeTB7-75)ug>!gQtuR)avm7*a}JV|a;5zRhZ4xe3V#!sl0kvN)g#zQWeRQZR>K`i1%5-E5$aW%%lod1zRn79nsKQe&CQgIc?V>*aC5%$GOWB==cH2H+@v@$ z*WXCi1=v_Q!X<21ro3LTFrjhRo+hVGDP}Y%%&#@5o#rU0*iQ{FU*RoCKQ#?gfXWAO znp!>ILwGAjuF2Q`H!dwn{9kf?n$y{*Gj-YRArEO+GJi#iNN3&o%#_H2d1bKZp?kYw z#T|7#C7Z1Z>NkUU=rtO5N}iTAmD=lIjJ`ah))&m=>`$<&`>Ccl(HCj6%skEiCdlI> z%B6Si)Zg3IdJfXLpxgzGd7)Yvb%MdRw)E4O2(g_pf;4l>C10rU?i-r z_%crf_k|tTXBA}aTlODB*-5s(r?0(e-hW-ayMkQ*mg7A95AXzMZ`k&lBcz125Xw&V zpQ1TE%%3aa)C^Hz0-P``&FyRh=~@>(FxbC@7`TJ7c3OlG0a54-`I@$dDZnOsg%o{e z8_iI29v)!g$v{Ge3dc`Oamv=wXMal}K9dd_>h`;CbZWfQ8DBp=$wrg*g&CL{gbvT8N?qBY<()z(Y))9~%l z-%kPI?=4K?iL0x1+XC0MHnd*CS%>al#h1bN1_*&=KbjY*Eu?QhNN#8emaKAFP8v7= z*`Eg#%a~EAuiszJt-RJ+KNeHic^Kdg5xC!;Q)Mzjt2Im;FrxyhX=Ib(>8MeO`w#lI zt@Av)oFg0;mzQVeh4J!H@SExyf2ily3hFd~T_gDaWtM;$w@x9ddLcDV3oeppqfJAk zlZhv|mXaunnm~Y)r8?gq;8?*;5}I0e^m%;++|!fc>Q~;r1ZHIAx(s%2I9nO$6+G!_ z;@EkdxiyeI&LNI3&7^niWHt@}GFVEeJNlyl6r~#H`Te)j-nHOVJJK08qxaU(9Sw=| z`rC%^*3`Ea7 zgT>N5pZa24P&-7krCNQ6PY}Bfm4fr*RXuE#=v}G;;u9L>;eRJ6v$Q=6F(Xj_ZBv_C zThE#;vLQ$?QF$IuEl^t-i1QiZWXVrX_i}kWN6(he?#wcCEM z`B>mfOZnZ=CS`9)+t<>@)e%og{46sZiey88Yq~sZCp=rpHVnB&eP!%4Eda%+&mwm_ zd;6_k*w|Cj)k`tQF=t{XQ5hmyP&Il-ra&Cs7vmK+hY!2G?AVW#vA4$2LVs(q7e@)k zE>+u=rf{Lrn*ICQm&d-|RG-d(=Zlwf}zXm9+xm!&J+tOV5+5{_&<-&RB=0!fU%@c;Z)Z_$(> z1SFLzi>2y|kqqogC|u3{;uOuIKag(4i7NmJaw+_l=Gti+G0U(a{gJA21?FVFb; z4{%OPP^*k+|L;5vwkBsaKoe80CU(NC*F1xHI1+w4gyR2~l(HPH*x95CAufgSdPK|@pLsVOvQ9F-b<&epwtF>za;{}|Z`FNz5$r5RK ztve_a{&!J77r+nO(y4Soe}&mj+rYGdDitDQo9R;l(D>87J5VlI?;0+*q_BK5yB@B( z5H7;t)%)$p;WH!6JH7Z2DJk9qTTp|hZ%ok9l~|v94q<%!&Oks?opaxMuI7AOjr?=JE{Q5WvCz16xc~ zouO&NhG4uv3j#_uF%nPr{0aG13@F3Jq0b`#p8LlZ^@E2b>}D=Nk7t&5)Rp7OiWgGW zdfekS411C5QFNK?t8B;M-%ZrsxgyZ=tpiBq8jm0w4k?4a&lT*u7f`2ZJZg2Bw+$3NBzW$G! z7>_`_{7z5b@T{hI#fLq_;#Kj!wIyYs<=6B{jL-8}o?MZT;)*IWjaP>I5$ z$Se7$7epClO;W=r4(=K#O#qyme+j2Ha(%c-!3Z9bFn&5+q~>9LeN^4&hCpu=>Z6oK zaVCEkcsWBqVsE5#bnDa*&d}h6e?JVrnGev;iqt`bg!*TU}N(_^~)`l9MmOJ(a*=C%P64+l~NL3N_T_f9Osl zy09gf)91NB+Q<&CEZ2gs#>`Y5<9S~q=;L)E({Jp4B%Mf-5a z-OFRPN$hCzq;FMZCrRGXSO%)Sg1^P|=|Iu|Q;_uuUxUMYb3o)1;C0J%3Zq)HW+8m5 zLJCTLcbL|r6EK68i5=9;9G3b)lw|g3YeSYy?J+d=mukTgQIOlp@wKBC1w%%p{s*i$ z2F~YIBwRHMGpog06#EkQa<=_rpC)0vm?2R^cgWdx_VId3vP!{Q0P?&q04s6hiWf%O z(ETZ7YbX0FfYt%S!iXEAdp0+x zxsEW$TRs7oBk+$~xb%5A)fLZ!@*-^83%5yD>)+8B3rK5O>6C35X?5qIjB8~PiwWYD zmNq&wuvqZ@!23NHmOR1bv$5~IqxYDYq%*M$(aS)@^vMBJMvQ?=jpqS;>IM7Egw@Yj z?9UaGeZDPkJ-Qak9mDFCTVu$r(?l#nYiQNFLi}1ch-L)Nlu8ae0|CrhFGtBciKci| z18UoMASNVRpbADKd-30fwtPjXqnobu+N5g2&%3D`1Tb3@TAGxyTDw4LwVjls0&;Hl zsjY3l8_ub@S$$jpZS85 zqCw+hv6Oxg?3bM@<(q5P!)2wNZ*%+#ikbcLlYFIF2_rpy|EVAQaxeOnXj|}{DT+x4d8Pcv6}?S@uJ*Ey0Dt z7Cj4;s>mi^SV$}Qi*dH=%np#9#ch$TDO3wmSTh%0D8jhOM`CKq@RV~V1>hf zqm#W}fNabVA_Q#}9Tl&h1ZdySad-eqDw2J1ls=#SiBN4K4HNGhA6Kp)aDGfC z^+_m z&4___(F6Iai!P5>w{?#JI|)`m`Xhx%brQ{aT-I;h0@T}V<;UymAexfbQ>%LI%kBVn zAbSqEE&3H79S9H|Bcw|>!rk&6zV7Hul2bIc(|UDS6@hYJ>h7)3FXNgsWz}JmSfEnr zL(hNUzpF?B)jXaV|I-H*Lq89X7&m?}7qbd~4aAG!K2}q1-tjU7Ztd6dJi}Tkzb)p3 zTR>qOs@Q*eMZ6S^UZ;207bk?UrIxQN%)iy+z2nX<%=Jd;a6y%YF5bPh7grAF)m{ub zy_mE;d~jpsga53&$+O^pxHA@sGzl0Y;nEA4N2zE(^F9c)VfgHEQeBzR*h`qHYz0@9 z8>|n1qR`J2>l6O%bY125D4i_TgCq3Hbvr1uU8rI^f95a(r4)F29`=2? zm`HSgbP3jpVJ!h;0nBU#(DA~|qsV<69;h$r zEl;6MI^C&joCZ~)rPn^8)B1ltfszTt6oM)+>*l{!-X8LqE?#Tn8f6PhciK&Tf(e_~ zK-}wh=XqBX4`_wE`qAc9teSb0FwLOvB*Yd3WV4PG} z0VsTO>S$(2d`%YRga*uelEvq^!Inn({Eq<}XZ|YCXa3bqthfFM+iOtF|KkF`l!e`X zPLRr5#z_hW-#4GnyK@Q8PwJNQS3vjLa6!|2e0*>uZAslvSSqG^ABjit>vs2w1HVUv zuBkkwsWZG^+r_6johWUa?x*;%)O$l@S1oSx4&?v0&P1n!_Isdob9-vMc)_rq>MR1C zo|Ydv`A&BDB*a#R;_UGcext@G?9{!W1@jox>g83J5cM&8jo^bq`Fs-d>Ey1E3vV_O zffntfQ6+>DhmVamxjD1f!tv#ZGAK$*d~zy3cJ<2XBPbXmM>Pp){--_|e_G>t1uNu) zyVzNqA8UsYdTph|5~OHUl+ap=!teJ-TM2g_Io+s-!}-+2%ZSt{i0uA;v< zT%%HYq}C4<{mJjyJm(w~w1nf1vE;5-lGCby;zj{Q2CrD)>HA#?UHl40C$1gt=6}FA z7zP?u)1Wo)>PGXJ2>9QL z$_$;{(ECCjBj_@Wew_RmLCp6@Rr;6RasMqINFfWOIE|D;Qfq79_I(XO2Icnx4{$QQ zn@ft-4I7X85x29rPz?fspB96N_Jr!VbxuRTu&=8t*V+eV7usxJxo)nL5fRmdjDd$7 z(8fjn$1Id*Hmg=vJR+Mn4tG<>Bpb#HiZz-qyf~Zj@q)3IHKp#$2~zJbC_ztt9`+sz zSeuE?%jxefI?cO5O2%{lju+bc{8X0Bi=auR+fVGv{-p-0@}v9*WlIhN+)&&iFmjyk zOlt*IHUfs72>NfSG+x{5Q;COdJnd9(R@|xIw3XB-TMW4Q&2Oe*ybxj{ix9mR8o3;1 z+vZjEk+O81@M7J456;?Pm~rJ9r^2NTnu4=brJn9YfiagrL2=Yg-PjnAD7Qjf^;gaF z&wxY;hV0b)51F!htj{-+X!vU~8+C-(>O7K=2xmZ{`jV=e@GZa5XX$U4et2nuT6lTkLQ!ZnTJMoGgA!bZ;|RNcsu+-I+HANF7%#gIZeI%(=4PG7;r z5#H5cf|zyLn0C*68#hCx)uO?$l=c7SSJBX{NJ=g@dU795uZ#ODa7E7PEG^4?&DC-A zB&eA>VYdC{EJjH@N}|}eZC-o=1*obSAp%@{^`EYtMDzbg)OE*G{eOQ=mCztmD6*2Q zB$<6sA!J?ShB8AgS(nU`Y_gM?nVos6$bu0FX)U^H|KPkqD-1b6OA^W9+%yzsYw_n z#uophV?9!0-lGcXp1+%xzseAuY)heDN#D@Qlcad7Aef(k4FF-+(XGIsPYt~wwI?1h zho0TWZ+G<1A`q}%;k;M)?V{YqOU*-DIzDG)zxk_P+e~CvM2T(fSSkCf&6J~3g4E6b zL4@C+?(Wz|h!lpq?Ppl*Yo&;;tVGJY1=@LYPIN@t@dd~%D6YKFx66YHPuT}M_4F7X zmMDtk0!|;WP6!4{Im39rP3pLX;%Z)VQ8{+Ax47t;w{s`lBe;w$1x)Pc`m$>7DXv>` z5Xr2je=&^5XY;o>R|GyON=~lz*(aD%JXXxZ{;3;#&yZIfMJ9No=gHp%TjrJ=Zppt~ zR|6~8aicaDn*1iie?Qw8uG@Q`=ssr#eBc*Lj@COwHp1SlE_m+CFH%j9H1E3r$|z^Y&Y8H$5Ep zf2wzguq5^s(e}x5u@|*n;k`)NsI+JiSJ)21h^g>tcN8c!%%Ov=+tp;e%N5#)EjK2AL^T(N*Ozt+G{X*ZTZA}j! zYLP(gzsXd!wwweBkTp$)B= zx%XMZ`C$8DEv}rn+jl40R~zg7>~t8m+_9~4SuO~g?-`mosJzOg4F|+M0!nqdoc!Hk z>nzeq8+{vWvX*;Nka$xO{9r=PccOL!aw4aSH8{vq7sZ#=ZzZ{yo+x`;IC{ThL+{YI zufO5WtIN?2b*FHbvfM2@Uu!$w49H`sR?JitgIo)llkK=1-6W^HL&f!=yiB_}tGjxb zCw&*jWixq3S+Z?_8d${FcKV~I62Q!`h1p$})z!kFUms(|a&;J&lmd;8;fA3;ZESTPWU=SCH#e_r<7TMJbY(&TtR2_5~IP?^1EeY-*V5 z&N|@MQ7V}1Vof8}9%mOgaZ`sEPrWDp2CmEtv{LabxR_pQ*>l{?KD0|W=%_uBXI@BK zmg2f^pYw_~ck4yf{0~7)ZRAp;PO5xc%7cf}c3g0~djc`28wccqfyM4CjH*6YEXA2$ z%Ijh(PObUl$Cr3bC%_lP$;yI05?@yUKNqR%6!i<1u^%BE4bbx0GPOPz;S)~Nnx3cd zMH+KY1uvebNid_6u=B#A?R>Sk>xS-fWkj>fTcEY+b@!iQR->Edtz5imk{Sx)_cd+_ zm$O-Ex#bL3I?Wq>sU}L0`kw?j4Jdh*sPgY#p{k3UpfcTBKC#wumVdUgqyD_UaFZ6A zpWfA;PJD~853w+n6dsNd)nQxzBz&kE$jXJv_wgq=?=TFvyqB=bfgjalqC+^yyHOSE6Cem~;49yYw0xyi0zz z>%)iDX`#@+1|p45yKBV1JvLn9E!dBtQVUV#3Pk$eXI=saX4O))nGh@2=r%2ya%o}K zgfZ{2Xr;i|3p9^xcbgM8oec8|@ew*Q4A)uW3kTrm!mCH6{D%0kVR_8^Rb$;|-_XZj zCOignM*ZYc!}4+4R3Dkuy0+vR!xyeHcjB!cztmKhH+xViYd!&{2u;9UQ%_Lp;^e7c?q!mgsqQRMCvT zlwn#mlWJqvh@#v6a;z90CodCaJtdD9mFISTC8oFO1~(VGYF&HoSMM9yE+r0&WqjO8 z_cYzL6)4N8eQF(UTH~uXWTG^-ls+*(%@!Am3qS$My&T|m#n;|ay)u4RgJ@g4RKu9} zC`Dg77h>ZdSsv{_g?D-RkbhD5-BfuS5vmE&LwSK>%(AU1?7qPQV^A+t6tFx zzql6--g%Ws+(C?Gv=h$Do2S|M_y0A{iRexV>%nbNajE*% zlbvI)6MlsLANY=L4FR=$X1DGd!Z1)co|=6#C)_15rOc=C4~Q|2z7e3_dDHEzAlPbQ zFqO9bPMcAuVYR}<;6}GZXy}2W!ly;K#{w{Yw8%=cPW~L2*NQmrhXd@BeT;~3#mvWQ z=&eDM>lWZFi3(Dwck$>=2XK6)U)bXKp-DlPdY(0AnKI5OX$X>$V}yA9A`}Fv#|Uh; zOO@F;`YjE3YqIcA-9eu`$HTX}s{K-oTzES8tkvElinaIjW~x^cKMcDfvF6 zgjA(NY}&??(@{`h8H}0F1Nzl7KcZjp%LSjTogQYSp1@)8?K_3lp-e7QLWdS-fkglhFt!voPXw4mJBO zIDJ%za%BBy*DWXPc~HJSk*vi(2~Ams-eqc2B@*7ZFTSS9zGfK4t=c2%;$180+U2?n z@QOf9zKanR{7H`!C6Ct*@metI6hJe2WxHt)ihh!E?}U@5pr1SFnW$Js;fMu`i9my+ zTQ6XHo~@cW{*h77sMK_*G+A(519vAY&U60fiEJzAlpby$yq^j5?d*Xfc+KY2qIo8c z>aNAq-E%H;wY*CQ6yB3M0Hg}c2E_zZv$HYYFzw}vRA5|%04Aw-m%&VAPfz|+#pGM7 zjS&$gE|{Bai+l)1J(bH&h1T1`wXC~KPYk=XBQjm<;pc6nb^FkB;p!M?!2NuP$IXz_i*fQ4xS9T zj^+X@ul1Sq|3%!wg|5%D%dSn3EYIQyPg%^ptp1Pk(}|$~`bH1_`!HG=Gg7W8Dn7hF z^SvL>Ul!%ozanu?Q%Byu>_O4y=To*=$TU}esKXMiwqcaj^$!QUy8hIT?~UBk$!uEt zvyjW066|ZZ<;%Cn#>GqSY-z&Fjkcm+kyEvt(dNcKUO+pV?Nl<{Na@*HeiHP&$UY%e zUIFTr7sYz>OCcU(C$K|{zU_6tI(z8~A_fPXUI4={j&Etvr$`PWq=93LxP`Ux=`5sd zbHMpo&eU105Rs|OffwhS`}OQ=E%Sa_kH(DUZKp-KO+HUd5g6(wF5A;4xT4SfLDPZ6 zD4O7{F{i96S?)ykR6YqiY_|MkRx(`U@VmO+#Oh@f@ht-v(G|Z8qwh zMco6F9!=uUijX3jwxQF;#tHl*Ue((<t<-`)BNJxHhO&z=n1-nR^2S+gfE-~X7@-+T(;quD*u{|C8GZZRM& z-|-vL>bcOQm5zIv`0Xu#pwzh4E0*XVGBqf_%4GGZngQ`FnHhp1)yo%W8|5vmtZ=kM zr%@_?*WqEZJ?hEw`~7FI;fabfB6V8a>BgZ#=ru;w`M(N-0}Hoo(sysZ4ZuYjm^PNA zs`Xfr%o~4=2BTKKidj_k6BLAgzjRL}=Nhl~8Qed8^_@2kKHR zH3VaLW3W8|DI_br(|P(rbCE@Sw5lq^=GT?Yhj7Q|m8sm7i$kf%r0u>%${8^xZNWJnN){}2t>U0%(08^!mmgxXAHvW@u4CbCF)U~@a@$#Qg2 zitRh1aD^t$1KA1g1hmF~lN&e~fJ)T+A?dPmXQckJ;%@| zBGZ{G5O`mdIFtjz{4?TvMgx;k#N$6z^2{_U1qqyI$2|=Gj=Q7sI{tjJGp}a_lB>9` zBT7xJz9I=@Ag*xrlhE2n06I|g?Z#U@$n_I!nZVzEL0h>+az4I;zc*vFJ{> zzf;i9)4UDiVws%#R49|7&8z!J!Iz4BrhYUx*UmH&%H19@#1Saq-VS4MG#}Vj8{AcV znt#s9LX)Y7vMqBzB2wnjiis>$0K$%OKEA3uh@y%scl#_&x`PFFd%a#(N$Qxne0zigUK!$wD#d8xrp-`ixi0sJ_T4x34OgxB|Nk*dbn~n%R zWR4+Qf7|4_mDFm0EI8HnYn&VBFdH z%9O3I%UOkHoY0dkygjoUTdtXzS4PSQg&3>5hD%5A<-#R*zHuJ%VVgjrbUttK4qm>{ z%~ru#O*a8YKb>p+>+yK+wjyENuo`!5I{bcE@iU`q{RhK9H_zpfG5N~4u}n&ucvw+W ze?L$QJdYC~Uc90?4%-9D7kJ_d<=dJ;`a%ho+l;4W^S&D*jR@l4Qqf6K|75KNtD=?T z=4M=G$O8&sb5{0Pb;B!|KR*j!cS6KqpzNwZwY1+yn4aGCyrBJB9@A~u$9Ek%&K+$= z8zi#8gPIg~(+g4NpqdpD51V-uToA9-H7rs6P`&Sg*O@|l&;U{GO z)Vi^3fkL;+5C{)b)uS|hsK}Y&FJCQC);=#-b51sTpO!x>T&?zXcVlnVb7$E0zCAAa zeuyqw{DksFycO8V-wX@j7ET=nU#=(~Y69xy`-J<^!6oSi?rTT&>#h2u$lgOGa*Uz$hldgoG9O9k=j&&o*b&3H^1@C zHRtDD>_dX7ie0lEqyrqF&U!IwOn)sLoRN_nHRjp(UZf2hkS+U&~xerCQKX6bFdwZm73Z_Ul2^Jh@g zxkA-9{G}wGXW4oR-`QqDFqouWe5jKjGZwr6pWBfcmdH1k;u8I}Ikw_fi`}g^Otysh z3XQsHCkH%aAIr*_8_UDsCw0sWH5$YzCCLYoj9Du~-+cz|8vLd(zgQE%Gaw&P!K^mx zhO!}C4X@B|$D&5ci)}1ajIWG)LyQtEN4Ko~HJjz(2d6I7qaA+Pf3t43{xLG6e4fmv z#TMB{H!|v+Cz#{wCKs)Fc$xFVYDpI&*f)G^Hc}15gmZg~UI%kQ*zySvI&l8}M9_^; zjoMlE_}rIxSjECnH3A1O50FjVAccYWOprWUg@#U$3y3wGcBE7Kr15UlV6^+4d)mmV z>L{-Up|?zzu*1xoTC)Ezsp6HMwHRuY0rX~s%eDT)LA%TXuwV?0T*rnv`%EiwFN}hj9*ta z5365T5f&;Xi5w@%o&BPdjp>`>&f<1Yl)3AQ04EvC|2D@(klSV(q|SvdQ%uh0|Jf2G zrZW2}mM)Y8nlzJduZTP=vGFHJxuofXHm0YZ-)OeywtoIn%r@JrpF z2qU)FlQ}CI+s{zS_a9V))<7wxb2zV^swaluA*lw~YQwMkCiirOZC}~nsgzv=6ithq ze%FRx)B{DuFCh?`3lXV%6NLy|>R)#3?9wl#f$6gFUJDS1%MTscZ> zC=h(xeBAe?b$(yYk1xetnNDzo61c#1sSj^f&gQka!DJ0L3IW?E?e{b(C#yt#DWSr+ zA5M{izAlo>N~=f;M{LfMj_v8#3N|})z@E5j4}HkA8)2i~&TzvGtWnaQ$D%1bv*2M) ze>xBf2Ll0JPn+>J3wXZ*tZh{Vs$d*W(V)YhppP)%(-zrdyi`$(lzZ>zi!eSxCF%s{- zn^0{XMU^%T$H(|$zrzft#!`GVx*nq5XMd~>je-}J+3P`4TH(}dT1mxoeQb~4+P7Z- zXR8RA=9S)u2E8hFT+{)O$L_MLE-wKR(%Titwm-Y0VfIzk3P9){ubN5NemPlYb{9s} z^N@%dsIdh4paMN6Aa-u>Q8)bZnpTmBcy&Sf+S$~8IS|?X&fY_|q!~hZOwPS6VjC%! z#EeT=eVtU^oEUDpoh4B!AhK$9dsgP1=v*b?ekmE0zydP$wC#aC79oK#I*C zu|O?!$0~DqGs3jXrUIM|h;oyd5J2z@mL>!G(bKppCFU!x^c;(#v-tsz3SFZI2fD>9A=^$er!GBbX3{zD`@c+OQ7qg z_V;n_L{=TJ@+wDRYUDDV9@|DI00I}|()67IY35gb58fBzt^rx!N+2bauRs4(MFU4N zk?=t!d;h~>ZLPprmNv8S4PM&5U zccx!9wrPkZ;E85EfG0}ncFevbVY_GvbNYt6c5(D8t#f~-dN%4P4Sso!?i1YN(@8^ON8;`7Hs!@#hlC>< zHSsGcwRMXL;s1}NBb@Qj{+U)E%=|H{nFw5l+U1DR7`K*-1B(Xg=i4Mx?DvC(A#l|?%=<4~L5 zt57AR%WtCNA$#d_EE0i^KfjZSIp!Pzmt86hsUp^Z2h3o&|oB1^9px z6eFdjxW$-_A5lFogcq~asB3FSYSZjy@Hg#H{kQrS1m5Cl<##IYzVZaCzv^t^us^FT z)~7x=SCR6KO{aoaO+F5w`=~2ITqEW$au$qRJN{29hO)~LmV{M+J5nEq?;ywTNzW=Q z$QHXZN)&onB^lcF3&X?7H4O7ErCz(B>Gj*fpysRGfL{GduD)-{GDWzmVzuZmly3B< za%|fuG3@#UKQ^Cft?hhHBG%B~=FX0|5%6vYa+E4w^i!@(UH_B#0Rtafy9G>$`;-CJ z6mlmSJ$XUPjZMe-+TGM#AdsjbebYg!c&qQT7q{;dk?R8EKX3HO!RfMBB#Kl#-!cA5POcNtykXdsp5qagJ5NjWCAF0I#BY6 z%cUCo;;7!uj7LiryxIR^SP^q&OoI66BmEf{eqzK?>%%X6Q*7wZh-+U7)Es={Rh$6G zqfX3A{FexSb4{uQx%b9w0TP7#F-)R4Mp&j5R93zMUPM2jlc_I0TXzO<_>H6G-90)y z>!^>|1ElZ+M|HT`-!mlNm;cZAUH`Qfp~@zQxOlXa^=z@}co+LxJbEPLdqE?d%5N}* zEzXczeq%0NHE{Xm?(&oBd`YJthugt+JQ5SKXtu`tJtM9qkjze=2qI>>RVTYA9{LFR za}c(ah*I{r)NjCld$hOX>7|qeJ9?9*X6IXb253SzGtq37v zCo9fuH;c!+=2sDRC=}JMMip;l|47}vm^u*k`AzAoRZzonPS6u`3M3iu(6IqGFtF05;XnsJ;*E-BEwzaW)U-K`+7@m$_Z?JPa3^ zL}#@mqG(}XwNetx9bec~$?FPx=S3Qu{#ctDjIYTYj{O<2P3GJ#B`#lV6eugp!`%Ve zjPt?o#A(-|rymh>Y4Xg!3M}TKzz|Yl-1sfKyB}(Bmp&upu-iot8&AHAQQUf$1nrC- z3r+Hp@WJ5NsZ=F(Uvw!Ej<`E_url~9Hp@vEoU z^da($z-e!{`t3CiVfeAFBcR}WX`H=y$u4zM^K+K%oSz~D)3bn;UI8aS=5l=2diliC z76|7QB0|}!p+MKzx}o{PzC%`NVfoF7c)uN$Q5blIZn5k|=N@)NfL7WlhY zR?6Cg>J^||BXh+PWy+OPbMFlh&{^}s}oCxzpcEXt`;iVd~jEV zr=IUsF8vjY1HrI>{jnoIKz2NZENFTd-bw9tyXO=B8R0*P%m9eBRz$nYXxND;v%F;! zL83l%6Hk`4Fh+u1jE`_O#fK`%;=ZP=qE2%3j1<1)Dud8}X{QCgAe{CeSd+c`cNm$j zlMOW-I?mT4LalaSC&(r!FIZjPlD6mw#NMd3$ul!I5I8NuJ&s&wUR~63)F|}?P9d+j zqPdbfRV}|}i>eL+lwpo5H5U(zFM!^8k(6Z+kHg{LaYO+$=LTdWPJa3-QBaP7^*<2$ z{>8%8L)?fw3qRP{hw@ennqQ(vafybprbVg}Z7!3SaIN27f%|&x-T>W;4_$DI5}O1y z-IVa1N&qOPjo$<{Qw$EVM=@w^?y;A+3s^AIhA9p5(iu@=7W#`EQi`oGc{|@k#P@1L z4^8lfpz*g30@AJUCxB0DIQ#|rn&pHl>&@4lvfq0=i;!gA%6PLk)DpayY!A$Z=1h5J4+*cFA8@J^SV**8!X`d`cd@zns7WA^%vf^P`4J|jPV!i@Q z1!`%Vg>wgpE89L`UlZey`^v&kc}ZdY!~hA*YW?jDvCNB6_(~3+7V+g@O>kXGtI#Tyd{?XHBxTkmAgnuB8}^#YV2W8;;YVAp(i zV0916ot(^EK{pO~+YqRWQue;fV+PB$qnQ77_c41OgZC10kL@X6mSnlIoxz#0jygrg zc@AkwU7n7h9I76D45)`yFtf@eR;5`z;-0;NgkJsgRxSSKbbwH<-mksOBZUnAYUDGQ z&ww=gm*B1!ZR*a4Z%OnY8N5)85YwE#QY&{B~aj)3hi|UK*1|wqZfw zRYD59P6vD@b>>(m_3yeM5)=3AtlJ~QG~*=gt8kYvmyY{s#%CgJT@kOz2`8Ec7MkWk zy`MN+vEKl*ye7A{ql4!sWjWT?n9rTK{#_&9Gsde@N1LuFsz%4{RzRoY<6w zQOmin^6{RvcWSlsOTO&g!}aLRLE-2$_9@Y`qA#k4qro^CTkIKWx*e2Yrr?G7`A#ADcHs5M9vEkU9<7-Oue@w+5 zrn}7+afi2n#+r$W^JKp?jdNP^8rpKCU7>Ej7WXoiT**&3hY`DqR@<5q)ph?;HG@c~ zZrC6vHVlAxJN%qpE~!le%_ENo1trGpq4vH&m1QpChjD0*{nqq7+uwDbF!&Q!i^1TN zsntA5R4d@%=v9#WgkPLxjecc9DEg%hpekQvEC`j3m5b(xY>=;f3U8%qJ&&cZIG7hb z%^UT0>4Vj!d0+t&;J~hON2P`JaJ!fi_eIY@(#aMlZ0nFG@EQlj>JhF=;qu^1Xs0|} zu2I$P*J0iG9vH!(Wj`HKakpLLo8vf zhEi1g$oqhg7$h+rdyXx#^-&z>ne!+zkMK-{&;PhU!wxx-bXz~ce;voDX% zkHe}lB)8c@f@P_3eneM$IV$Q#q@9~S%gzOrMfJ&tnPgun&gQhZLNLK+cgdGsjxu{s zGG=1el;g~kgzL~NuV$$@zM#^)q_9#TOID@uBsmmjr5I2*E?Vdl82NfiYx>cJl z1MMdNJ5ZSG&>me4efg)}{g#dP?#@#6>+j!7@4ceRv|8pQvIQmtkv33l|K*>Zj}MDB z+IULx-FPMvWKLJv8DicecHLJamsyKcHGM**UBQ$em#$dg3gq`&ve6svSVeo+YELrs z^~hZ37#Ws9&XC^#Gye=1H~kDYQWYZjKG+Vb<9_b)-q`qMRX1qv83a4ze=DN|&JjWC z+W427CB7!MbrvCOe0ZVoY))4%(M@)=BH2qRCc1n{hPE~oBQ=V!QZlM3D&&?$)}4m1 z8+cY!{hurgN160Y1^Jn-HgN$XiOAs(sYcU@#M-{Ps>Cu3u}230O}6H;tw9>U1F1)f z%$b>_rx*PAo61pfcKcNgVsRzX!xe%Q9CjE-q?nb8 zRPHyIf%5rpeiNZra=J-qz4B!Y8M`oZEu-w+)1`7xdHLqOex46^a zCPK%&sJEVhX280DVOymj&E|Rq+W`QiJ@kA?y-J%W*ehovY@yN?-R`~+9#{qd9a9JF zpO^S5ZaO8cA;|euR4nD110#U&=c0JOrT2B5`cj=~Fy~Y&TG-4$mdxCpS!J^<0 zu~vtcFo(rOZ8Dyj4*O!Pi}U4O+WEvO0*WAhU2x$INx0JB-&1-UK)!vgcG|8AgNX2! zXD)~dt6m{FA?&i#UPFBb5+#7iBuyd-3J&5M zeLGzzmozPj!aKl~oVreV{Q0Wo)Zn+*Bz|`vvZJ?WdBj3KJF%Vp!x(iNLf#o7RfPuI z4t5qh{GFK3yzsoSG}S&RX1rGe)L;$w1Gc;SiOJ=$(@PuXYLWaQhI( zpY5{GB!1GnG|$^oFWFcU`I?3?>y<v9F86}82Am|Z)6HFSkK zdk>yQ@Kuia$1sykQ<20zA*Ayj)_Gz0;Z5M~GikFVe};=gWU8KRu`_rKE;Hog;Yt0t zFWF=0)!MwncO-SANs>_*i~(iR@cGyAeCNpBmq#j%=li-}@% zjcv?11vPZ})c!^|Wnm?k4^dIiyVk?c(&XlNTJNZ)Zm!n9q}Ut2({%4uh@UIyEY^#I zRr9Lqi)Sfv+oyrUj-@SogreX71ZY`e-z1sMB@#aK{Rd+Il2TN6i6zBRx14;Ha`xj_ z;iIL@*IvG>&x$Y_!+$;RjSkq}Z;RkP7zVkftM`dUmLBBhHRDZFhR>evcokMp4pUV? zo9ceji~`LsZQybM6t(6GBrTxLYGVSL4VI2L7qaR5$J)(nri{$BsmS{h1A`&^vbsL_ zU$ZY&#q(X>WM1BLtnngyI&>W&K3npv5Og}Ii(q(M+oS}&WWDGgE{C!B0 zYnpRnUq_u6$`wcv%AVF$>ciBP$bg?gQ991iCAxOy>S#s?I zAuqz(c3+8QM!dW8OG0j$RUoGmLQ51H{vF$;jEA0|5CWYg*QsjuRSMR8C_9ozVLvX` zf+9;U+&uNZFQN8}SdHDz&C05qv>g4|`0-Agb^`MZ(XrMP$N1#Ec$v z#>LN+s3ycooq+Nh9KMrfqRTo+!u&}nxUq9MG7i{fu!wf>Je-T%pvuFq6cHE5+djtr zs?)J`c+5~S3_pkOSTU4Sn46K`XsnN5vki`@iNX8{iI%FNAq{+HpZ-W>;Y*uCaiue$X>Y}k z2{S6NQb!_hh^=oPd!!x~vgylx4{U3(2St{nFk-ZPhHb{GSH6=bpWIi~$|fD9lj&l2 zQY5L_Z%^0PwO6NjPlFtNJkNL#LoxlX}$nLZlUNa+-F7k(-_zIcDFBfT)fZL5d>CXgr8*eD)RA6{d6RlZmh7 z%oaWxY#t<~JPDWgiLln?9s5A*^#g6mo!z3gY`>dfy=B?bqUPx^pCLrSfv@}<{Cb75 zg_~F|S=ES0Z=Q;vkmcpto%jNO7?j=BPsBxPr@fMrApT9GyB&3zU*_Qc1MRhKy zd29ZRgQ2HSr;i2}lA@~}cAg*&bKa>#+pTn}Ch~bnqe6Q5TQe6kyGw~oPOT(;5z_5P z6?}W{MPKY~H2;t-=y71CE&{dtNTOrVBJhv1Ei`@SeLf|2)NDbvMo)sOm!C^qryf>W zj)H4QiWTLC1}WK1EQ?YcJ^e{MXNvcO9+uwHIvq&NhPi#zj+;M6M@#>nMLEdOliPmjYbR~bFUkKh$twm_85s1Q|J?^lz{ zqd32PQb~=>>mwo_x%C{(l1i_RW z52la8NU2E@2}{QJcO+)9KI>D?NC`F)_<*pcxN0lkpWlLH%)^WMRj#Nd)juiwd67Bm zCseZ{r_FM=+%k5we|(>jCT`_g`J~3khwRY`$+5YdAtvL2bx7+6;}^lZS8;9EwN1yW ztvnZX25C^G&m`CXE7bgbVmke(zgc4-eM+P7I%-NG4#zZ+@C(_dsZZ~?T$$TQUO6v% zHct_BOT49RQCya82DcC#WrGO>>YMu;e_Jg`7lSzOk${V=l3!7AvPd2H%;3+OSHC}i z9cJ2S6y#kl%;VIc<%IG;H2gwWA+1^jgQ?$&+QZF^IM$>l(A(AI8^OOk(}pUqw@h`O z5q|i%Ub3Qdv?TDi^U_1wT+7n&x3Zh%>s8BR{tyg>{JT6#D>-8<#jG3Pt)@MamrQHU%ch;?_4Fu-`d;@OW(iw>k+}fZ ziMh2Xekpo<#?*4Rl!<7SPjBkG5F=?a8Xn@+rFT{3Bk`yM!6?3lv$vWTW@l122ZT7v zHB@~ce~!^*RnC5Bpxk~ZS&|wz9<@g9g)2TfWMy? z=^_9coBkeA9a59W!8U-A8`ns^X7AWORC96xZnpBQ#U1VhIbI*|O0QSDB)BGClfA@} zI&Bl8YxO2LwucAFgkWLVjUz=BjmJh07M14@xH6Ll?m3IBF684!{QyF)Yv<#aO5CTM z%yKV8MS&16^Dij)VIzsTC5(sp!K`PNYCdRuCT`DPPf4lFG5 zvA4l5uY5OdFj3YI$*zjZs-FQA$RPBj5T1Op7v#oEblMdQ4I0EKgftYz{|_~=VtGC) z*Ev{qM=-T~Pr82aT1Y|xsFfiXh`~X&OYV5;tSn|Ay5mbX zu@4_dUnwz}^L=U{A6GF`0ayowUJ*n|o_N*pD4t9#J&q^ASFg2PEuKZF(lTo#DFvTi z?5uY75N`7mV>xmjOrRN~t8S_P&xn(w@BPDgC=z%USWb8yhM!n=C$-X%9Rmy*ne6=@ zJbSn@=9t+COgX~o(4oXpUn}m)wqkiFVGC9)Jx}5NXYTk zjB`zfGJ%e(!+`w{X^7pM#MlT@=UO0AIrHUY45{tj&m9aep!ykaeBCydFO0mQu>hxQD5Bm z!={}j2%l|Gu_yPVTtdUr*AthhpsZM@>Og(c~GGilQ4#vQM<}MLi3@Lu~WRM z0urkeZ9JShV{Ev{#MD!@$#QV;8a~>YAq_<0#^8B$Y58YjWtMwWD4VSweSx)`ZO+2> zOpeYXJ3aaS553AaQ*C(ObgIdrd)hTlFjI8yQhQ$%SeJ_B&A?>d@qY2cu70A`1UHZ_ z7T!%Mw3-uKuf%6|b(Q7a<7G}LCvR$f9`L5~x!6iqCpR5Nnelf%!|r(IOktB=$m03P zFs+NLIR8={wXi+Ub$Jx9l$Ey^{}RvawI8$}u^Hav(v(&3b^m?*VEi{Q*f7HXgUfzx zTM3T1tbi&O`_=Netae-u;dI{fxDJDW$$~no@g3yfCflV9Lof`JKj|5h7f&cYAiCed za;on5Z)bg9n?+OV`O0rnHeNckvBnJy(uj-8F46~eaz!n_1z@<&s@_{DKtQbY!H^Hd}+cqdqM$E0MtpPsOO0%iOpwX(N{OfCC-H&%2s<`ruqava@ zGn^gO^Y?%sl-4AW^affe=>dkPYC~b?2 zyqyJ@Mr}iEz`f67*G{`dm-2jZ3;~067gRHLzl|Psly=(50w?l<+~A_sBJMlk#}Tkh zb&_^VB^^IEoMc}S5209Ih@|4fEo|>jlS^BibQUSIBzjm__X*`RNz2C&ZqZyGN ztc(xwkl4Rdk!WCdzwdPiH9`&1Kd3VV*qxH2E`* zFL{GYKP;>@M&WAdVCPyD83JOd|9Syj^%SJ125wX5JqN>QG+!E*cw{$_DytmaTn`Mf z?>k8+BOcVi2~$ji-{a!ER`%A`Y97pJn=Z2d^r@qtiXqo@a7p9z`}X+DR330V6j1X( z=6lL>>99%b220tHDw{;-@toTmDM4Vs=UpVp$`et$OC(QHe3tT1gklwLiZC$^N5rVY zpLmVYNxN9(!}ZaxyOU1+XqWz(X+i|4&$Gj>PI}IYK)7lbg}r-#L1u|Mq!RJs+dUFq zd@G2A4Y`^71X5E@JcjZ*!km^Gq^+bjnbaQzE%JX^D(Ly3T+^dyOg#PAY$rT8eXb1X zqkF8kn-Emln9};N>tHmuc+#fTnv$w>g00$2Vi^lCWw#sVOn$7M_b#zIYt47xV9S6$ zCvRCF_1<}rrm2~vb7tBx#*;!|z;`izFAqIG<_1|gtFjVFd4Z#d;dSqKbj9v_!MV?e z;ikhlBW)Kw0MY$x4sP@5Pp$c223sl}^|JfA>(yT$d#(1By&3es4!*j>rK~Iu4rkX+ z%=&QY?0r2Kt3~jRfJZ9{f;{95pWyJ(ownwtMotcpi^Ic(&_dHlL=4q_KN%H!Z9GfE zxFQ@@R#G~<5z@~e5bPBN6U}!rQPXEylO&Ei$}C;^%UyPK;C6=s(X;YpFdR9;uN|!x z4qe~1+vE2S&sy{31WMkhjjpNt{P>dLJOw$fX5&v-`iLrlPbo47y-IQEeGyrS-ZFpprwVxTB@2d25d=Xs#b_j9W9xc3Xq0B9LquI@bj9iXp26*bH z<4Cl9Z?L#iX8E-|w5=)+1+KdQV_a>O9-zgfIY5wb>>;YdQ81Bqf#kfJIzu-_mD>w3 z0hiX%Z3Qc%oq{p(^i3sJNP0d7mkg$2-ubRFEPH z(wl(vPUumoQf+hyg9u2M4xxk&5}Nc9O280mLTCX3;kyBU-~aHw@3@vOvlcwL_uO;# z*=L`7@6(R(>mJ%H^uA9+nBYbB6x4t+m6kVbkSeNL% zyAH3C57b;?iS0_fZ{@R-gNt21A$K{~PRAMBJ-$k0S0<#2Av5Ab_tyF_x#au4b(jGy zp*i!52M?ps>%p^!D{4_3{YAO47Kc9@swF-9Od9iV_s?7i+HL4z7Bz0$OH2(bHZ|Jb zW2hk%m`#q3AtcCx|Z!`Gx#SYQ+HLUmQTcwmiJnwoETyU?LGsr=Ors&>;V zL_E_|;o)u-K1VQct9hcCevn883?6#jT`f@j@W)(!6+Ud>BOL6`u)vCVv$x_Ycl{0gXWCQ<1_&rrp1PS9;9Vx@V7=FP^y7 z<8vs-QMUG&_eJHf0%sMQO1=9RX`k<1>Mx$qV%sjhCfd?b3S0N2A`l2sMeafZPt#-XJ zUv(`+>B{pTyK#By02A|1X|Or_cYraf9wI*WDiQs(275&yyY%zB^lx?Rehqm}CL~|u z2wMGn>^meI*j&iC+R4IW#?!iMlNBIWR~wU}hwE(DszSpBzzz3Kt?{+yBzWv>7G!+s zu%1<>8)@cesN*lG)ch54V02I?+xQV4N=VFr=oW27m(LCxJNVi9%E);!s{L$aveJj@ zPpLhj>=}3(e}XW7pBDmbPCDtUL%XnwRUtYAA zvM79Bu3w&%vbk*s@I_f{Mpo7^4PocinRAi`p4ptlnf3)=JlEqWBk(%i@V(B=(%r*N zy{uc!_pGP6E!VCXPY|kAb=oKv1cZ}k_EWEq3v7&BQQxCWbp2;HoNeZgsOL`D=iJ-} zlvg}xd!2GlUDP1`lvb~D>#8iaDlX>6B1iMRY^Uf8L z)x(_n{lU#LAu#2eVH=1Qx9vD7uU^F65ZEx~RTV>TPA1;OT=@c=`W&$*%nLdMsQMa| z2M0fzx)?cs%(M4nbT!IBlSiyDnOk;joxB4U4P-yt9A|VyOb6Snf)6$U!gBzjkg#7l zU$k9tb&3%EPNKeGlnKV8Z1X$yGnUraccFk})0r3*n)!)_><+7Hyvj~=Nx785L+qm`jSfheAB>E zSdqf5by^G474Dl&`qs;Nd@B;LO=BMU+cf#kd>u87G^B&y>saWo3~TV@8pE@k zY}A2dmxYk1^Pt1qH#viV__94b%#_U6ziG%chyF7^?`4<*|%9^9tS{_?j&2FU^ zX|eksU`w){(O-vEA_806o2#90rr(sjD1x_A|4HsQ|8SD~M`>r+%O|2!(tkRlm-9Tn{+gqBX6W*DhXETD_!C-0|MgulIR}^ z!?d12sn3g3M}I3c{jDIYwvw=AK!bP>KQE2jZDSe@r6QV*Kne5yx}Bn>C%jgXR5)hc zTkelU?gD75bcti$8Jn2~7nS^~tv1+B*~mrNoyZKqB(bdJgw5?g@{C?B@P%=yMmX%o zvG9yn!55mD7PEyd-v}91uhyp9!Wuvlc?6QkWUyFRHDl0L!Rl!~0AM+{aZ2-Ul(|k< zh4$XHK^ek?HK)&UN*Q%81hx{Iyi)5BB9Lnou+DNk`cxV=@%HHOZYrxSo zYZZM+v4<`etb^-szs4tx8RT-hYn0N+7LERw|92G~V@X#$ERIT>Y%Es$I*_g+|^Q&<9s%DB3(5i z(RtFp^HwCR{JzRdt?Aa^VcEidZMuKh3>w`;^Oz5I~l zpNq)Rb>Dl~WLq;m7}*dk!S|QZ7VEDw-_KB@U7Z4kO$3Y6VckpEUH5ZOz~UcIpXDsQ z*0wRL!r4%W@FyQX;!=@4DQxwSGl?TDTYw7nG+t35bdZa8k56590eKn$=T0(wmEv*< zatQE*_+zgkxT_Erc_&Tz9A{l=oF%VH-4r2ldN%7-3CyVZs1X z#NLX9#NlS8{0NjPV``O>it~SsDh-qdlybnzWrbl*tnJ%%8+7;k`6dbATd0q+;T6Bi zdrjjEYc$wG#aYh7!h5*i6zOZGeq!ujNs`UBx3VLuJ5DA~?vWcGMYj zVtE_dSBYtgwjFPwx-f5m=}Gcz$^m0dT;RpfV`+*XWWnk1S8sorI+n#0zTUQ&{1_-O zXM&RVDzCLkr0xdI(W+``T>vu2J&kke9J}?!ytEwM6f<%LvpGjjbghst-#U^GsT!65 zM$))?3R!OARyM5;-EjtBP#JmQY)Y5P=L3%brg$vrX-!_c8Cp*q4g0GIIrfv+=si5t zim5fuQgCK?b~;F%QeK`J6g{CsY-$EwW*P9aK3!D2VXJCttE-YlxbaW2Vc)GOBs|?C z5t@HE8e~BUV=6JQd!1n&P&PpxO^OfwEqjVH3oi%<3b|4*EDTl{l144ZjYIaxY`cLk zzd8KSUg5l%K~WT(?9z9LnB{UnZ$4SpEdMQ5Z;$N$SnZ?04+it=F<*Gw5@QsuGd!42 z73=T{R)GXmEy*~Dd(nLyI;lq-llQYXx0eBg@>hpH$4`!r5aLg2Zw>BWPxuQ7`zU)h zC;2_of^B5s636o&gmsnbzDGuFUnD36pH08h75_VFVyNNrYWW$lZ^UqYj(QMR8$733 zEjkzw38R!vo9nbmY{WJ;MQQnfjr?CcO(Vz`Uqla50z6MXP!Cw-L4$npw36&#x!1@; ze>zj(d3)BUNYrspD3uVhnRVNX`YDP_tb>T>5F-0ct;A9T9P$!OCVde=#$j`Da}U}| zMy!_D*X&8AE0l{!Z%X}I-goV5`pOCLbNqH|f9PZ|mIh&(x<8WhC{<}hp7nB=LP>Ag zM%XYHXH{L)wR+!w>(*UPn3$-dRv0^Vp#3~+IGi0=UA>NW3TCZZKVb?@!>1*PD0f+*jMq60 zi|ZJ|#I+D2V__1pa#-xC3Sh;v#zFEDUes}dVH=p|aRtG9O2R%#1ivb}^S`we z455A^)x>-Osnp1l7l z4l=+i`(NQfEf!@|KKF;)244`nWsPEV2%~eo>r97A_2dwX&O`N4=i?DWl?4G^$c6`Z zG;eV&eog^D<@MtdjxiHp92m{zQyE8;A(BmUWDPyX+P+b=ndlI{(99tP5gE^r4PPl3 zVb_Zb{DPa~-C3Y>dAgd#>r_;cPj(iUONOQxeINE`x6c9UNS+ z7%1gMH`MuIClxzdW&mT5y3)3{|HJR-!{dU*+4r)P^kaXTG1#c5^dRyzj;%1Bgj}tM z)91doZ4|M!c#-a}O2r<@y_XGNzQtJHCq%JH(qiLM3N&t?GOVD5L?Szr?p3*dc$ zo*%vDsFpfl-T=>YlxiEY)mg^Ic-?T zH5$ifS+5uy5=U=3UbQe#v|FLN!B@tl#X*X9^Dv_;Er}qF(tCrh1O-CR9RF`_a8fTV zQP>DUI^so=?|9L2hLV({GxAvp2hH*8v*3r-Zu$qTyorP>b-0eaKIlp*D$tz#ND0?7 z=^SPMPOl}_3+{jlj4vq7t-g?R0N}qHq5m5R~!H~pU=hskMeLBLWouuh+c@8s9 zc~*1MQ!rWcNYX>VN*aG0R8m}%4J1GX@R0xYL2hMCYq`;UW7G0gAggzE&ZMyy&NKAuD=QRUc|J{M zRMq)MD+DZ-dH$*FmGrk&NwzWEg2-dbTUT0Nn2pOEystu69m3>N7#^6xwPq;(NtWj3 zV8;koU4qm2mJ)JkR%80kTdru{fHF3Ej^FirNjMdV9?M(GmFaWir)JmqM!xKnVF2h& zhP6RWF{OyrzP-aVFBQq8m>$Ory(>xci0ik9mkHN-`|0L`?x9$ey`AXVD~|*wAvwh$ z+FmVuKY{J5ugS%}(8@v=eU+{)#^17}V|yP&pfi*O6)&w2%I~{wxP-Mf$4Ry?!rq1M z=Na96&8D*FzQ0-tTz{YEq(eiwPW=_(H0a-Tb>Maxy`9I)r5Mp+o9_2lxN|-Q^R|h! zCmpRfAGID^A^*B{Ebhgmai7B3%|@%SLcFxlc=%tL)4XAEVedDZ$+TD}dCvkNC!jo> ziT>rRA;0VOkUX(r3UcL4lU}Ll{iCE_f$zRX?0!;%16{4QJ;j|RS85Xc*WEi(TDPf+a=YY z3f>QuA1)T@&q#v5{i9SNY$ML#0@+QPb8r5nm&EXff3j@zrx0>tk{mHLliOW!3T3+==kF3-&ic z_C&-&pnR4!`?z?nSswpvs};T^2FYUF__H7cY8@|`nKUD)? z*xg(>pB^Zek&N+$u#_bxjP?HU@c_AyDi&4}*D)^ufmAczbfkpO?O$cXpW_wwWG+l6 zQclXzUzwXjgw8;6x2@(%UY$?Z_@%j^=iFvdnR^}02TNl-+;`6DHvJ9hJQh?kt^Mk< zx{#{86{W&$`qcDT)3@hJE&?-d#fuk?E4?>^NHDt!Oy6iFO(8g);lG5VJKyGi)^*2? zfm(gFi1!87me=3Jx8a7@n0UET*hYx~&GD0v`HXO4Owg}DV|I0<09x*0P0(5w=-YZv zy1s|=y+>6Uk?3~#Ii>dD@fDovP$5M$gAMv)uFs}9D z4{OJ6QW8+-sdU%GyuI-;M7Y=AU&L(ipaUEB4snOpV$BP>u=7vWvi;h!3l)Y?bgj(= zt9oQar8K#Fbdn^)&o1QF5KyH$QxG1-puP_?y}gy45*ecGEro`r4ubO!O#3E&!}kWqOTR$2I2=4|8?V|-Aj_FRCzH6>=BwMA zNid48ZWj!YVnJ7-sjeR%L}@`&Gj1>>aZ)rXJ|zbEO|u_&Ehyc=|FtE)Mue-e9|x{= z`MWqhAA}zo81$Gjm2cD;&}a$Z$1dK|yh#ja!$lj^Hyqk-2m5`6iYV3K>yB7O8l}7_ zS09{l-9Dx9{9i%+QE|A34Uem?*xAhx&Go{$gtG++AY1Ns`)e@{p?C7d)@Te4mBd2G zf?;*a@29yPu2(a9XOv}Am^y_3o{?@meOS(G3Ze-pHEwKN0R7#B!ih>jWK?drpb@u9 zm3+vagF%;U>#*MkRP*NeMU@)4C<1O1(K#&^7Ng2?z{AGA0^LIpMGSQ`9RH#`R+RVG zV>81uVt1k3ia|CH@r#S1b;}j+q|!h?bNHPttx69|ywVDd!AnN59ucLfmc7`;``uP9 z`$Tx|q5ry2Kcu7dRzqmbUAgVHYA#il>P~dP)-)?*E#T*^hF*?N%6a-T=66oB-kj~J z!GpFp2gls`{6ow|cys(yd*SUDrFLs2{gK)s zOi@V~t2k?cZjM$=KXJrrTpz~M)!pw$1M2!LA_Lxmee%c9Zj%MI+0sK7%2zfaPVsy$ z0<^oW1S&GaPb~TB;b~n$`H?fP9-UJ3lMnx1JT=D2hp)e#(4mw6pW*-2mZg09?^jN0 z{u9IeXR-NH2Lb>7=P)N*wodHve|)@jmk_1?w-?|)9QdD){~y1s|1IVJ9ZUJ@7&()E zro(H5cuo=9_n=HjuHa*mU}#diqs47DWlp2P8wujvZ__*b7D5|dGhKXn`%}@U7J=!P zz5|Nf>i=ekK1uL<|01>2vGX@k=0|*gF)RPeRK;}eVVchm6r{uZ(R1bbe|F#BpH1XU z_b9osnw$J1x%Td{46E$_|0Y3T#{WM2X9G$!{pSb&+2B7vpjhDlML$TwB#4+bnv+XA zg=4L~+yv*zEZyIv+k7+xTR(6Ar55^Jn2uMN&U(cvjdil0J|Rg}k^O=s!rzb{mhMq_-o5RRpSBX^F85Frgz%Zo8O_Jvthm$>XVpck`G? ziV;5Q6+Bz4dqFXHlEdj+4x8xo3P$ot`3;C?ZKDm!(YdT*FwSmQ`-r^IJK$d8&+Y25 z2k|b^h~e2g5ro9XMZGD+GEW3z+l#t-K_oyp>{_58oS#&E5YFyzzw?79$D%G!2}9Pr z(h9K>2&lV^bP0FIL#KV#EA`h=wI29&6G}(rjH}wo>p|z<6&csM>!d+@{umjTExjCh zDM}m0W@*f+?CUv>_)|yT9t!P>lem^i3g#Dx+8}BQ&<)Qf1V2 z=ReouBf5v@$(P>-C3K8clJ0y`daQ&^sR(>bhhaqKjWXuUH{xfuKV}4IS)3Q;5aQvK zhgT17y?P!-ozqL*!IqF6WH&M7PZWRX&6Om~c!0T9om|kciOQp%BG-^Ev92F4kVBVQ zgiOmy^mvZEC;hFjv`7ujlvElPl`#7wOEffNp>i=s`|}h%OLXdVW~`{Pi#nAT6B4YS zvs=~pzhu4h36sa$@e#?Sxoqgu^O?%bgj2A8FWx#LNT)uAPy)}M%p#dXx19o^hx>C) zdVdMl^85dIc1E2y%+W2v=Y`Tct-UN(^Doh}fP*E~3K^`k~z z!k^1m2qMFC4A#y@@hjwp8a;NYP9pr}0x==UbZ4)7Bk?K1SxkP)OC0tnac!P``^jUM zWyR%9wuQtN90y|DQL^NF5-XU*t1B85d9aD|SwW2OA0v6But#pQ+8x%sVWb3J!6=1> z+-Hp3BIK3{f^(pKeKL6M8f5b|3y&Mp{vVQnhM#3!&HRC3;LpfyIkWfP(AsBJjZfF; z&Fm$|ptoEc|J5xrl<;#DL+pJ0%%}IIH=M{Q$(@k%Wx*xyY%C&x{|VU${$epVj`4A8 zIGNZ!JjC()O4jjZ$RH(Qc5Jm6x@j$_mc4zI3`@-e^s8H{jJJ4_^uO-Vm& zb4yp3;4vjx+;9hCYQCE{!k(L;kjbJ#_*&tsS6LnS=kPohE0<3Daor+8xMgTCe8@6d z3k5aPTTag(Q9BmuU6@kltHz~FHXIG5N*+l=G{Ml$#LAz%q?OACGcfQyejw210usa9 zahDX%zTQ$2S}ok99Yt7eTNpgAs~&9VFHvisZE{2EAh@C-OAyIrod)0Ta46ny=}VU* zE^7YSBUowZWU*v0kP%j(D<`$hOqA;9c3b}wKdU)JbjjU6Y&k#Ti}^cn8!7fFb#OOl zwPc@MB@V@Rb5T0`uUfTEmQ-*|33Tb$3Iwd81ymvm;E}j&YQ-#0EB3TclM6%Eq`f4D zOjaRNcaM!cmGPQQCtcFFSPjT;>-jBijqdt?w?4+OUZo}diLyA$!d*|2h z7Q)`chg($b$TKD!+!OD2V=E2)6~$!42pP3hpqq2X)#oHElC143fBa>wWe{a8pBruA zrQ?~AAUDKWCR&Mp-iF@2e{8pMYxOk2&76Y0rp4f{wcs#9drvdrkM_X+O#J0eC(GUR0n)G<&_^fzsvIEZNL zWmO#miD91?7nH9%#fHFK?>mYFT)J%Wg0+>fyr65z2OH#y8U^;-Diu7o&NneHwQEgu z@1(-PCkMJ(hdz;hk~2GW2N&~dw?v(Asl{XZ9rl>`K$Vzd-~r^@72VJjl=$wI_LGA} zzs^ zT^|XzGzfbB`h1VDK)*?OVL`!Wg?NDB0kTmaC(mhWhf@>^tmiWsW8GntD9)#kJ2Z*V zO%(6!XoSger015DxG85pdFqZmV-xrN`;L-E=fxtFjO^ngy;N3ZyjZQrnIWv25`1M6 zDS`;l!f|>`tAvO9$4w7aF7*ZAg(PSZ%MQ+YwlzilJcS;6q5Wpiqn#qvD3+s^MxLW3 zwW3R!(r5(nv8HAfYIbqNG#LT@JQ#cYzBu)BgNO-46^@JaY+zaT6B^tx`?3l>#_oDT zSf~Brk-O_T(Qam|a7!~C<7#3VQn+qn>s!UM{`?{t3D3|1cWT}eoac2K1(VF&h9A20 zTSP`bm1FI!!Vxt{xHj%jUK~HtRW}-=HM9rU4qF{$JrZo&gx_i7$z69C#Fjjq8vCS+ zCMgxHk#MXXoi05GQD;5BP0po|2XrSs$aQM`7zx~t9ZpY|qi!w&ddS1@ymP{%v8iR=j>JkM;I&i0YZ>vhXrAhrlA{2eV^fJ?W5E8K&?v0{OTh&X zIGkS~co|X*S4*irz8bPp;Lv(~O6AHd$L`GDdz*4!6`Z+eSoh()hn@F6F#oHG1sxTY z^=Ek{z3Ev)pG+FEN@;ld(y5XZJ2Tvgjl)jJbgN=HSv^xWzm|-dWPC%m4O*%;8-}hz zJ;Ay$YgyWdlxa&3Gy}G)Xu!>7q-5DK&63K+-smD{~JH&;_V@&~*#JX-K=Sbc|@d^jS zC+bs5Z>{%qWV!2ZIvszd!;HA)d+ju8*B(&M+6NLN1xE3WR0yf^2;Cfa|4A)!pSp=X zx`xeV;#0jMK zE=s5!u%26Dw$`cb67a0PxDeThey`@tWkyU!>6~F`-^3v6ScDYq_~whlWZN0R1pYwDRSk|w>3x+oG?|hoo$T;#7G%^z~LPsh{nG#3148GaouxVegzhp7g##-XNJgb6?uds-B$6u+l$Ln zwBJKrZP?R&%|mOcLPw5NVmJoBxSK2AMv5fp7Nyq}lYU_2AUIU2%$=s_fstqRf}9DE z&ppn|#XsFDCsX{zQ#p#e)6dV18R(SojG5@cE?CKzJSmcwhH#fT%qB8+Fi~q6u$Nq6 z z9@G?O#t^mZQKQaSrN+8m1p6EqIgJm$695+k8>V^JYIav#*6R;9k;I=d-9@(}ta9u{ z%zSO?F0XHW(-W}xG5?oT?n6uz@23o*p^!3Le6O6hfYJd3d5qRnziM?(dg%% zK8tx_xNddb<1*B}GtN%tuDzSjZwKYdJZju5^RuKCX5!)^nL37H zpzt2f3+l>raWP_wRUaflsB_oZw_eW>R!NGsuC;R?8rSS!U(CK5JMqF)x+cY#4*dpJ zyuY45C4kh%MHGE=GP{y`N_HN+YAsqb-S{ld$e&X~Z%x;4peVg}kj^1R7KgByI~!FM z{_p^=3;f9zxn-5T8L_L_-h(>3p8CC3^+Mk|k9~ju*H*p4S|j23cG)YrTs;nxi}2R= z`19>igElTWdCxDGRH9wz6dG6=ht-bf7gr6Tz7y7~&;&WkCQ#RZgvJs10W_7v#}>2N z3%5WB^zv4WR(gk)78#8l^Ff+w@89DrpB`kJAJh)9t#aw9JXoq$zM`0XGyZX%mBT`8 zX{mz;^>0a<>+_Ky_^{YMv5(o*DN!se)BRel)r)=SdL9?(#c&%f(~HRZ+6B1z75Y`9 ze3M3S+>=CGg@zLFio}_67yicH)K_z{GBwvw1xVONfXhuZjm*vI(HYU>q`kq{2TK_y z`nbTUlRUvi*)JizX-U(WgL}x!td%cFQ#85JWo=EvGhHWgxRsqc7AOE|to{2QCx|XP z8yS(u{MelkOS2-yNz)JRg_SP$)1!|$8b8T#Hko);4@KF2!F@r=*wnL`l?Ci?B0!OZ zok8ims?9=}DizU;L=`lkW$ZjMcjl7ruTKq~A9*B=(Bs+H<|8)L$|#aCs7&*leoGs! z_|nXSvhFwYm+iLr?ux0I$_r29W;`_t8*3!RHz&V8=9pP!rW|k-&ArUqaak1qYPTF_8~Zgsc2C6HV;)x?bV(^+Q5uazTTR^@ zg~H>1GSIm4MShL|QJ#qD6XW<&dt3n$o!=AoAW;a*zgtD@oF;}sE%?i&2Wu4*>2i!* zMIUe&7>nhYuEWlcO3d&ukJChrg)L57^wO|>rTxUT|BP7a!GXD#rp>FPf;JYk$x(1j zVvY>ya&KwnLhW6!JL1el`}XYhp>VanfqEM<4#2E*y4#Y;@&1-l1}r-21bw4Z6rM`bYiC zR%0*s2y}~hsqwf{#Ue)~`aSezy#K6=Ez&dM^kb>dFB?Xs$4}2KLu0=1Z~kqb?@6svmP$Ge+~7_s*bar86HMwmTsaXkX0@- zuBte}rnP4UBa5N98QLMMHvQj2S$iB+?fdyG$Bx<9xR0i4upLT{09$`DoFB?_@me4Q z=Z55>lWWVHOk6V_>4-b*ystKc_A&-Rv3==v?$jqHOd~OiksmKD2L=LXbCNo$*oj37 z8y%OUGHqzjyKSrrX{?84(ue>~s$>#2Dp%5YC!lLt@D-;X1ARK`nWvHjB-S8*X(it{ z>3|i(aIQxY948|;AAB5YUQ8WYRB*?>RvF;3@_7hfSM)IV`57a1xJM0b@WA#p)T%U_ zykGp;ouAo(6d0MOZ{|<;47(QM(+mWgQMzHZ{;4}v-u}41fgzisD{i!HEbe!<=>tC? zTjFDs0;hJ@r%L*{>~nOPsH^ckY>{_CdL_U3SMeaWhSULhZhcnz11Jr*bjQ-cB0me; zP;#i8bm*CG%4B2Xl*xjAC5+jPC(6;H&Ud!49Xo>j*pKJnZ17p_s9g1+ej{gvsg3d9 zWh}0}L9hR_?_79ugC)&J0NzGGXD~~QHQL26fTKredj28QFtu_N(8ksN3xtjZyy=8r zu4=k->Ko~r1~mtwAa$H~1r~ooDOG9JMNE)7LXwTN^q%R%nCEvuEsMit^Wu6TrV%g3w-xMpf3826UUfovK$ji=cqL% z7vgmp|DZL?-S&0s*TC*o{ZQBpxxAmtck&d(uYwVp+eGrFy@Mm9eeK-tIn@yIzsFed z*jE}_m~RniI!cCvbUwPQiB^Zbcvl#=S~M5M{F4kX747xXiTaVG)`vRcVQ zC(1f9d=h`#NZn?PQ4G?_&TZgFeFy$!G5GbVvJ)_U0x-Rjon)#Mmf^Q*>-|j(EejXo zCzq-q*%l}0`dj+Q$YsPk?vL^icguA(k8lbq+7z?3Ec3D)4=B|vwpV7pr%k=Is+cG@ zr1AMG1Frs2iscFF+V8xN3w57ez1(qjDDMG~>R1>1e(-9_?r(2w4bM(Ji=gm zyPqTCzNepOsHSq+_``)pp+T#JzS#Gk$3ob5RPtQn$jXkt5TMK%LmT(N$loU{E?Ze$ zP<&Dl*XZSLc~o#B5@>*NHPNvwL^8G7g5W%^bC~UgZY+cBk|sB(@h=z{{93Q57ES4@ z8>MIp|3;+mB@jv$8P4|v9R)jZr^Ed*eLrqn@|o%}xBClSp+jBk{iD?78=-z&w_ZZ} zGLJ`ppmO#g2&$O|e8*nhR8`T@L)_m3qf#|CU!UZWt=PFg%F(`H)+J4Yf?cK0xZ%JK z)AGxnHnGD_yG~i@Co=@lA>w^`ZlkpLY}1j0WL2!#}whU8iofv ztH;zjnzbFpEBla+(byNaVb1>eXY9G3V`e28qe{a*`s@stzH?Wy!`TV7BQv6RM2Gj% zg)XfsCCD*(7(p9!IbI(AZ!dtGTlSOvS&QSywZHF3BV=8$O>zy+fd99B77t`2P=M_hZV z4;Ird@^g(eUqrBs%fB-|dMQ@N{Pb~s>E~@C{wn2DOo=s{LEt-FGe_p72Z8TOcoNAO z*I4@d!4ISW|H-^4^4MehQi_#>6t<7TlzRfiI!3IuFzQsTSt7G=x1+kn`70I7ZO!PR z)JX>bx*pUkF|)I>V!N+vrN5tnXzJ*R{FxA*>b=}NTT2yu-S)Z3*~J{gB5#c)T%c`q zyDo^ZD=XmsNY}omjjpY#=n@VjS6D4?Z!hhT+m_m!%W`NbnS446mE(=2t(%==rXS?! zz%WF&+L>6g8KG0a(me(R`WeJiiZb;6%T)#j#K$T1Cpixn&liNLiHu7&?rzmmoxM9X zfsI8q)Cnl^LSh*%Tm5DH>EJkCj>h%H4r5#)KipsG9RoQ?0K$@l)_f~gTmq!Keq4qL zsZP#&t>!l5s;s`5*+NM7oAVstS`pyk8muZykz)1LlJf3`9s$!lm{a?(J1joK9vOkc zdTnuhl7O;^Wj^ob`ZMiW47wa-bi^P2-VN{xN7F<(CegcIkWkt;q@`9q#mW+w-hwLU z>5T@|!&j{(4-Tlu>hmKAJDGv@{&TS}OLR%|u&mgkk+Az2GilFB?WosF@|A`Q!`*K_ z2Oe}SPCeIjuk~wH7}8S2gIKYR6hYd$XC_EtGi>9z%3#{Xo5N{{XV{n>wHoW z2!s9-Wwdl#PvAozZMFD>m~jBQ=}@oHyCCA8NTV zKlk3#LWdc;5PE%y6PY04r)g?<=oL-UpZYClRv}V*TCX6^c%0LSC{^1kS_k-ZducAD z#qNoPUprIAgwk*gVQyJBeUh>kHCcX^7ew;t;{;PtHE4y#noMw(m9!xI@~7R^Mej?jDE^x8B7A{Lk3do@dOe*?^eDaj&+Z zFIxquLIbyRKUz7=ev+eXcos#rZERoCZ0P(DFQBjJOxj%^!^Ym_0-#36tl~2RXj7lz zvl1oWsqt|HKtawRu}`e{<5g^geaQ_8(HfimgcuHQU<8y5pr{`Fs$*({8%ZbwbKF}- zMg=!)(~3mgY?)oFb?uPo-5tgvU!xWFT8}1U?U0QQGP>j0$A#5nMam)jGv=c*x$uo7 zEm*Ce*U@ROAgS)rJ5?jm95+C)1 zI1I!=l`$ohk6$+5%*U{4U<=Nw8+ro0;B+n4CHXn=HOeB%;2>PhRJ+Y52*)WJ-gb=z zkK?cpuw89CC;|@!f6CQKf{e8J3R4FMqiJ;~sjkmMANXZ*kyO|o3{+RDAkRdBJktx@ za;PN2gf;BQW${)N8_Z|7_IgRZ=N8-r1S!gmO65Ajh`HV94MxJ^W)f@E9vK`BI;zri z^zkYf;rF1vh5(XkdDJ6Ji97a{%-7r}ZE?l&kY0l+DVLgS5Xxp}^QyPR62Y&I1~TGL zrxeB9yl~=>1hC>i&n(PhZdV?oHpxhl|HpbX-8aCWWS!}hGRdcfpu^@ZC4D?6QdHjX+; z2zzY(k;l$l3S-U*pc%7R)j%euN#t#*{c*hya+Dp$qN}LfvVY6+^a^&QQhOcT!)2cS zIdASQBleU()j=go#8I;_&*U%Bnj7b;F| zX4N$gd*c^QHCHgWBG|sxgQIYF>a#)4?U9$a{9|Qt3n$sj_2u9~+K}ais-3abm=l=p zYejpxHU>dAj4$G|nsJHDs)S)m?!M=V{cr?osqp(x+(RcQE~D%i~)i1D7VX3>o_qZWU1ue_Xj4c8du5GmodX5)Jj!nN!*RcO8#E$VtO2da(gvfC@Fu~#8j7d;1U*T{j%g@?p206 z`6Kot9}{=UDkmgZBaW@*a5yl){3~Jjlmasj0m=E%@jwbB$Ko$w6YmD=XAOa;3L|9) zjTmI2k|n6x{eRU=*|0|!l*-&+E?L~&7YzGz{@>s>COk{zH*526rGChc9kZ@<>b+-` zlS>~4F2P`O)Cipm-H+;C5)M)K%!OCLcM5s+ccoKFb9NISJEJx}pl&BAo~Vo_Ok|3U znLz6~v@>ISQ{2x3$9Lk}m0twB1ARbAM|F4SF$8?&P@8K&VOuZYuGFXcl5~^EvT(l^ z^s7Rxm8+cPVFo~sv9%NAU`|p$Sj{RCRtBrj*<_&|hR6Wvyte2DggHm6QmS6qs}tMT zEW)o+b31HfUHs5=axdsgH$JY)kajDI z1%M1dYz>r}4cKbjMcj3nskg86v-Bb#roxHO3TQ1SD!)ZoyIff)Escm8LR3q4_&hln z)KxKnr37`jWM>2tx6A7`P0yiVo(&D&m|6!#c;Hr;fV=HU;?HB#7c{K@auimVh>hJ` z1+Ep#99Pis%@@exb0^tUlZx{j&^*sC-)pAzz|Ju&uQQD7dhK-91~(atiK3)SBHB4T zuxA*;H(M?Gi*2h7i}FwplN66yTXR`US5<30dMwsISC#-7U!rXzzU*00r64%2IF0Tu z`nU>xN1o}n+q>e=umkR!GxRd6Bv$jQ`u11&BG?v=p1h;}LQt`!iOS))UaBuB^8^11z%f`C^P z7EmPiTDJL> z_5HnSa3G1jD~ZoUUFhDceO^W0IdFjPWp6_&1t4F|Mja)mpsV`aj;YQQW;X=-^Jc60 z=36Z<@Vxg57$?uM4Ng>4e}{gQBhM&7jygOi0{wY=ZvE)) zax;hLhj<=csq6@m8%P{>V2{{ zcq`VYwQxRfC&%0KWyvefmU{W++DkWEa3)%1gZ8%hPh0WTbzo*1b>?fjc4X4?Xwb&7 zc3fMnrQ&k#wb=?Aj^S}cuc6VG>Qu(e;n1(Sq;LszQ++?$YyR`2eupQ=o1p0fpxnTz z28b1(hY{By?~H)g*P?&2!Eg3)b1RYt2?U-+1}YDo==S#7yRNzNUtL^#YteeS{^xQ! zDb0Q&r`Ou8!Ho)(K@DH6P!G`t0J0boZOqw+XO5t^DAM2pr>kHm% z<8B_fs~QNPb=CUo>j%I+jI|OsJ?#1pn{av4JO{%$kCnR8Z{`N~>mQobrMq=pnZA|g z7iS0$;7(m6-ue<4b+>c%g~+mz!X~uZ*^pRyI`IjCjtvA@xlWsI<;Nf=#yRI5;4oF~f>4BTC#mbv$0-)FtC-<9Q8Hp3QIv|rTOlgeUw?%gU~Tx92E#?EC!{C# zX2L4__)-)x{+n(87@(pTZU;<)-bnGViPyIyL)&80z{5A=UWl@bcyPU;dCGU5Pu@y5 zp4%;T@gbd*{vdI0g>uO`cXNfdX4SEqrNN6qkEL8ZnS3Jw{UI-Jw64n8@HigjYx6m{ zKpkhM;0&-?#GqpDOXi)n+;%2z)zB&gaBv-JU%S#}WzoKaKFgY5}NRW>)c7_3d zVA{nzVRr)fcQXDc%F_bn@Iilnn3P_C-%t)uE^aW`3E_E$TOU@wI8*vWSj5uJh6a;t z9l;nI>6bH=+X6F*5fM*?_)7e3ljA8^E4sa{2-;lVtb8;l$?H8oWFK(m(HG0q)w;E2S07h`X`QJCZ|Mn+vCSrbK-~k? z5b7>P#aGx7>EZfq@eSGUsDdqD>j{HYK|9!2RdSJ-hPwCfB|5se!dS7!zaR`jF?SB7 z+crNI-4RFTxs!IaVvjmCd|1akrT6>;*3C(z-NvI_Gn4wCh1v;$fP7BwM9i`67w@g@ z=+=#`IZC>8afP&gwQaupW`qSF4Rxw)Nv(1+JSK?qRJw;J_onnNdgb}6)n!{1sWDsc zK$PjU81!Q5R{7(V9`TSoIw=#dslFDbq~Ni$;zd1M4~w})ve1kaVPj*jbq5u<()aY= zWKDoRMe}3jZn-I)mXgEHihiIyFg6hK$_?rs)I^Lho)DB>2iCWNyq_?W;)YUS#}lG2 zTRA4U;I5P6xap9o7oTUpQgw8@Ii1DhNywv_kn(P9D~==FTQXIt7fhirTIAaXfw%TE z9mb(r({V3X^R_7L;mytt#Z-8IPUkX03UoUlgEiAEwy_eaJC(CF->d_sbN+`Igg5s6 zsmhD0DPRRSFGuYyf|lY~%EGE5P&GXBLB%S(0yYh9xns=@x3`~S(=!gn78%Z!kDP3M z5H{V+oC{q+8DN z%viLEpP$(bj+}FzZB~9I!tDrsUNDBg+}*7^*wFlOh^vL_ZfEMf?vl0J#G<~2Z3VRl zI=(hInsU$|cHbKA6{#xQcn@*K^dk(b3JKM>M1=a(<7Umc3U__^cC)AOi9EaC!h5wA zOn%XSL!Gy1ktYa#{WD1{%53vRwm-oVGfMdvc-#CJHQl~EUUyLV_leW` z_w-;heA(N#tM0ZS-^yk2wWNXvBE(aB1Jw>}A5hr2p(+menI+t*}K zTIlj^=E2b3;hEi24pks=xn#y&Ec$QTBS9$tI*rh?0zqvL$;Hmn)f( zm6hyqks`7(u01MT`zqTtZsyIngt)le`#V?f&-Zu#0mnJd^L##@kH_PAUXqNJA~Cc! z)OM3_`*=U6{B$Pt#}VqeqOdYW&+Ip*gtOY4MYtQ1#30PiUv_44IA$B611641xh4Ye)$Xs&T*gwkZ& z16MjRttp2kl*~*oEwMxdpCb*-36>4j&lcCk-8r2V1bpLJ9DP|#2QgBu#N&prXI<$hOr#|13{^ZkU$aMT=(1nx~R`Q)NK_c`8L0Tri2C1zQDd3;Sq`M7C)$ zvcOJwT8VKk=-&?Mp^xY?2yk0)N6aGtS2`OEn1h60SoBC-jXUFp2+afAo9V^g+KP#>?U7=haQK zmgTy#LW;J@FHaZVt$gitfnt9>CEQ_|ZL7FCTjREX7KTR7g@pCBk_?)5$JGacUa5e9 z{i~NKkM`borzSX?A-mc@pRGRD`&H)hWu3IcK6h7yX#&1^Xnlm;62ArVdQg?i+>hWZ zcLF?r#mXPbIp>ba&DdMoTs5KKZn(v6Y{z*^=f$)O!r-ToIc)MUw_rnbCKAo>jh3!V3rn)P6tfP4FXI{QoDgW+!Xf1bq)AG*`Kin+|KDiaZER~$% zhmRSZ|GM&{i{)QrZAWlhyz)=h51}H{Qm1%%7X%n zKV-cBwD3W~S!`#{?Bz3?<+s>P|4k-MIO773|N0u*-Llxt2*o3ct-+3eg6>kc86khy zFyArSF!vq8<2T^990<#jC8dKZDYUB%(*1mP=f^+FD#%w74M2ck-N<&=k(Qd^=Zm>E zv74GZJ1Y&+90Iv+Oy7iN&rrB~9*=AgHmB>iJ_0kUvOA@bFk$`cUae`F;iXYOhlu>` zblP^C;a2}8?FlRE=SwUO*;^#BwxexD^;PxjQ460PfvjJ!Q3Zh2F~fbM&C`9%oOCwE zppk;3hs;@koBq5oA%e9+Q&qU}3tYF1QT!6YQtm@i0ilYdMabKpFDZO=G0c;{kkS*wc>XE^FyiAT?Rrsr!MyH`E7DE9$p` zwecsx%H++nH6GL!B>HULDf)XkN|TcOJ#EiaRWZF|Id0 z7H%T_%95j;To>%*SK6@DicfMA7ouCX4K%)vh^78RG=yCAKBFRGu*&wN6wb@@px|w@ zeSI#!;Kxi{b!SVNJbY6cHbr<`jgZEHm)7>-q{GCy)*P;g9N5Wci71>OS5~aOz?gHL z`P#%c`mEVlehPD23t}WHjcn~RCG0x4IH;(wKZ^T<$M7dS4tX-iwI6M_GEFzu&>Sq# z+#9?Jz9c*^VpVovcyN6M=RtjX;#3cd*J}@ZXRNXhvyIMnS5E>o`wY=(?wk2>Xc%t% zRfd$XomMuXe@vb0R`5WkOctU&PwI8cR4PZH+nbI3(CX#tdml_7!v@cv-{Dh~`o6=4 zu)7x{za(5hixh`xV zvho8Sta)u+S>GvVDej>ni4*_ zkWkLf!ND;ePbOz5zC1Eq$C(r;9oWPS(}eZNjf@{ucg8T`QJHe2Qk)SJNsaM8`(2B4$?ZM2Sf=)AaI{4^KYsks4vHm36xL3gN`_2wy8h1#u&p6< zneZOVnfQCy7bBJiypD>6|J}ozEE{XY2E<+zvUWWhIkD|exr2}%ee={rme8CyJB!iT zTU6Rvm#TB^Xw|GQ%r%d=jCfx9d#MC$A8DtKq4!5&)!FhOWH4b*6s;0c^upED+`k@1^Gl-k z$W%>*eZEX`CMZ8xR%95Z*N)7FZnAA`9t`azId+U>rU-BD4Y;sQZ8yA1(d_81{^Wzv zv8@i?ZwaWc@c&?nk9LF<^83E&2#A0MyS6_4TtT>nWB+a-;!TopbZHysr&p5Zf!28K z+x$^&J!oc`dj6IBZ05=TqlWCh?{JWPTo&hs!#w1pw`8)9nWFOuP!WL6#)W@F_B1nG zoY$?6pt5NDJ>aRG(XV3ocJqo&?^!dw25a93lGZxZRR*&4_O z|Izip%Ai#!E@@UxHtfDVDuu^_Q!U{JYKnl@lEGZ*? zeePOWS0bT7P>=KCiJ3R2VY-E%bE&`usX-|7-%X7B85rEc*p^WrW)TeM#QKE!%xj#i z`y49Rn2jyN2`3qXqG=`bAiyFLGEUqTwg z^=3k}MDjSzE^r7uZ%h9tHa#58a%HB0xB;mcp?T@ZvbuI;@usIHN+Ts^USQ2SL^SBb z@5e0CxnhivP51}GNz73s@qyg#G0R}P?b3Pw;12c{PK$1o`3#PdF-Fl>Ym7rzC-!|V+ZsE)S!Y8@Wl;j|J1lk&PnwbQ zaPD1xQ97X+7(94!9ri8S|5(>@JVE%gxYpf!L{&&=s)Tp0SrZIrh+~h+v3~aq`evj1 zwaij}ea6*kpMx#f-D?cFXkzQhNBk&?tUB*tAA`$unU6l2>mchA2*d9C7W2{2iv{%HH#nB7jhUw@T&;2i}k6v`7zd9aTuu&oKB(9;#{G2ma;yb)O*5O7s! zFm^1v$Cr)Q52s~%G)P>w>)YnyWLXKg55UN!ft6|DI7A#;-nrjHRW66?`W5$ano#tO zk*VXB0?~#w10jN9@Xp<)$`ak|X1~whcaI;~wLeMJeWq1&mon7I4P{Q1z+cZ5%`BiobbxNM{gTCK`qG*?j;k3|^e9J`cjE-^Nq z%ar19V76(kifKB39ENKuxNz|6xAJ(|)k|bA3k3y_&9gi#I^OFr_8vJ- zi{MR-_wU~?p!FnVT)851N0NIi9pcQ_3A>PeBRF(B{_7q5)g&@Y)uzx$ByuTku~{)U za?g0aG=^pv7M(Yi@Mt?7zk{m-=b5xr^xn%~E6Yvk$_|T^FY-Y3cxF?Six(00bd}}0 zcTiaa;+Qc>E7`$g7VkuS-oU(3`t=Uhs3h@{4C;fm2`%>WNbnPx@6b3H%}6{Y(W|S$ ze>3=JOd!6SV}C)BLFe`j{a;6;wpg;HN3t;O@bjowbdPytZE$BUJbU2c_Cm_81HUCBeI)-IxtuPrs8bw>`IhUx)cXzOInbI7C(#U2_h^8&GNr1P@TI)D9^|aB7g#m5k1S{vS zhPksD{V~h;SFK5W{zbM+G54%E9o7Ns0#wj_oy!Q*=B zOR3sZ!$;qlhYy2^AxU@|##?Pgw3iUQ9Xs>q7QO}HsEZU!j9@d9$vtrUxR%x6V@mAU zO`F>{jeoH|dVSJgQSjK;w_#gAj*C=UA=Gc#t|jf_T3f1yAV6NMgxS3N{6kLg+)Df? zDw8U<+Z(5a=wN~BzZnRsoS5>@f-&ZqW=?sDOI%=c#OlIEeNLZV%UxB9k2Dk4K{xsy z*f@X6e3##GTUL|(tY`X*Ro2;swvBfjyz|*4Qn6~Fh2;;L(B}<%=nZv7@h90?i1DMo zoabK?f|Y}wfBr5nWPq|+m73psy>OO$!9A3v`Lkxk#Tg^Y$@qoiox*m^ksWgc7yIg3 z;TQ58F$KkuxD|71yt`ZA0zeIz{VakbHu@jlAs+CaK+^k(aUZ`o8}%Bc2hnW}ZLl0u zJu#T)+oSZ&RvnOhWuo0Q&muFV-H$?p@sx(UyEDVt1ElZ1mSzQ7dLy&RC?(&biXC5& z+8zi!G{HUmt}}|F^F-yMa!l57h0HY9d;2oXTrkrkss7!U#|9a(wA&{%#s^vJq6%}Z}O5$)$3u5^ow>BCO0=$_B)2jDMo)igC_9Qg<|8c;U} zV?dwr5FeSX70`2E3U$^KX)MI_Fb9N8QL`?u&b7r0oA;_~t`1bZ_Yl9}_Z+?wxog<<)xRXfB3`T^g$^%Wy%)oLF zkK#ogFk+FDXT07lcRtjN&U%u?Cl*e&q!h%@Y>}wqn7zvBN58X&o4c`!V?4saQmt=>5 zua}27Lv`FoQ71b_5wbt=2<^4;XR!vxTLt=cH)N(gNGIv$oz|W)h&==dXJ0l3LIIuc zqgH^1!rMoRlk@{rP11Z5;h}Y*qxA9!_j!~!lbMV9DcC4LZi){4s}DisS(LjJ zwMAkshZGCIHkmTMPglVI77dCmsT<#+b_Tb68#g(@T9O!i9~IzPn&fW2gp(uE94g!P zHOicCsX%4RBR%cx%F$eBD=Modi6X_mO#4Q`)ZlABw{?u5?Cqmn_yHBUr~xpc87rBa z`9TZul*>rzIIcDBIEACgLRx!9sCCi+qJE=cjmwLnE7Hn%Nar7lltuNH3pnc0<&LuP z|DD!zq2Si<*(=X0B<;J5Rx(3l(;d#JxRb0a3F0Ye@-_`Kuj7uq-fS>4xrkkTJx}Ba zj(xim;NJ|R%^ONBn6jMXnvZnEeqYRPvL5~PIALQQGiA<7SdCq**_H~G7WmgN?lacd zMI3+H^dtPtgb5YX@+uR%>);TIP7fa&xgg$FS>f>>MoUqofVK9OjFRmCMRr~ScIgwF zi1+Tf$s?au{|_)mZ883sKvflXUG2`+^1du7jtegRY5cI7$@>y+YuA2`(c^8?^1@B?STn6j|<;IL)d9sQN+=O$O z^^b7g(Arm0TfL711MwgjLwc;N<#q}#wU?n;{st25Lo4Q>Y164PxN3^bQfPhdBWGWb z0w9h>-+i2*h3{vXtM5QGU-5VFW0s~2Kmkvo{{U5gjZk;N>MnhNerVfy*mO7^$JVB- zwdzK{5NfRyBFgkwpEfdco~y2~Sk$)`7Jto)K1XU~ilB*Or;zZQR-2A+=eJq?zfjJ( zckYFM))1~PDxEZh%I}Qw)8NMg*zuWx%!k}?*r|@|R!3B;r6w}O_t@U9Z`7@~gk03U zfI=^A9bWsb@v30v`WZ3Ci+~{q_}khN3tGhx4n&O%+y3~jDgvq& z4^5+HY-}qA&%FzzMbEf8U2rLM0lEqKr>qce^(#~Q;3~4a0Bv$b^wS~=jKsl)o_b)r zLS4Ipcp^-`=SuOa*(ytT(OCyq7I^SA{})4ZE7mM+@xXK9b}nd6VSWyrBua2 ziFJ{xazRIEgo}-w3w?+&`U6KE`lueC@qh7QSz=iB;zsvk8{jJP;dlF}lU|g27#dky zk{88I-97{m!4tW_+6Gya%S{p+;r0Tzf3rs$NH!CXRcQNRyO%tor@&a$e2{eQ3M6GB zQOP}GAmb>!)nA2w&*=T?YSOR{9_Nn;qj)HoQYA=0I%A{gnRNV4yST@!LA-2 z)6{yz8?kf@KiiJ`+KS==B_YK5LY)5uvXmR;-wOM?4vLPc`@w_HK;CQH+;|Va8$M-7 zYeYxhmUjTBm_Z8{p4)s-e}4P8Bu%7m(UJ^IQ(&}D9Ymir-;{BgJ(5O1uT8PKUk_|} z3lVv^vzCxy#B75qUJP=Hrps6WxMjVG$(g()X_~h-$2LA5po-3^xns8R5*rm4D6ElD ze!7sb(DpJ!^mgoTz?Gc*Xdf=&3p1+^a#<6TYN{P2knH-!fqGv~uNQv3Gp-D6TsTH% z?tt4Ar}QP+A=VX+qp`zqac&&A6!@B69Jt%_Dw#@?K4sRoOackF zD{Fam*^KV-*o1M8t>Rg1U0tffxiMf}99_H|bQ$R$bTg49PBeq*5ccTAObHfaEkf3L4!#H<+Ie6{{*xO!Knh|%HaI^vFyGhdJWKhS{4M7waFY3*7&_FJ$5Gk zk1tWEdl7QHHFq_sl7xOrV=|WM->`XP47u>TBGh$nl6b``0~X)?Y@;tMyj%sxOrh&0 z>y1B9sf5LTk{+uCHHl?A;+i_b=I;`?TYDL3&Q2a(Y9-L)L@q9a z(|?Zq_3KyR%nTjak?>NnD;pkK*xB^Yg&0iif;#2HUS@+QQYj#OsT~;l7M@&~&(27m zoZ~cF-(=YitG6UCEEADMQU}?wTG8mI;x-IbRbt!s0XM%S9QJ(Q^s^&Ld#Pq zx1SD%;l*xFB&Ws2PkrAHk1N9sa&i7Thy!)$~K0q`m%si}V-nX5&dslR%! z|7ud8lAe8>{J31j57a4zav`|O-T)CT&okc@7QUf_qov5Rn`zvT40iViSEdABfTr*1 z&}fxc{SWa`%=bkf@wgj(x-Dk)?f6ybP3{+*UJIMKHg7q2mT7S!iB?N-#I+`{wd7oZ zz2EQf9DK=A!B|8`YwtL{QX7+a7%{lNaXTr{Qg4Z}vVuBv?{rLO4y2}ey${htV$Q&TW8{ezGSa&%NgM*??<8B>DW~^ov4E(*n**1V>7GoBN4_r zP`T0$F?h-n&tSTgZ zkcd#0;kU!HOH;nilz$PHvO*j+^--k|eClVDG@|+uJW)Pn;u63xeAo8y)G`j+mXh~o z-@*BL7)9I$Qq!ym4KUiD#r2IWn-V<3Aj=TOGG}KgOMD#W3Ww4dr0F-eW}ZKp{VC7u zVKF_nm-vH7Y;$o?X&xQOo|?;2tbwY1fx}!Em}*ohl&|c2_HtPVRCccbYQvI)^A{Rd z@E>#tfj`c1Es~$wqU2pZi#GW*0Bm4tQxVVBTdL`IEB?$JpiT0*t#B?N6RVAo)9P8u zso@&vHK^4!rbjz8;Ncu1tL+X3KO^^OC+K}gRDY(^n$F(Lpc7aMi*T2VL)1S^z5XDS zML{TYvrcc?D6IIkXm{ToLwhqv=$uT&fdiZGiaPo#S1B9?oi~=c8dbIJs-&H9u#q0^r)Myus8bY)dR}cwY5|rUv65uU+hO1Ru{im=zT{tT zb`nIJ9g_pKhw;Mzrjv-ici7Z6nT~?bC+Mj=Is9-~-aufgO#?ovgG{D2W;&dVg1?8W zWhwU6>XT57w%S6M0?8A9v&pBojIp!}4ac|2v?L z8DY;Oimv+>e>bHkg7Ss?`Kp7GK>Qh;$ZBjKrOF!-#ux^`$9BnMQ?rawGK7byp0k z*wN9zUGAz9==@?Zv8?PXnH@m1|J@L7PV*i`_CBYy><`x~*)Gn9I5P_e8m}^RX@nQT zN-gZ&jpb}IHYH9|@mQJ2`Wp}NnbklYy?8DwiY#Eh_v?zib<^=RH-GZ>fpny-1H+QZ zd<1A_pFHS^NIQfUXQ1Viuf7hP8Fz{VE0LNHT4`(5gQ%ZT2!2r&^i)H3s;aOKuYx0w zw}M~PDHpD>_`I)@s_eRu*C0g!fLXw^43HnXRI=gcxOl^WbM5z3Vd#deGVU#prVs#B_8>1hNWsI1;GBMr`f~Z2y ze7*sn6}MY^QUX^saaw!TbRVKN*?k|Ysv=7g#RN9N81KraJjljP`8ranC>R5cd-mV0 zaw<+v_FoN#kTVo&qszN^qWxI&K~{io1S~b`c}~ir45&cjpaM0h{4Y8R#cUWB&@9d@ zpctl*i{a&}13>2~qD_)HK8s`p&ffRf^gx7h&+6B1+$h%d@xytMs$bZF2}mJMrU z2(Q;2^|!jBn}yDYUS^ExY;$t>Ix;?jCf*@=`$vG?E6`sV256SM_nx|XSC%w~3%Iyw z6CPWVnpCC0ecx=o^|eOx|65I=e(@DkwlOmCE$edn;*^cKFCvZs^hoUtmv()KV%^Jx0672#V}jLme0`*Qr8@%6Ayn31 zI`9pK%W7lgrY66}K^D<-=(qy9zG;%Ic4>o1_o5lHzoXdYg+JHG$x{H}bc#Dan47ya z0?6X5T{+?B@n#^{V!nkhkJ$wsxMKp8gbGEAZ8b@IKwI0%QSZUZhiJC=dhfod6dRI6 zX4gKm+NfR|^GN`^k<7G)&8+{eX~ldCASGHKF_GhGzWL|;t27c`<}G|_vIRS2j*n;z zq4^t}h4E@VV;TyWRN=?gf?x+8%iiV;x(&ejfxE(Giv;=W=R-39=@AB_?MT?o9BQaW zKpmm<_ncw%U0@IJ2f?`*!bY&OgI>*M5M|F=pno`wR?DIxzS)z zJ_a@d@$|XfoKa8!>;yye=kHY;20hAIslPFYV(0wT_!#mZP$>*~1uy1M2@|epF+N3J zQ?0Wdo(trj$R*((MD?{#!u)Kkfm3CGryL#`BT8Wu%mBsVzj=@p@DS3GBs4fSl09Jg zR88y$D!(U6vCGt-zuL&EYGP$br1Cb?vovWcUW#>QCC&?BuIdR~@3M>F{PRPyPVO{P z9BFPsae?b(6|Ls<-$b7m(eSL6xdfiS8k-)MGVw50>CEXX;RP{#iBV9MY`)T(^JhJE zf5w`)*Qh!|H-z=X&iGBl1w(5!0*6SD0k8sO_dD$sb2R=UnY}kz3*9~~TKHgL!K)aw zheiBWj1W3Ax)S(HZ8r36+9OFeca0}96BsW2`-6LOT zel4n<&(k_IJx48Nx9rKJ&&QKREU4x9_3U%sIuoq15ldeINGf5MD0!Nc*P`=7-c6mo1L1fOls zm>Rb_1>6xU*X)s}U#NU7dFSh0`~P_X3JV&_Y<@fq9yi5_q?@J|4igGoXP_XwJguHL zfUH&|<1$t{$!@!C5hN1W3`=E|9w4D~bog)@fH6wBH*Lga5lnih4lX((?C%zFqhk}# zqFIs^Orzl)d#d)zQ|(egyX+c)W4`ziRDQ>+;t0Fe<4O?9?UJG#lp_WGq+!&%f$+JD zIBH0~0@(K+QUbHQAtZ*!hF(91e`5fxB7xv5DA$98+H>UlvQ*DfyR>aaf^i&h zAErh`<($x)kM3GLKqc3=ZOF?>O<)4w!(urYQw{I)&|yPw4mQ_TRvrhow<`vZ>Vyh) z+z&{Lhi!vbHBW_DR9hV%hkHq;k^9jCB(B;`e$>|5@6ABPPF+PrNB ziFu1{<4$tAHkkSme16RA*j2tq&Y#TtcXcb{A0kxOMVK0|A~QY69ou3uFr)-~d3$>& z!LE$7@=vZ$ztFh-5E4n_OQkt{>dS_t)Yjv~k;$pXq8jWzW))JF?n}0>BywkQ(O;l^ zz)%?cJa5eq_w4Wf6y=Y^s`3}gSXc=inep#2^jUJ!UIoI|vMbT--r{>up!giJMl=H; zKq)scqCKNwhIs}Gj@!FeOb0eP8jvnjBbEk#e3mk6@NgplC*mBtE(5JN5ImgaN9*Yl z^xrA!my#^mZ3Hobr%QS z?QX<4Oi4c)ICChk3GeOAGTQF6ZOvd^9DY2_78yf@6c4a8DU^@ykQQ6@L>RfobSg`b z0=4G1MCEaTt!5RVbV_b>X}jUXcV{fMJ*3c6+U}JB~<@dqqF{Q@|Ei7`aJm2iO-9OKr8ub#5i^ z1@GDqWVOk3FM3>EaQCzhaxs6dSZ0Wl*lrgXJtASUqo zIYG?`^IwTMKusvp?lU#Z#c`=a0gioEKo3|EU6tS_LVU$$%g--(!l~Zsn(CjJPdlW0 zk%x(k@1{_%V^h@hZGpEf8{qS8`jWb6Xq|kb5-V?As=(ry|E8C3IoY0-&33Yx+_m(z zx8K86Vi+1)^a z^rLT2gYj|1oAT>TacGkpELqnE>G7sG_9AU$;gdYLN>59>Fx9vVK+?q@6q?9^%EIdv z4V;i~w+SyPJ*Hxv{Ez7Ky(*_e!?b^O&EWU5d|juqce191#-=52bI!IdZQ?PVQl)>- zJgJL$oO4RU@{X@r0qm{{J1t9j7uN`CMCaXCfkg>$W3Jk3udaK6Ml7w#&Nk0i8FIIv zUc%g}G|opQn~T8ducg`LN7gipInE-wZu+UP59Vgq&q`3kVKfx!&cwYmNNX*gNZD$@ zDO~z(MDKPaI+y^9W!_1^TN(%n7yIq>Rrej=t-adUlY1nL!@a|FOiT$*@4J;yg%fu3 z|CWZ4+5*iq)B{xZ3}br&wARy*XbgBe&{3-u+a5Aph_+M7rt<~JUQ1NHwsu2?S01NE z&dY*KS_M3B=3dU`$`nHD2&aCkfW*wdra$#W7&S^0%4(~n;1ETnT>(-|`s>;O`ZQGV z7vlraP(FHm(9&_b&h1`iufrdhf2TQIL*vbW63-oMJ71 zvXLRd;_^1INV<%2xT@wgetYoHS*H5>{&#kbxC8eVSn8_um5z7$h5R2|X5^%6pufB> zyPF4N>S_!g=`%+ZENgY$weDxCss1J90a-1YRG^Z#G zMt>{geY1udw)^|V!Ts-^X!j7mvQQ^wCYGvsAC3XkX5>+s57B%HKi}W@y+?FRXIg4x z+}Gc)v?BAb+mrjB*c3S@2i-^Lqq?@^clwa`AT*=csKVnwevM;haCFf7z0Q5U@G6;F zP|+nYiuI3f`hJ!dtzG*>3TtpZDIl*ONSn!7{JVRJYj-9C>?uyt?ct6rhFAxq9f?OG zr)Q&CV3s)cQLMZ1@Dw2rc9A0eQ*Sk$oc%477Be(Xp>{;zck&SnnjOQw;;)|S;aZZC z^x;M!6!gnMaAw~ah8;8B4LPZ0_v?&Jqhv%J7EkDPY^+NoV*U~-8}r|z`gn&ZwQHE!mV0})Pg zxu5&I*zdN43m_3(f<-7kf`ri+jOYL_&WVWzK<No<~8Nthsui<)wL56s-HsHsmPppa*cUN9yR=nSIBZuEnV1VGU}_%`IM(Xq3i z>b=(h8cA3ZVhsG7eG9kcvZDxFx*nUk2m`Iup;t{3i8nqzXm&gDZ#i1o+9&Tz`t_TW3VIT(db9Xk@9<527V2L1*bd8D zgV$SzGk6_;%Ec3B+m<8IPJfXeZ1gR^X5lza5}(W*l)R%RXXHf|PUyVwaC`5sU!|6t zb^%F+auo)ej;4*7e+(C&2vBZBuaKKJTi~B)l<8jf50v`|HcuKx_I7YACd7GW|65-3 zBd+2H2u)AjCpnGsQ=Wx%JBk3GZrktU9F-d7(aGW&IQ|u8fmL zW|zQKvKUx6R&mYuAYHalRM~IRY^7L~G*3)c^+7>c3%sPJI`3RMb1UkW2?UKH`H~wu zrmz+Z%w?$qql5A6g_>$^*xqlmzCG=?m*$Ud9^iwT6-30?#oS=+C&)1l43*Gcs^)9B zb{{L}h^2r%c0`hHKe{$u4v3xWPCxSR`uL0A{N3VdzJiUeO+^No^c<73(L+Chr2VTM zEPl1HGb6R!N%D`hK50WDWwGCc6d}_KfIBjjEtvAyqD&xbc|A9U2FVVhoKyxpJ42nc z5bwB?Iz>qRBv8`Pe{7GXNSQm-IFTrs1$D(GlV7>Q$W}$k_FQrMow`H1ANPMve;*qf zOCj4_K9cc{@?D;<{?SEF_w<#v21GIF8@9=xD|%Q8r#guipme1%{Zz! zHg3cO2mjnq3NPuE-gBWOkH}I`Y*imiXD0x#iv5sZ-JU|;zE~Dtxd$z|bE8gPL@H~L)0%V$1uyf>9CZNq;u@gE>1Gf!%Q{zF(yPwbqcHspN1 z%OU5oJeuW^*0&x8m<5RCTUPXy!u=iP7s~v-i|W?Y(ej^N&iO+q!?d~tJJK(^0Mh=B z-~4oYpxNwf%g?7xlHJRZ$3NdeT>~^73M%gXb>%$2KI|}sKd|Qlb>*9e@h+gfxvl@> zrowmt(U9t_$qLQ$0R1AW1p4s+tEHf+L2}Y`s*V|Tbpo<|R;X5&9T7uRz=?c{^&5Wc zx)*jrX~c&*dKuEh&;7|@yLYY;5M*DJ`01m450(qi^15uL-CTePF~7vNx_Mv>VE?O4 zd%)L2^1gcDf}YMi$;}MXc!S$dVHkG0b6`?gu_N#wK7ZiBK(RI8ON(>l2$7)+P(AObD12)d1%o7EllG(GB)+t|N2#~cbaB7KEU=S!AF2X8*)ji_2BTq<_#1V=jk6yi}xfYMWU*` zLSW}ca)7U2@vGbpKxM)y4*(o;WM0J`LqVt6%_W^MjLPqWw*vl#>V&@$RQY2pNU3>r zeSZH7D@Af#toQAaWE8e6#pP&V673gaF&C*z87+xDK=}b;&fjyx|D8034Y+9GFxitR z)NRT{do4YF96yGlqjP!PIj>gW3tm{FRYBF8X;CXRfM-tHbt<2Lov#JWi;nHHlO9pN zre5DETtmRxf);Bjom2T!dB+rU+z4Y-OJMNTokm_o^_YTIk0R0Qhpx>nU)OJF&N*kJ zkyg?bo7=8&0XMs0=9}v580MtkQ-5i#zcJ?ARi#C|tHH|!3?)~^ygPd)!~u74MqYAl zez$U0Y5gMy1)`u*G;3$z4v50+dSes~zW(dW9G5WH*5>JL&6?_>DzA9N9xxc*0(X_s zroS4XNsD4&f2UCEAgfkc!v_p;DJ35itWOo5*_q!q6yMuX5x}rC{_ahDIeS})QQS%J zFXH;)W9zc^MEdP;&^2>~!^=pmKU&BjU@hWtG#3c6^VfI&lNim(Ehf>NmjPLIc=cd z!Yn_=>hEfE3y%d+BRp65yXqtk|56koju%3?pd?CpKQ+A&Gu-;~uaPBDTAFXq|+gg|aDarOAiEdax z-3_`YHx<4^!(d`3kwZ)8bBT$Kw+t=G(^)^;gO&kDrN)xzLtjlcFKoJ$$%x=1ITCNU zAJnRA#~ujwzGr)2V+8d3P%!MVRp?5j7v2Wwqjw0*0{pVI(0GYX%9d9)WXV#>AC#$)=J#gzqW*qZlL*IVMhRN1IpbX@f31tNqCN7pNSkM@WQFqah&-<+t?8#3!~KZ&4T?y@x?%x@s`qXc25EeujQM1 z%el~-H<3#H2~N&^Gk|o$_Kn41M9E6UyOaG<-UY1=q%%Fi1sj#uhB+3La}u^BXZchD zlwFBUVz*pDnDeaXeqE_%&Ppz4W!CmD%U3)FR0vhIp5M>#r-PLJ5FIVd{oQK`z(9zR zIei-Mr2g_Yue~MN%anbj0Wi@|3~&ntN#6HWxIPxpNtSF*3BVIokHW2UW4al!>iu?iF)OJlfS?oR^!xnI`6jwadfeh4f4bl7)h(vzoM%}zPPmy=- z3ZYTVLkBy?=F%p*l{3e*<&}q6&OfG+-E=}NDpgt?w;08PpUZ#TV2WLt3~1(a>1|NM zD0&mr@lz>Z@a+{^Ju8ik2OYm}`SIF|gqS{~p_ULIB^_~xj?>;uO6HVHZvoY$3 z+uv@lb~qGIBhx$6h2F##-qSNm7uvr&{bvkr@*vBwS!l1c6eFJ@6#ZkJ#drShX~dr9 z%VhN66&>Byu_ff}dJ}l2^M%icei)I%0)7hy4v=!h(TH% ztf{Yn^@)=j%?Rops^Mxs=kF0vS(`G4aKV36i&cT|qY9aAQ`XY2WOxTB@_ljnL33eL zR(-8;vk_2co?=S@&5Nay%HW4;ZC@lc?gIN{Na%ZQo~;FYdSGOu`vDZ(;$oe*V-xTMR=*|6D4xi?Ev2KUbaE}_bv^H$QU9C)4frzRpI z%g1Qw?3VudTrZn&rk=rU1P*S_U|sbY>Zue1B3jTEz|v6oNr11&xT?n^fPn)(OW zm=VN}zL2DeOTq>43q@7iMYuD?X=!o}w{G8<#y}+({&slznu4Cf%zuYy@S zwp6=n!MHzrMAQHDiRWG_weM}vuKv-Rw+$eg0}|T^=V_qOD}QL9Jo>?H-^uP?2>VCP z;9KSrnx#fQajR@03v{=7ROO%v2K-~&kT38z<(HD@X#R7KzNF!$FDdATLjkP$az%H2 zt<~MrY-&!LVb^l6!meTzb=vWF!g1#XjremJ@jG1g1XMzrc=ibDFZSfD2HM#Z{kN9C z9e%H*YioUOl2tHeZK`7R1*MYK?XnYP(e2?pIK_|Z8h+pV*X=CJ;k^F@)sqo4LUrWz z*4FPkqG;G&GW zQLP72LCDixoc|rnmsG3JXLJV=S^K@=D$oUcKBdeAgAp+?Okf6Bi%x$@7&ktLY~w_m z=vs`S)eLUQsY29cuGiDc&fa^B)+HK{Gl;2`L%t|k68 zu8-;j0EAq>Fm%w~**)2l90*Sqf7)LD%u1-a$kZ2nW`Wj?t>*M2ICQv%m)rFJPwc9F z@i{TvL>9>`@!WCIDy5q5t$t8{3tG6@Y6hYDw02ii)YCjdA)>y9vJCiXtrYs>0d3m!QmaY%5rU|ZYigS%GU+(^w0MdJbjYPm_*z#Xw4v%4GFUH1i z!~&ZwOnuoKqhEH?F5akMZZEVi<@_(_mMV z4=3&a2f!x<1(2<)&JHTO{h9X%UQbPDBFS0&=|wj3WHHz96@yFFN{g}CcJ-`GTO+r5 zC2uS~KgAQu+eqr@f1UOGgx(cu1%*p`HNY6{su`s50WlszrLl?n-+;3~ zG5Zxt|IY;qgXz70NWL-!J4VdHBpiVSr=>i~;%9iL|FhD#yj#)t0~y@k&ti{M@r#O5 zt?KWJ%UGK^pt5pzFqsVPx8i!1rtbq|fRD`PThO-j#=4Hi>-r#{g)0%Qz;G)&HtQ_V z9l<0A3)$U)(knr71^%7_?up~rO;DKfdtS>=nfmK7NALr-gvQ0%*w)?rC^d^RB%PHE zr9G;E)ni4CXUhZ=+zJ!O>s>v5zPSy?Xv~qaIR+w(g8wOlv_3%dqFG}j_T9u=+w{jA z)x{S*)VAE;5B^Bw?v?yW0b)+pxs&CYUPsUTE9agx`4jUy?9JbtFp#BbV>yb_>l*`; zL-O6;=Br$~n?Mv=|AHKwE(su<22-i2&rGME7+MH{H=il2!4$>EK~u>0bX4otsN;0f z*eP^I$aTof=)g_em7#(zw_KGIdufRA=dnzz;-+7*({Lfb!$9U^!6AvV~-;bG<(5IN?;bMRg|=}KWgP@Y7um~0=)18E`%NO6`JnK@7^#s*5QUU zfl@_tU~Vw?nfdbN1&$WsJ{@QI_J+^q9%Z`m;2o#Z(vZVA-#P z#n%IG3ULrmZ%z6^QXgQudj8X$M7-lPxpDZ=AOTZ%z`ruO)?xY$Juygwm0v)Lf2WbW zTDrQ{B-jl*`od*k2^gKpMD59=AF3_zaYxFpi&SN0F;g+lcWku)dQct&?pHt|2!lzh zM(fREnwu)6ejnRfeyxibBH-gqmKM$WBQOO%crgGmUVM2?*HE2b%iokdH$k zb@0bq7{#w@4gcB>_ZJ<43x+K!YVV{_#PN^vCenHM=Q z8A|-0w<`)xuyq}dlsOF?7i#Hr#p5Vlw|JI$>k6H>9}n#_!7kT7Bg8TmaYJek)Iok? z!pvAgVgzg!2Y}f?NkMAp;0=sXi_=BB`O};Nn}EFgKNQ5WL?W>%V22n^1?4h(_9hUa zv=vm+Hgm0CH%W--=*uOdN#Vssz? znFUEg^eKuaXR;X*WBzzHQz|2@ln=8QEB`e&hW!KlQc>QqzR(ocnFIb#YitYJihLy~ zl&NpZfl-T{OE_kL34Sm(vCMa*i=yfBK9a`>uUU2iUOo%D@%nLzq2edF%_lTq*(Slf z!sWvaK(b~2H%vHB4tHmVuZy3@!aapceXV#3z#xaNf5gS`slXjN7o2*QsZ^XbgU6MH zCv%qSSZ9aR*MB}OPTK(WMAkFsui7UtDIi%;7h(0QCXBo-8A&FR1oo!b(E9YMu}q}* z2AEQ$)4d+P%!8vfm?V3!fK%goaDbu-B&O=@&K~6FO-zs`U^j&+V!Q4EZsm)vCj`Zk&i}{MSI0Hk{qGM^u>chbl@Jt=PDMs4DxK0H zT~Y#Lqr)JTkZzR$A|+BXQb}>tkY=O|M+|VlXntqt^Z9-6|DH!*?%e0R>$=|8``p=i zb;d1q--6t#T%i`#b;7qiHxYY2q#ygUb;!RL20+w|R)j#yF^}c_tD`IsF@2Q~4HjiQ z`o>9uqX$f6cPq~;1RSis8@6cNg5Yhw5LkHm)rHAo_oX z@815oH(xcH(skZmH5W=QvkL6}+`(rN_P6Noo=ZoA4fyt|gP9B^gK&zKVsCf;H#HX?Y|gmr7h0jv@_;Fg zKH^17?X~Xmwgmg|x7S?HcyL&cUj8U(Y5znK+!*gNmaMlwx4GP`+K=DY(HY_z+;-1( zT$>zIlwe;Kf922FVFNlCXC%P1(9seeJwH45>#J1{%cx{lI9S5_qRNQ8^+Wp=4an?= zAF|PMJwcmF@p+5_#TEB>6e$Wkm~z$QokMM6mP(F!gg@)xNtTa##_|5t%y#6dGdmwo z-DLd`ED4w3NiTd?r+=cj)NtcLv`2zMcM|y6!QnwZwMxw>&eske$Tqo*&a0bKO-E4im)1(;V1`e|e)ra#sprsO4n5wM zlv^X`aMta~`zaPF3UC3;#Jy>e^6fZ3vH7)+xYb{GlWbeNJfGJ4JT`XDisXAqNAbAu z*&Y9}uit;wu5`_psmL9@JOuLU&4H`^ySdx=$f0lY%)yE*@U3aFu7}D9HSuEn3H-?v z!vSh)&zx!Hc4dTjQ~Y@2T0Z8&-5J3Vr+my%TG*&luE`4#K94Y+o>-a?kv7Wo@NXD; zx=0c7M_r$V3Y}gHn;uJc8KaTx`^_GCuM+26-7Wn{;1*jpm&xf*t%3cLOAYLc;`o1} zDadN&B&l(=h?}wTD|t09d87?ccaIZ2F{g#g-cKvDNu-Cv!o=0rvE5eg@i4{Sb9L+A z)Th7qmx$Ls0xQYXGDtp9Lm(C(Vc3P?Ct2*RJKmI}7>~2(y=QTAX65HVuHR}K`NnnfGkiJCp~*R#&^gB%=jtPNclYw;V7EJIxQ`Hx z(~>Q~h|aSV+!^st)l4R=2AjhW)iQ!C7j&O1>@CMWm`}$B7w1+Abopy|d2JZf?+DTi z`v?`8kT77TE?10;spk!uOdGx|Bh%HBrXILe)rzt47&;Y3+h@#%ZdtjJ_Z%HYdKb)A z`t8f`KpL{-))=kI|Mn`z0a`6KInI~4rZ0S+YDzm&GM2AJQhNOrj8g4fZUVikcxFr? zWtt*o_JaIj!u@gEoIoRvs+J_dQZiq*^Nc*~q>do`LBqSvuvM|2dsUDLL0J z?Rq9ymas7&ahn@|OTlAcU?Bc|e14-sc_4SZu}v0{Mwkum=6RZ1Pxy*jd+SAfdhMBF z7YzBMY2;^$(%5_id|fCD*0yPJjixC_94{joT1cs2rv+r1c0;oE5QUqr&E99a8t;4u(L0&-fE{}awI(vTXJaKV0rqd9Rppi89bF^=gP|HZQn%_?RFdU{?{3j>7`D%F2i1qlvw z+_5z98?(h+mO4i14#{fuQ)N8$l;5fU5$1xh$fY$8{7IqSv@&Q06^zlicG+gLVfu&m z2?n}l^RX0_RI?%)Z4UHgf#2n-sdFukz&0WBpNd@Ff+m~^mmBueUB=iWAI?ifj~ejI z(R{CQduVLoN^qhk@+k3je1oT2x5@A!Qume1{c2!;XZ?Ve#1H;Nr%U*wh0Z!1ZB&W_ z8~!APCSz3VUHmDgAUH<=6Hn!RltM;cn2$yCGUl?dTK_HFSJFfLImgKDd*9?ud3-dW z7WYKD^i|5tHQKcxRbq`+&r;{H3se#WhQTB&>-fIxc=1 zm;UAy`nB0V&Luxci@&NQUgK|whCq6FR&mtr;6p%^Lar3vS}rbd-w)61R6vkYuXqfO zFI4}D&#K_5JzwKRH^k2Gx9@&YJ8xZnz}H~42~X6x(CR!Ev2?ff9x(vJBg~bRNRwjG zmzIXCRT-bfu!HqIo@25juC5jnD;rlz1n2MPxII#+qvCODkn~JF{1{S$4Rf=lGdM?w z%y{qrawLnN<*$qo0?EQ?>|c$($q>u}R7UK9KU3FnOWFE!boveT+(V({BfirbiRkOhVoXJI`tu2<`F( zR~p2(QPU=C1C7;BEum1qkaC1l;eNybho|r#jjnaU316N;3zmefc-xl=V56y!3ja)A zK8XvJxhvt{5!3uskQ`0V(>1E1?mpuxI&ZMxJeHN0L!UpDe+|hJ*1*X^Q96XhZh8ms z5_Q z;S!M@_v=v03i_&w=6eW<*^xV*ih;N-r~1Eub!mT}aBnO%diI=h@~cbW1nYR-Qo8Sb zf>FK_V&Rd?jNr|cl73rX*z4a2J6c3@Rz=&y`A@vvLETJ~}U zyUid|I3Q|PuXw6$GrO3n6DIjON8s>ix`7qF7F9IXSRM93()D!IgKvD}jiEg$%WFs0 zN8Xof@{2-n@S$?}7B5A^OJD%BqA|QHXGUpDt1*WOg28K#^z`UUG9>4v@iaoc#^%?t z5VsOThwaSFPLTRkb4_lf&cDM$b=*IX)a!#4-Ia==#c4fVPuH$3N zg29qvj7DQeu3yCAF)SB<+P!8pRyAxDtYvk&O+-pbPnBYqU3~DThG~{f6^f3@oS(NE zoaf7Jn4O=Cecr9zox|%Or)cLJwvE!mEe^(!Jv*2M;U?So)fpPnCYjgw1z#;D^DLnS`8e=tQ^eN%rt8k^-- z#Z&=odbJE(yj3wWzA)iDMuT^FKuyPDOJH+!^|W17$(D4Kv0a;4w=LAKY`BK~&;Mr? z>56fI^B=mc#-RA5GP38!Sk7=_sP7#_B$}!uv$#dZ*|Kn8*}It4q<5LA$8TUM#tirw zs21Z>7tz*#40277Jc9&bzLPWlAFCxuF#b4#M6EVHDFT^7bAuetAqnS5TjS19Y`V_E zURqmTYtjrIT5`}`?NW!{W_$W}oqm$K9(M|MMd`nk()6VlcUTgzL=n(8T3Ni05wWHQ ziG|gg)jjDG?8PZy9+0*e{lPyLfo5(|xArZzMCGR_6|QdYzB{v%F;X1sdZE?Sg&Pw_ zf3}#Ul-~qCErYChG@^(mOkf1)A_kH|zDz11nu1cS#~IOOP5l@a&K8z~Psr;K$c=N2 zsUuZeK2GM>Sb;Xx6x3!Vx)^(uIf_97J_&b!1V zAFu_*i$`u|VMQPHfOOG9PFH3BLzPIt3mw>L%Z=d+VqtOZ;5f{__--sv;QwCx1U9Fv zB!ptVVuHT-kefwRn_TbljNdf7cAwVOcdZAMFPkvr9zR#w4RB&XztOiyDvC|vHO;T? zvV~vof<4ibKJYQs`5Ln>_v}l@J33oL;+-RdKmK!7Dzn;DV?e_$S~yI>T)h20l@emR zw`8v0e#Fis7U-Iep5V-!h%75bbId^BX7jb8Go&Zv(+D3OU0T;VQEvi{!{>b9r|N3b z13V3r$yvs{98bm%o=B-W`S|Ap)BZ(;y~g337y@t7u5z`K6==-zdW4Sbr@_uH-RL?z zH6_hlHsLYy(_9%r-uF7IhB=yCT~9M<&`d^k`a4v?;&hs z%FmxO!ZV#dT8&nX=bgCENU0mbeOG{K__>We%9P(?f4&&e&iXbR)B_!o(K(BcM%+@{ zKLQsHw?6RJ(;JX*uelNAqwRL!$N2!16lhgM- zy98oOo0r7%t|#c&(+2&wH%-mr+VkhB1sA{19o)onoCX2l@mJnY$+Kd#9U+0&1RtpI zL~~+(fVh2k@#5|AN>dQoo@UjNL6$84n30k<|E(uj zB2RhpM@upcmqx@0JSJxoS}hf$vindzEc?sKgXbXK3J^CkuI*`k9cXm?aCj0#^Ljtx zHuc7LG$~>`9wUH3hpfK@0agx`{Y>IK* z&Y}4;*){un2!!8;LYWlS))ui1ET`jZ|5*0MWYd(*s+U>$nIywbE#2`HU1ubIV*a8U z&YS0ZgasgV5z_;8LM2fW^a4LnsmJz4(VN1Kh-(B>L*djmm2@P@pb@)VXae3|vjrX&-3q9}<~0BvmHh zm@LU4ocvgD>^F}$?Eaa`ySjq^?fnqirU{(;+AD>H8~Br!z&6+|=_T8pIWR1&s}`Dy ze1S?@?<;ajEn6kP`ugv4k5uiCkP8UdS?6rOH#Yg77tsW1kje9IHOBmn5>g!^Xq>ZD zs!vvm94B^`M@wF%eu6X0Pl{Q;##gpLuK{yxe?~Ami^gdB;a7Rco zy)45%;?AAaqLl7;Vy6vMIm0p*(IMYsVN|bT+F5@#rql*EK7r3GRV#^3;^Se+akE>q zXTZT|@5mA*7{7$vu`4mDyCY*e@Ntk4X##$^NiK3bNppz1oF2XIj%T2i>AOOxPSGT1VuWq2pG%aleS&f<3kN?>DGJK&=W42JD)}lw4leU zdiv^&Pk3F>)NiK)1wS}>?waHqWiGqAk@`z6ga$=a``HSeu=7WMM*hKv3vRe2)bnr? z)_^8_%X_+6D!F2+^A}aQNV?1#N(dz@50HB~tk&{Ksa+Eo7Krd#%1_bUpRWnDXLC)a zimCf>lRS}1_Dah7`8r}d9yLKn+){Akmt?A9Igj~_pF^4WLGa!!@LD=|v#k>g>Z$YE5? zoyfVVdmT#w@u(R50rE`Xnj8+E0DE!V4lcA31wt&S6XFP)xWOHp&69ReRn&uvbjzg) zR&^;ymzaY5!G^oBZF>c?vh3_;bRB!l1URI;{M@S%HjbIg7?wBVoOg4tS~%ZiF=kyF z(STUsLQ!pb2sol*A0V%ZG=*+#q^>a4OnA~@Tlrbv(R8+0SMu8g&2(KAeCP1ekN=hl z>Ot?X(fd|eBeER-A@?bIHSq}kgI3GeuT_t{9YqDKNO*c2(bqPEC+12a9g7W=B$|E; zxO6OCI8CHNN?NM@|@-nPG>AhDe`bYP}~&?Us;xWTT<$55cf}+Zky3Ap_jV zH%>TH(OZxrSYT@lh00S!)sC?y3x-IL*6CvQ0zqp#9qpP5i;9hB6flO3q}_75eFSEM z<-8a@k0k$GcwmyjWCITbnGo3o1zP zFK@-~TEY1fw=*~na$!--SwdtSPJX5=_d-zKJLz@+uD-bj;ziT(k5Lkw{6BoF0ef5Q zrGcV&C!CP5kWVp${|8DJ_4RDZ$uZ+`hV_(09MjBgKvn47T@EA*!k;_>S!3;8)2-`~1&C>DW@` z>TruuGw5IcXfjzpOpVWN&*kHKF{p+POO;WDwk{g3B`-co^~$VCGRZZ0My@#Mkb+t2 zCJX|JgxI`W`}JbkhlL)2rWa0YePnIRJ$CsYcqjI|tMwjg3_ zugh$35hebr+E6ZsW29%)!N&pL8kOJ3kDWfD2FnK(R+e35M(%Y;Q#?9uU$)GcMRLN$&F9B1{n9I;~%2(H8}DpbEWDml$_2fZJ7q} znl#_nBcHihT6G3wdTpY)VsIJF+yZX&OO!*1TM4y7V!ce-YR} zin8N3Z$*G;lEY|o24|o=s+B6wn7y$0HmfRBEPng3J3_JD z_o!4_lLAzfoTHDMlz{;lc6*k483>HI+rZ$Q*1Gfh`bjBf7n zBq_<={R(O_AEOUw8I4+bC|Mk}SCH~!zgfd}Nc+4wNn89sbA;GzY;4p!k8SS4kTb4! zqabhXZUt+dzyd8cv<&Xz)Cf~AAUofkk51?%r(11fHatoFEE-mGdNnmzJCEXAsgbZ+N1&coxICdh)iL>D|6F2xV|x@d{CK@roFk zP~A>HB2>3%>5AiAW8X5B{>xZRL9n^Bl@d)GO^ye0*G=7>Yg2rT0z}J8#3uhx&%;NS9K?a^cIwiZzM-})8b1K zz<$Tl&j_V1|0PSCf+}*@toXd=JFmE<{C)4>q2K)ClMJf42m@q;Aa;fdy?2Ya?OVY& zwBreW8AQuqcP-Y)z#q;yC4~1&E>vTA?;k0Ee`{)p18a7~+W>~FvUv&zWx0$^@-jp{ z2d3{^gdb|Tv>tM2NE=?qqwnmt__zEE!Vjgh@#To(U!86@WdBzrPskgTY&0(E!W%|p z41nt;aCkj3 z)H#TY5|bZ#XLnK2?i!4h8d#zMmLowey$l5+Ky{7;<)8mZJG9vm?!DVm290B|OS zzun8vD=G+YbZp*N6XbP^Ukfwp)M^z<02Az~mK$p@g%gnk!eIWwqY+l(9djY<48;m2 z?oWBwrU^N*f6~*FmaW7@3J&!}+7@ld;1t~PT){&g=KvfYS<*;yO||egBTvK8)qPg~ zPYVG4b0St!j}>|+b+1>P1|h1Q2ttHfKeKE3^7l=fT-D@>O@F%4Uu?@PsRy>GOJ=va^>u;h!!Eg zvoG7pwfr;m-!9~Y4d2dbPsC2rz{k|93s?H7V2Fx?2X|(r@`^N|~O3 zeYI4!w!VFZ*TlA)os>XWgAyfur5@-x=}AAXMb+>7;Ini9Ru2}rsEQQ2UR3f2nuezW z(Zd5S>T}JVsZbcw^r4LFh`8~IC4Tly*DEPu&$#*}FI4i6OSy~=h+fyi0f@CKn64xv-ESG(C&^k3QAG)9KqY^pZ>$PYakw?G4PEd-(e zv7;1JI|;^QEj6C0v6;3W(iWWA$<>9^!PC(C{z>(sy#=@whZo5gPAusKK$zInR>~h0 z)JN5f%OM|Gm=J6qb(Nvdl&2@St{u<2zCbr(4L#dct(*^m9De|CsrMg2PmMf>)?AYb z>Rm4JK4sM_SLIe3QlN1GkTL59*BlHrU6;Qc%@GSjwmgy-ecDQ?Dl(=bfmaLCla07i zF5@)8D#4qp@swFnb$;tI01pp&lGF$Vch>2M9WM}V4HG1hnkDn9%uH(@-! zRgJTcc_J02HpC_6Y;~Bzui8Z~G?XLD%BUzAA&^;BvTf+F0p5Si%srrVd7r2zUV!HH z6w2B8zE(;(eA$%C?3#rZ6fBAS?7ge)9c59?)6^?+W#@1EOu41twU(#Y)%^g&85j!A zQTszdLBToAEO^*RCOdZKF5pquNx&m6aqKV?ZK8O<&Pk!U_40MHmw(J1aAqDGyW)?3 zIo2i8g+RN$gheT$`O7<9ubD#WAr_`LbH51o!PJ2F`*4j05%i+cp?vl?*eJTk#2mAcOD#_Aid`_u*1nYGkkC85mXG;VJ+j#dWn z0v=uMUm%-^jgQbsc3q>i+p*(M^eN&S`oo83kOSmS)2Y^r>O(1EZ(qT}45%fq#COW- zOL}f9qQRFOsUQg)0fcWa@rTBs31yBtUX`2bD$4ziDcq0V^Rtf^f86r&-(UurLX_FG z>7xy9<|e(`p#_6TAooJ+PM@u7@;7T#2W>EpiEIA~J;*`x{?+cmxIaGVj5n@p6t>-s zLC)>zgN$Qi`YP>P)MOL8mel#!nbo^Hj)jC;W&$d%cVhi{lGd(;3QhOLWnjOb2DQ zal+<~Rov0j*{6j$lB(chR5c+v5l@}}szU%GZ|2gUGEBW>B!NypkjnSfdC1M65C)wFnZ0-55K|v9Xyh|D0&d z98R8_A3MXT=KnrZ<*+Knsa^wh0@Ac-6|r=ng71l+v92=p_uaeM3ai)o*y*0}&V7#r zspEVAdkcGhoH)5Jg#SBK23aiydY!TzZ!UEPQW{XM`1xBj@ER1~x^>HoApZ{bijAB| zsRr^z+dt2wJN|GAT(tvK)8ixKqI{TlJ5@%!MVA}IkldKmWI$p_=Nmc)e$%iGv_1%x zy@s8+mG)C&;UfMP_T``}nXec6_}ZTF1^-Tl&blR0+Dg5pF+PH}L|r`730|5S9>msk zbyl9Hg&O+mKfubA*u9x_3pA=LxSlj$-Kg#f5a8Dp*#H6tJ^VnS0j(0rn+4b(T%JEs zRwHglWZKWpFKcZ5S;JibB?Vi2B^l(82cLfPDgHZU)b?I>u7ZV0bl%EMu5&H*p?;#S zWjl$PDlGcwxdV1A7#X53(_L6r2@|a3ho&BX{KaO5&$$-gZB@4XHR1q4Ib3D-urKmC zAkm$5C^i3={SWCqyZEfWFX7R!>(f*NW3A}d@%G&ZAvy~;$QBioZ25)CLTVu%|L9WT zN-FlWPgrfKvcW|70P;BcZ=t;%P5aGMy+wX1d{64}?ITz;F7mG1y9)jyH zPk2MwiRNB^s26{?P!_eyEdVe5f; zRaF|Jal>s31svPd`;R#3&C5psbX@aJ`CYWIupnfaE8u2S z@4NBP^P*S%_Eo?ewAOCzUn>5h&Exe(QX7GE0`iq8pRBA)nWD4Ip50H&$S`NVE5$le zd2}W_-E)U@DXKLY>GyVh1m5o-X&k162)78-`bQZ+RC<8khw9_Z8m;cf-zrv6EdY`D zJ);lkq~c`j3?1qfp>0m&)8{R-@zA6%{_;Fx0^~jJwDrnNLzeDpz+Grp%=m+{{ZvtE zA5TAauG9?(?Oa^)zB)BuYUpJVj&)u8L}#_O{xJ7vDxr1=X${tk0ar+Jh|PD)2|#Vz z?9o^((hvT^r+aiVW6abIu9@H(g2``QFr4^f%P4Pr_~?IY?O)j3oCk;x))864kzL2d zS(K6Rq{IoWi$GNDsYBaT$?QszY^tN~rGtWz*4(e~XI#E;2g)FI6y-yWFurJ;_uC-v zoVW1@Ug&3N`@=*ZN7NiGSW>qmso@zkt)bVGl%#M@Th8ZNDdc&cP73E-kCzn%R}1&N zk?Wz;_@HTb*qiFs0K^?C$vzYFfI||;)9|EiP=ZqU=+Q#?j*hXFI5~F$%s)1s$&a0- z6e~VIpCo=;u%b!>0*Vgy<@{pWI^YqC~e-ayi2n z&UW6jEv|BD2V{#XM&xm@(3Mja3!$MkW%ozTwtqdvGi3Y1n)@|P`B*5f8zgt=nnX_W zfdJoBlynr|5f2s*K|YYPwKDZ}t=+0X-ABFu_;CSw6pV);kfoc5+``c|*kH!hps(Wr z$*>+K_sONr@*VHIyz+Y6*RXR5NLviWvvu73#y;l4zunw{5(R2N1;%W*17e6Kvr1p& z7$~$R!m=NccdVC~nXZeV77>O1AQ5Lgc2vNCeLYrv~EeKIt&?q?@R1tLL! zFQO{tpa~jY;>9zRTECwKWO1tRQ&FF#`t7}wcveX8as(&F-Lv-F5@6VEnbmBgJVBRe z;D~o%q0GIWq*AY_{XrR}(3^84vS9g3xRiTdflAjA{8j46d)e`>=QkCPy7*vYPbHOXR;dD)^tVE^QC5XT zXaZU`CIu_s7z)acQxx}{)^FaihdjL=ZniC3mKk=^;l$rq17R*$RuzjDbJ z5@awLQm$^QI(afomXXGU?OKN1(s{0rkx#dR7OtdEOdj!UtT(Ck*0z2ca_lRV`z656 zZW*|U`Z2kj~&;c$jQS+BYiDkOD zmJ-t+nHZ5Nx5|Q_wYuk@9R7aVI^X<@ko=VAghQLS(TS^ zn#^z^?srJP7c4V0tpfwO@mOrJrgD4mAsUSKirJNGp%g}O?+zwa{?Qon!ZmKNSDf+c zk z%d~(y?4WKe+&2YDB3MG?mbA}kf!(e;NWl7K!<%n2J0H8jdjO=lOwwBa3@n9ptV$Ey zUaP4}6(W~Xn(TO~;7gWKfEMb+9{y8Xn^~uVtrOZ6q(KFmvQ9lW7irdZAAzM;hP)fB z58kPgm0s@}==oR^qyKJx@YF)*{Gnra|D=6)KbCJaQI)hM|8m>()>whAmq_w%7z}+M zpr?cc5LTXot`2vX*;~O|2vRJ8=WdYS9>x43`3S^7Req;Dl^pO^8M+Nr%ogu-$AWUq zX?7})53Gng73&DhCjsfuTE9`S6bAL|st4xd;uQhQq%ZiAngUODr8q?8=geM~N)$*T z(?Qajz!#yUoT?EJ%sCxW(x4Rvc8Jxci-&Y0{qEtvEs!uppKG-rjZ~o>)w#3F))6 zGyTH=sdEi9v);K`UatvY-XpyCLIo%jBc zB(b$5x_dTBhw*}-+iB}r<8SigS5q0^YnhwX;lB_ztDeEmc_)+mnK4~Nk}N31-c}Nl z{zE!Xz2=Mw6_4;|(IdM6K+BdPUNm=6amQ~Sh1`?~z}>7?F6cgas2kS()}eJf(jgkA zsoW22a~+un8r%(Ip>Ecw6m*u*zJ1Y3wtJ%O%ML$z{Gh-Eg*RlDqlgAZNZWD%PH4%?v1 z*28ZV)f27r`+Cb4!S&;YlReL~6QYCr$`igVJcUd#hPCR9aGKTFnI?U=Mk{0TcJA-o z(^(h-F|}=lM*BAjF6ucK)l+nMMnCGTAfnPKG9f5hVB_>RN$4vP01z@ezpT8ee~fC) zxksoC@0$PJd%&0_Q~S5ay&An^gOj#vFhn&gu=3(O4rBkBJH)#Z9ExVkMZ64}pbfS> zE>Eeox@*%K!#+zbRQ24A|8Yjv#mtY<+G_b_tzpWf;}NsHBLOeR8-d0bM&{!T*%QgM z`>=N8?|bE6wOYHSM4&HQK1w`S&gUtw8k_{W|xvy%j{btLSUN7;JflYeZ9T^>wrAdTUhRkHazupNVTKk8R z5?OXp+cq<`3PV&oV@fxuvIf0yCMB^NM*=oYFfXKSlZ=JCX+_FOJElRu`JXEMsxGX) z-d}Q0Oh1^1f)^=xjfAHLa)AkRkWN8NUp!O}Yg%;dnoqY+luP3V!osM@)ShV$PZLM( zatYTfqpUyQ2w!soN2DYvxu#fP(1; z-zA;-bb+iW=?`ReZzg&#S<-1I0j3&ZL2P^ksx`$S;Jvb&n)}6-PA0p#ITp?%RV-v; zSg&SB+OkqGMJZk4u>v54?Mm-KTW9lx3(}WtZr}P6*G<4X2I@mkaV7cCJT6IT)bJJ? zxh>xJLr8}B%M8@Ve;>EJ+S-i*LykVAZVO9fIuEF1i|7M-G;FhP;?h(-^t{itgm4dk z{}q^a&)rT#ZT^#stS%S!IAJk!r{YLwa_ZIN7x(Px8cHEeVJY}uK=?MlO6ln4yOz2^ zn3IjMelL@2qSA56Q*Cbm%dlz$7xgnT@LMb#rH40Fwwrr_SYJHJTU7o9{AOzV&Z+H|Mluq0P;ri%V z)kmFq+1Rc#LP{qN)8lR{quLikOCc~57tyPW-z5$oqwI?SFyqS5iFCT!zbo?Sb&Xy+ z?ZAaFQj>|B%)`aSsO=;M{5ht2fXI~Nc*MHfNKBBepTD#u1JKTmKyF_;e_$b=q?G+b zW%{DAweycy_I?2zare6+ho6t8;^K1jtOH(f*vIR&dYB>mVpOCg5B3;E^~knR2=bINg?^(;={5tidJ^RE zX(Q-~$EAh6qoCA0_$fWaEQHm0KT&XLpO-;%(w@8DeCbKg{uREq?dOuTql)go&gyqS zqo3ksR)fjC1fRNw5r2^^&=3;x=YPOsng2OE|A2Z3^ycp3IhdqCmNdKn((P7lEl7*Q zZC_vaOjG}zzk^I&&3uU)1N|J@^4(higQT#0zG*)O_u?_Td!Eu=n2W~Ac|4c@(x|Fq zWs_xPGRJ6)(uzT{l8|_#+4ch4;S~4XtZK*blQHeN!Ya?D5}q{BCWAS#j@PZG#CFhX zhPS+FYa6kB_c`oT;g>JC?r|4d#M@sC`19V@~)v1{22G$tb(9n>aYu$Fw!aT!3P0U$JN)gq}ck)}h zp7%B6Nd-z3sp+S8#Kpf`3Jyi5+3=Ly=;{~myTm2)JjE_`w>GVKciVIFu2iYCu71ND z?TkUGROck*ryDEnWhg~I6E%xuquhn@`k{RDq1n8l0Scm{-agpa7q161qT z6dy@TFzh&vI0yP_k}V6j@kx;3p$s&b)Ymr!uQJKfOF2XWavxBvUfN4Ac3jm5qjd!bL^C#O)%VTg;C{plcB(Zv!3RjG%?~ zZ1B!WDhT8zCxBWS5GJ8_v4Ux2o_T5#b;f0WsI|XV?2$f5*h%;K?8iSQKu5_E#+1G@ zOY+QRJ3NCX+cBSZajPG8Ph4OLG%p&34sUkGxNjvUVpAU1hv?)4X`?I0ubEL;x9j*y zHdt0;?l%fnE@mm5KWFNam8;3+f8EW9g)a#`Y_^^B9(Ja#%&K7{87{i(Nkn*Mz*NsW z*Dyngx$M!V&ax|rCz#1aSGU@;$&WG(Rd@=kL^AawyRCnGvn~0`wu6tt{hX9(_z0K3zXS=RSkwxrB&%uV{Iv!N7@P%3Y^AVjY&=+vH zvWJ3EHg%KHWz{1i(i_v$psS~aW+d;TbWt`HsAn7sMl$)Z7cVU}RKLVKaLGa$H07MR z#UyB{`vkB+Ru}KeXKSK=HRg9edVUW^+@3T;QCZku0wF7a7M(WBTZRkx{ zQD*1f8|H(H`mcz2(?4>u^js6oo)8Ee*jYZ~!?%E;SF!*1Mz4O7Yr?T^J02B)GPcQY z#v3=iNV zE`e|8Gn=%18@Ejrk?D|!&cwFTM-_YgL@#LtXzuZwL?7wFJ>OS8OUuITu7?Btah}!2 zQ2yUXGa1_vK*QaWTI)B}VUtgdvD**8{}TlU_jycSyP|57Zm4~vO9UO91FHaz@{;mi zk0{s=C9=#@ToU#Kp}1%c$ciOh332r6Q10nBUd`f;MKn|ma|iO3f^mp)~bM|)TKCPP$DYWo6cK>?Ks7?h}vOb>hQ%^!DQE|u6Y-TE%XM8CRP zPKbp;9I%>miDC$um+$ErMJKjptZ)|7o84B6P*9v7J;7yDf>m0{N^FiidcuetfD@aL zTU0BOjrEL3yu#I^a5s9lSB^T_+2ZdXpH}R#selDwPnBR|!&Ibh0~5 zU81FeZenLHaD%Z@xqd7bR6VEaY82U?3TD;H55!6(ocNPMBihK^!Vpv<*th;A6OF6L zt1JJG|3<+iyL4Xg8P;x4kJ zxZ*uQ>rXK<6HI*%`XO^vGF>FP@;`X#&w-9Br|A{1^^;-3!vBOloVow&UykebtUgrL(OL7^D_o|pVC;i;5HTvMB{x^z7 zKQQ^SIDW!EaU}CPmZM9$O8qm}nxq{G2+vn>WUYG)=S4Of`tAB{slo(8$f3ndb*j0B zrDu6(A`reD-XekTh6FtCZo(dcYY7c;Afs|}hNfIDKCgOgrhKU_&v8kF{Uc+V3>Q1A50Ic2`8)#Ttr@yU6&M#ll|>npIQS?*?d$-elI>kk@ia(GVs9#Fq{Yy9RZ%fl{@1h@EUL3bn&` zG{Icml)lJ&>ELm8XNGc%TpW-TwBmxO(w0M77RF=gIf0!Ey#~66oyUrca_9P{0_tCW zv&Ou$F5qR&Hh#wu_BjtBMw|iUV{~XtCOTjm8XTdAzE%1`)7DMGd|Oea`gl_yP6WrB zx60UMz5K-Uu<9%Oj+dq%krHDWNbH;|>WUxk6Pxcke7oJtFYOz1Yo8PUkJ}Gk!=ySj ze3VVh?75$>TD=vr`*2suuo`ruZw1r<{Nx8GkLNveWO0FiCNU7N0xx!UF`B;72b0zJ z#&)u!N(v3h^HsuJx83UQD)I?h78(DN{nqvWCyKqD$RfREU3eK;ymA6QTqhub%ef@R z6B(P?nL=lJ5I+cO-0q!ZZr@JV>7L9ot|m-_?>9In(FBSC_nhuDgOB{^o2I<9NQn+= z4#ZLX>Q&)Gm2SIibWUH4PODRNYD3~BR4Sid?wE4x(JUjLGRC(Y;8e$=*B zRF2nTsQqB6hXEm;X8k?6+6A-g3#oc}pp)K`M6YRCw^vr!JIMPV_bHt9C8&AiQt>c$ zi3@467eX7Z^_ym_YW6|)-X05Zg|EbKy;EAK+TkhmsPY z+9D;t_>XGXx{21>t5kbGH_l7gkrj+s3eYU}Tm5E4#Iaeyla;EQl-AA>RbQoA5oCz< z^9L%p6)#{wY@A#qOi)a09yj{HSA9oh`FDQxxc4d;ho5Rhp-{J94Uh`ANyM4^R=y6s zY2Vqilxezb=FoOJ#PD@Oo;5;!Z}Yp@JX+$=+t4M|yLmW0=3E?HFe&;JP{1sI1rYnp zj6kw|*y(nv6muGoCH1;9h;K4GVUFr7-Ld@xeg+3Ucg+T{W+wy(Qv&ntT*S&*GDUMH z{J;1wuFQx8NISOIFt-o?4Ubw2fLXUtSML&;j&zi1fyI&$qBiw|A3;S4&_UW#Bai{E z$&2>|`^4_ZBNWH7ktP=YH)rNB)Q{iY+W#c9lFeyC0dHw)>Xm4=Ts-H|h^6e7&i*@Q zjlGR$@OC1cu>xwM_J_^!f%XK@aSsu(O?7)nVu97V)sR-`htP-PJmQrbZ0vk0K&;fE zVQVfa|{pCITS148bnPz9hwP}PN_BOfpc~TMUetQ zt9SCK$1X^%6ac2Q5-_Toa)ta;MT3yk-gBLS6&2@_P z$9Ud_^z_96BK2OHWtah>adTE{r{ciF;N9F9=JMTNkMsI9Y&0Y$5v1P16@%52B$4t6 zF!!ii(h1laRH4U&y=xHHa?xd`pH5jOhaGbD{-+lW{N_9c@4wp@?>kW2(;#L%Rr3B* zQU6`0ZtRbyr6tA+v6@mx^`pzDqVFWOTMMjIn*aHXorb@b#n zeZfV(z+guVXy)XM3+A>`zv|-__lIPbyJ*P?lnNf&a5o7wLmA6mxIa_TpY0+k=QnM7 z!7GYCYT9&LfMJkoNH8Ow&8xE7jKl1LP3yfMD}d;76DD$lSpfB#eE+kdZi*N9&V)HG zauz)r2}8FZSh~`t&~cQnNdNxiYq&uxGdOv;w?hUV+#J0?)kXOP0Sb9!A#*s#8M8x zdD$vqb(hI68SFE4pk`mP?Mr@3a^taeItL8K-RpmwpWjFEFlTsT(3Yn2&;m{y=el;R zxWcdCH(8Swjpwfzw`vHSw1f*Kxl%w@gy;FHJrCRZ%Xs7JlCz8sx4+k-C7-2kg0zx3 zQ?fwnd^^61Oda(V!GCzr_xqe^Poq%kO4SDLdQV!=lqB&rDPv^P_p+0RkZrNV6C8!d z`ZYdyMIJk`r~F+-P8VAn`=IZe?L&d<@M>MlK)pgv4~^3!E5>u z+7znEkgpW;bpIn(bijGiZxo>0!?(f!L-)1lw_=%%UA@7iMHpye>*Q}Ff`X|KDQ4l% zLsL8SyDj&%sWAyQ=h>Lp4Pi3aIUK8HZ2IyM8Z&$7Y5%5z`sKYsj!#uv3xp?L?h?ES z;_)Ki(KZ)jliz;2$K+8O2DUpHp8e8+;@@jL?GH%IHA)G4P4HLOwBARXPGJ7{ z1gUC_8GW)oQ?t&R%L8j>7S zWg$d&9j!%-hbzBqB(putP+H=d73CpF1(77g#KbNh>gtEE{UR>>Vv>oB6`~%vxUx_Msvui zJTpZ_#jaf?-+=Fks0__CQ2=qls0Q*@!6j-x3^Sg?|VqGIvnU1p4}T^TY^E(2a&P@ zXJI;+GUOqR3gR@t1nhZ&krRwRS)BIF2vT4?fUQek$PYHHNAD`L>$lLWGs*{ zOdm*4D@S$lYhnJNIGE6?e&4u~u2M6}OkY(=%ga8sQsdqzZP!m^{)ZW{`#Dn&CN|8z zRwGY9aAtCK<+V1|~Oa_}9RLP)zUeJ2ZO3xxlelYP1uPMt@=0*Nn*8>LNFN4c(*(rn#PXNF7d4 zE3RTcF50SvBcVl)he&0c8aj_@fyGVI<@IJ6>VLo?+Vp|UzfmYzY@>Dqeo_*z{#qZF zQSGa!WA7_-iE->DBMZ%athP1wRWgIFZW`Se(b?v$mt0LGp%;-XiIdiFu<=k;#J!zl z<9O;-+9C(B^rUyoq{e0&(%>6By8}u+6$egy7%KV^aVhK47 zQ?DwPy_Na14o*BBHhG4}1O_=S+;`Ha5A&W&+DWWQ(ZTx6(bnOOU&*h((CaRe8oMUs z2BMu!@yX{4^gQF!Hw_&HH+FjvO!W$uO=kTb>*thnh7p(ObfBHj{A<>_9a<0f9p@an zA>e|_4bp?C10ObVJqQ#z&bI9LMTp=L-F3;YGN?oHR@Qs|mpVLzhr*Rr7p0N@5$Ha{ z5>_lsf`B`M*W6g^_>j#VMpz26;isBPCYV}7PMg*!aJkv*e7*jEhS zyA|I2BI?Z!c2Wk|RBv}X>t2*IEvg?LnpJSek;}vTFrUfWLsUYXV+-t)FCyQsTh*R~ zAI|{0{`t0}MuIeb&Eg(9#(Z|5|3Q8dt{i#$>e~7qwf_-;hc2b6m#-_hBOTa#TzUGZ zO_hK60z%##a@Lgn;Clhjc+Wg{-2I8}F$dz2GM!b8g>XeD&#@PThh>!AIGNkwQI_ml z`zA=^2)fHpe7C0?jW+}de`nDYKtW2QM5CuIv|MA!{Fg1FY$YX7T<7^Wa+cFbLZC&W zW|EH;d^+rC5kFSB$HqOnq)0xVtX)mX*;ON4qa*a=+sFKV?TQtY8Zf`%9n9r7kw?z%Bzb@2i%PDb+^7K>Z@n{6YR!w>Q zpPC7-9|M1v_7v!*5lOi!F&rQaQ3>3r?}#+zUqXIN^ioMP0s^NPHa6r$V$4!NZtJx% zxI5L9jxYi6L%;YFMUk^pXLEBPo<3f@b9NY8jJBjhQ;Bb8y@+>}+m#6hum9nrR@;be zUrzYG@@_CeV`98N8BQO9OqCjK1&~l06u*3Fb&P`|QwoRB=b4&GcsvyAS>d|i@ex>w zz^jQ!xV{D5mT8Y}?YZl{mYo;F!F61}N-1NIeU&s<-_|DAklv!vd6|9zovQu@pcQt< z{aIV7+z;g1UIY`q(%rVgaamG2ACJ3m$#5+MElT(BK6t~+b?GRv6ZR#A%#Y@YrqUn; z=JkrqkC)nK9^o4qi4Rru2+$gxAmEPuva%I$MVO%%*g$l3{dgoN=&@cZnGJ?69{JOY zt@qRLejYWqO#GpIXl34akiWf^jI4bh+yP&(04EkQEvIsPstXvdup%!Em2#wq3q1i# zeWpr=djwMPL8Y{k{RE;Rz>IixGN(53ASd8{`qN?Dk>f2RD3Wz{IPO41Q7=F^Ry*Cs z#^$*=fVCF@CDa4u4$Q?^9b!keW4XQGg9fCgy6t_gB9ecwjQjIdFh@D)@`ANg;F_NS+TjrNv0^;5q-XqQN(S_2&Ony}w6@Iy^KXx$=ot8cP^Q7@+( zKmU9FplG31V}$W&9X9--l0|zT$aM5Gq&QKRtmC}xTW|LD93r=K{#@;@?iP$N^?XAr zMX?`;bO*Ba|NO~>#5i|zgn!(mInha`2qpLFgPIlm)Hm#XH;Rp=<m$ern?i!`7}=k+38~Q9K-KO%T0&Xj+8J@Bq6|B2T{ zIVidO#AUf|QF!Pp|Ji0H(J=xo6|0HdUsWXB49YO*OW9Tw8}~+XOLYJipqaHhu*w}) z#0*gRcGMKxm2i}qd1GBZ%|MO)jLzChp>#thkk`k*e*HQ)6AsN%4~>{@-a!B_?zR`? zuitD*TOz>!j=0W9zARFBJ!OhLpD$Msjm&-RC&9enRq#QjI1ML6>tC-L$nCp5XCWwV z13CR#(Vm?C#G&={agFKT*9%c2V1c7II{2P(Lm!SJmfxXKk4ODHzk#8nk?kqRabeAn z0DD~cgot$uDBqD3Ev1bEAC%QZ1Myk_s@?`E2;WK zjT&_D;WiY$zGkJ9Al3?r=g!5G9ZpF_)qatR_p1_0*r&Stp94{n(c@}j3*B$P$gfwC zxlc!34sV7u@s3(R9{}0upuX<=IN0i$?%QvO*HxuF@!jcos#w`OZ9y6Xfjcd!L)~slErp zAR`-gJ_{bk!T<~4)G;i*l*=6^8vF63rOHn4A*-!4x1*(cy(*OW8%2-(F~#KynHq&eJfA zCYxWvi9ZnkQhk%JYtR3%kqO91=^BR&q^E;ck#tpqNh`f`;lT|J@A0_Tzij_)+4$sQ zCvOA17~7fl8L;NlwF-?p31HvKiTYIniu|j;|Bam0_WKdW*M&oXYFv$cl%V3U*D~8d zngEajdv#A0@<}LVzLNd;w`o8sBO@cr{|KIS)Vh7OK_0@~^An)Lyefr12V9VT|+cn{*gsjcY%@=xe+nY_D+)E_fL?k!p`qZ5U z@0Moc`x*ujf4fe=>MZW>wLamGcTOEiIRq=-W@hWO7)Ll9zM5{7`bh({ZZ?mIU+Djm zI$ujtLuYe-*@$Re@YJ7-2^rI_?y*`{AY2PlE?0_ctgd_0eV@{L16_%qWqCpx@!qF* zF+(9w^yLf1)YcZF_Vm?hmv3YbP7%C=c(b}NEDrwjE8XYLom0az26vRbQ1rpNq-NhP zpH`gxBMRL6Orf9MvHYPN%s9P?so=Umv>gj6ait1`@I22e39yL31OzzAF^p_ zD^l+?b^q)dy3&06l5g#@7>mk*NMKLegjnc35Hh;bflBt|_A?>I^Ct-gn%-Rc4-epX z23Ins3L-$OY{*4TwW>LSicgj$XM%4(s8eXqpM-Zc{pN0;DbHuxG_aYLAg$Y^fE>Im z9pyHvyI2NbN7(HLbm1&oZhdR$?c2AH;GqxyB!!QMH-Aj8K;C`;YPut(Kew9a<^if& zpC$k`&E7*_s*{`|`J_ZEG^l=d%v>mie3woF1fzg`*H7ZsSot;r}*|l(> z2TEKdeU6J(%I`~6qVHB0R8LC{`pRiJ-`}k@C-s_$Xp9(E5T+ebeRKESmQQx7{;29S2FB zd8O|j#dsIG>7_IU!-+?Zwuy>Pe=4_5-|zkP>lX#xl;J}qK)^nFu4BKG-=an2jivz; zy-O%DlH=QdhZ~j1V+Aj*G>REQ-n@RWqHmxlpjJ#?=H76hu!Wq3 z#V_P>NQ`~!@mh7(!nMHa6F}E}CK~}vUzP^v50PeCuq5wHYvz zjuL35*NOj=@BOzIbWn$fuLOiG5`j=geijc^p^-F7vrBEVAtau~R~zG>%b^Nl7U%|b84QFEgv_zqk)camrj>^ zF%`Bw1wZwQ-BH5dM2wXjSG!%B-1I*AN(J&vSgqB)8|686~!iC=yjDvZTE3 z4!fJreh?V8LMJ=v?fM%BKlk@cI&p^T) zu<%=$<>+6kZLLwB76%*k9}^qXxv9K^EHY5oJ_vNLhc^ixU90i&@Wfx*JR;%XGDjK8cGs>!pnvL=@S2j@G! zO0}sq0hP~5fB!%^R1bna!*Gi*sdN#!f2H~01w=)7E@m^L^?(vL`t9tKxDw%fwxS$G!~IqQYjTJqt|TEV&{_RfQ9Z&Smu`N zB<26j00ANJ?Z?qjBJ<;EV}b$ile+!^GskUqLvzr#`qvC~AQ!V%WjdC=yp-bCb1Bia z$3|?ie<0ls@>p}O!ZVZ1@DJ+R(8VJAXI<7<=O~(VB75aYS7_HS0y>8CDp$nqZc`vV zj@AAdb%VB-e_v<*)FiDj{jWma0*vr5h`lG_Y2d+EFBzkv?-RmA`vc=HlZpbwb<`lo za%Y~%f7B@`NvF*n$)e`Ye?e=JvwCV@fOXjm4-~|J8qetyH-5EHJ*Q1cPf4o^T$B%F z;lvYm#2b$Lb$GxV(sQu*h+QB z@P$e`&084hV(%2E_IacBP9zUZ2H9$V>{WVkhC}Gm*48+it-XBP8m0_zwkcTUX7W;7 z-r$DQgUp}9%1=Z8_y#Ly%bgXBa57i;8xW&c2+am4N^LP3^t zqODDI`Y2=qL4}pPS4?SaL#>En5+SzYbPXZ5r%~Bm*Wjd?BR6C8Y)@yO6tN`1cZK<&e44@@)Vw?V1n7%o#mV z$b5zVB2DfA{6`t{49UG6%)TQ1Z{bcGib_VwW{$gmpr9nQ=iF;f=7@$C7dr?D{xG%o zW6OI#5vMaoE>p%D)XJxQ?p9!FrKWnD+1;gJv|EPACEv=H;4nHmhPzLp4|RJzQF7_=F;!JiqH3d)!2Ux^^ekASKvRzboSLyB^AR52}3z$IX} zm@kKOO4wKNh8h}8&H=!cRelCKNvh9dL7%9#Fh z%t%_UI!|)_6hTWxA;U^JFdktIUJPRjK-Io)b@Q{ymGsYlRp~Pq1mE@pshp0ydKw zTdy=fU&qSIIxY!Bj@9o3o-jpmA7ki5juNr{LVKCv(btO&#l>eqUFp}1hzh!z_Mz7b zU`1ui?ekvp;|r1V@^J-21ROKnX?~D{+k8C@z7RFoTBVvP^%Vyv>3?gULnlfU$eQS8t%jtnv5)#k zDBr&)LwwoS1$e{hs`#Rll}TOTn4>ZT!)ke_r)ESZQErF^$@c&huJRp1IP!R$?dpp4 zxYH@LJAcW!PM56y$JiB%x^~UU-SncADx!ZKK88ATmSvT0UoVi0a=njzsK0+d!RzsR^$DxBWR@EFazUZk3#LvLr4?3hw`J~!XSBN9Hx9`f8pk}vg-IGB5 z47{oN^e-jWQ&emkLI5A$&su9&?hin_W%BL~RxNkn|7;o$`S&6Yb$L2X+1->s2?9ol z-&y|gyGUkRG&0|{Am&y$E|fm0gaS5shHbQbve^;@x^Qqmv^q9@(=o}<^?HGquaL`b zu^;}E^L2m4n<)(w2zqn0!B8JX*R7tynme6+h%oaPdu-O2hdfl!l^N2BXZ19=^dkv@ z2h$~Ph?u6hg6yvIMo_%=(b>8#I1@GIB!dL#M1APihl(DyRuscr0WBGrV-9ujDe12^l83 zIm~!WHyeomMI=WD8+(Xbg63E#cGve`sd2OCCD$QKdd2JS`?uOqpJ^|Gf(Q0)p|+b1 z-y8k8FdC(NDT2IsHMdh-984+xVH2Gn*R}^M>tT~Kk6IoYsq5~$xLHCX`-j!p-A@ri zp{f%nLahJ}LCb|EWM6{Q5u?k1Uq^CD>%hFg+0Da@IJa6bw%S$#PuK|~39cB4jnw3q z#3{yo4DND?vGqXoy?86bV|bt{v)mdozQ{SG@D+RYkco<5iftDBEV=KrCN5N*Pnyb& z&%T)`k=RZ(yqjx3#&DwZ(DtuU;thrJVkordo9~%fQlp?IQGN&}e!s<>=yx-G8Np1= z-V&M*lew9d&Q!c#5Vu({IRC;VJ~aJY+>IL96n1nf%V~z({%ha>Dir4N;A=$%v6w-M z@q|l-LsH_~S*W$8S)Tn@#1$-K2?5paKCxj0bj{tL^(Ts>vLRg>mUZ3)dx6%bejnYc z{Sa@?ElRP5BQN8j{%{-%+oK8|z#80k2m43%OElZ0nz0SblIKs3J3_91t{qA*LJ`Nl zM$+0ef)~lS-^q?u0|oXrduxqW!TQL}7vgBE%6|R_-~a%hi`szdKII|X7Gv>zJT5=f z^5KY^70VBOS=k%6qk97wTR|qm18f%;G z`JhMu;*6llzs(SJL~JrRK4D4>m4us};X8TqBszKNBqk}sFn`5=C6GQu!C|(?r^uVX z(fGM+Nv{p1o86G~@9LV+S=l6`nt-FP+XEGP+yfwd04bH8cHR74u*nzgnNl6X-`cdc zN}NeS<<}`&E&n*w?2a8N>)NRes|+Py5+!?LI%DD1bg<<36*IoVnRTsYc-S>RvR1YT zs5jZly4iZY&Gkmts@n8=M$(IX<6>7s{Prbbp9axxhvfpD`8GI9rR|=}u!?_hNH_s{ zFe~dfpq#6coyy|!ZqJD;0PeMzQDiJCIHyYQt)0M2+ORJA{HLL_2c#fM;T1EgbS+XJ{CR5Zdo* zo)}e~mFuID4G-`_gAdB?)|6&X1J>@OpEbc{xR=fCT~szU-?;)^+siwM+nCnx(5jAn zNF zo|8rw{bhA@|NV3bSA@8PI{Lb)moE_N)b?)pHV!jPWW@$lYR?N~ts5!v1X^V`SRqXA zMQs-~+7<_t#*SvU>_qDKdd59MkBS0WwI0mv=Iwmh0}rThsI*WbE=@|!c-pmkhbCFa zhwF7#eXQn<9wXl^eqx#Dz$D;lAK17tJ%oGXnF3S_H2YryPxnnU#dMD5j35Iiu0oqE^`sm*{;6FparJ-#+Bh-bIfXH5`egH^V6Mk}Kmd1|7qhB@lC zXOhYT)#fE1Lm>qDx>eCw>`pv3J)tVTLi=~=pu{bLo_mZIWu-f%v|6dk=h3cdp>-7r zm7nI??5n`FmRDXb@V>I+dQgU8OqbjXY0O@iG2!P=VW<7?Ax_-BIEz*-Up_70B=04U ziUw|PA<RK+!CBJ6}duTrtUqhCto%_3zDM7pvQES*_@I z^C!vk?Gmh+J}Wus{dU->f+MhBat3CdZN1#eHSr=f easS-}7AEhr(A!t68x^Y`+Wb^W9a^Gb{`!B}Gp(@z literal 0 HcmV?d00001 diff --git a/python_modules/dagster/README.rst b/python_modules/dagster/README.rst index 74ae42b59594c..e69de29bb2d1d 100644 --- a/python_modules/dagster/README.rst +++ b/python_modules/dagster/README.rst @@ -1,532 +0,0 @@ -Dagster -======= - -.. docs-include - -Dagster is a library that helps you organize, run and test your data processing pipelines. Instead of writing ad-hoc data processing script, Dagster encourages separating data input, data processing and data output, as well as separating your processing in several steps. - - -The core abstraction in Dagster is a **Solid**, a logical unit of computation in a data pipeline. At its core a Solid is a coarse-grained pure function. It takes multiple inputs, performs a computation, produces an output. Inputs can either be received from outputs from upstream solids in the pipeline or external sources (e.g. files). Likewise, outputs from the transform can either be flowed to downstream solids or materialized (e.g. to a file in an object store) so that it can be inspected or accessed by an external system. - -Solid formally separates the notion of inputs, outputs, the core transform. The core goal of this is cleanly and clearly separate the domain logic of each transform and its operational environments. This allows solids or pipelines of solids to easily be executable in a number of environments: unit-testing, local development, integration tests, CI/CD, staging environments, production environments, and so on and so forth. - -Alongside with the core abstraction, Dagster provides helpers to create Solids that operate on Pandas dataframes and SQL databases. - -Example -------- - - - -.. code-block:: python - - - import pandas as pd - import dagster.core - from dagster.core.definitions import ( - InputDefinition, - OutputDefinition - ) - from dagster.core.decorators import ( - solid, - source, - materialization, - ) - - # Sources define *how* to get data. Let's define a source that - # reads a CSV file and returns a pandas dataframe - @source( - # Name of the source - name="CSV", - # What arguments it should get from environment - argument_def_dict={'path': ArgumentDefinition(dagster.core.types.Path) }) - ) - def csv_to_dataframe_source(path): - return pd.read_csv(path) - - # Materializations define how to convert a transform result - # into an artifact, for example a file. Let's define one that - # outputs a CSV file. Note that materializations that are - # defined for solids are optional, and are only triggered if - # environment says so - @materialization( - # Name of the materialization - name="CSV", - # What arguments it should get from environment - argument_def_dict={'path': ArgumentDefinition(dagster.core.types.Path) }) - ) - def dataframe_to_csv_materialization(data, path): - data.to_csv(path) - - # Solids can be created by annotating transform function with - # a decorator - @solid( - # Solid inputs define arguments passed to transform function - inputs=[ - InputDefinition( - name='num', - # Inputs can have sources. While inputs define - # *what* transform should receive, sources define - # *how* this data should be retrieved - sources=[csv_to_dataframe_source] - ) - ], - # Solid output determines what solid should return. - output=OutputDefinition(materializations=[ - dataframe_to_csv_materialization, - ]) - ) - def sum_solid(num): - sum_df = num.copy() - # Here we add a new column to dataframe to sum up num1 and - # num2 columns - sum_df['sum'] = sum_df['num1'] + sum_df['num2'] - return sum_df - - - @solid( - inputs=[ - # This solid is a *dependency*. It depends on a result - # of previous solid as one of the inputs - InputDefinition(name="sum", depends_on=sum_solid) - ], - output=OutputDefinition(materializations=[ - dataframe_to_csv_materialization, - ]) - ) - def sum_sq_solid(sum): - sum_sq = sum.copy() - sum_sq['sum_sq'] = sum['sum']**2 - return sum_sq - - - # After definining a solid, we are grouping them into a pipeline - pipeline = dagster.core.pipeline( - name='pandas_hello_world', - solids=[ - sum_solid, - sum_sq_solid, - ], - ) - -You might notice that there is no actual CSV file specified as inputs. This is because such parameters are passed in environment. This allows you to customize it in runtime. To run your solid, we'll pass that environment to the execution function. - -.. code-block:: python - - environment = config.Environment( - sources={ - 'sum_solid' : { - 'num' : config.Source(name='CSV', args={'path': 'path/to/num.csv'}) - } - } - ) - - pipeline_result = dagster.execute_pipeline( - dagster.ExecutionContext(), - pipeline, - environment - ) - - -We can simplify the above example by using built-in dagster pandas inputs and outputs. - -.. code-block:: python - - import dagster.core - from dagster import config - from dagster.core.decorators import solid, with_context - import dagster.pandas as dagster_pd - - @solid( - inputs=[ - # We are using a pre-made input that should be a dataframe - dagster_pd.dataframe_input( - 'num', - sources=[ - # A built-in pandas csv-dataframe source reads - # a CSV file andproduces a pandas dataframe - dagster_pd.csv_dataframe_source() - ] - ) - ], - # This built-in dataframe knows how to materialize dataframes - # out of the box - output=dagster_pd.dataframe_output() - ) - def sum_solid(num): - sum_df = num.copy() - # Here we add a new column to dataframe to sum up num1 and num2 columns - sum_df['sum'] = sum_df['num1'] + sum_df['num2'] - return sum_df - - - @solid( - inputs=[ - # This input will check that the source solid outputs a - # dataframe - dagster_pd.dataframe_dependency(name="sum", solid=sum_solid) - ], - output=dagster_pd.dataframe_output() - ) - def sum_sq_solid(sum): - sum_sq = sum.copy() - sum_sq['sum_sq'] = sum['sum']**2 - return sum_sq - - # After definining a solid, we are grouping them into a pipeline - pipeline = dagster.core.pipeline( - name='pandas_hello_world', - solids=[ - sum_solid, - sum_sq_solid, - ], - ) - -We can specify in order to get artifacts for the results. We can materialize output from any solid, this can be useful to see if intermediate results make sense. - -.. code-block:: python - - environment = config.Environment( - sources={ - 'sum' : { - 'num' : config.Source(name='CSV', args={'path': 'path/to/num.csv'}) - } - } - ) - - materializations = [ - config.Materialization( - solid='sum', - name='CSV', - args={'path': 'path/to/output.csv'}, - ), - config.Materialization( - solid='sum_sq', - name='CSV', - args={'path': 'path/to/output.csv'}, - ) - ] - - pipeline_result = dagster.execute_pipeline( - dagster.ExecutionContext(), - pipeline, - environment, - materializations, - ) - - -Dagster CLI -=========== - -In addition to programmatic API, you can also use dagster CLI to run the pipelines. In that case the environment is specified through yaml configuration files. - -The folder structure should be as follows. - -.. code-block - - pipeline_project_name/ - pipelines.yml - pipeline_module_1/ - env.yml - pipeline_module_2/ - env.yml - -Pipelines yml specify the pipelines that are present in current project. Env specifies environment for each particular pipeline. - -.. code-block:: yaml - - pipelines: - - module: pipeline_project_name.pipeline_module_1.pipeline - fn: define_pipeline - - module: pipeline_project_name.pipeline_module_2.pipeline - fn: define_pipeline - - -.. code-block:: yaml - - environment: - inputs: - - input_name: num - source: CSV - args: - path: "input/num.csv" - - materializations: - - solid: sum - type: CSV - args: - path: 'sum.csv' - - solid: sum_sq - type: CSV - args: - path: 'sum_sq.csv' - - -.. code-block:: sh - - pip install dagster - # List pipelines - dagster pipeline list - # Print info about pipeline solids - dagster pipeline print pipeline1 - # Execute pipeline - dagster pipeline execute pipeline1 - # Execute pipeline from intermediate step - dagster pipeline execute pipeline1 --from-solid SOLID_NAME - - -Concepts -======== - -Transform ---------- - -This is core, user-defined transform that performs the logical data -computation. In this case the transform is ``hello_world_transform_fn`` -and it is passed as parameter into the solid. It takes one or more -inputs and produces an output. All other concepts in a solid are the -metadata and structure that surround this core computation - -Inputs ---------- - -For each argument to the transform function, there is one -``InputDefinition`` object. It has a name, which must match the -parameters to the transform function. The input definitions define a -name, a dependency for the input (what upstream solid produces its -value, see below) and a number of sources. An input definition must -specify at least a dependency or a source. The input can have any number -of sources. - -Sources -^^^^^^^ - -Sources are the the way that one can create an input to a transform from -external resources. A source is a function that takes a set of arguments -and produces a value that is passed to the transform for processing. In -this case, a CSV file is the only source type. One can imagine adding -other source types to create pandas dataframes for Json, Parquet, and so -forth. End users will typically rely on pre-existing libraries to -specify sources. - -Sources also declare what arguments they expect. These are inspectable -and could be used to render information for web or command line tools, -to verify the validity of confie files, and other tooling contexts. The -framework verifies when solids or pipelines of solids are executed, that -the arguments abide by the declaration. These arguments are then sent to -the source function in the ``arg_dict`` parameter. - -Output ---------- - -The ``OutputDefinition`` represents the output of the transform -function. - -Materializations -^^^^^^^^^^^^^^^^ - -Materializations are the other end of source. This specifies the way the -output of a transform can be materialized. In this example which uses -pandas dataframes, the sources and materializations will be symmetrical. -In the above example we specified a single materialization, a CSV. One -could expand this to include JSON, Parquet, or other materialiations as -appropriate. - -However in other contexts that might be true. Take solids that operate -on SQL-based data warehouses or similar. The core transform would be the -SQL command, but the materialization would specify a -``CREATE TABLE AS``, a table append, or a partition creation, as -examples. - -Higher-level APIs ------------------- - -These definitions will typically be composed with higher level APIs. For -example, the above solid could be expressed using APIs provided by the -pandas kernel. (Note: the "kernel" terminology is not settled) - -.. code-block:: python - - import dagster - import dagster.pandas as dagster_pd - - def sum_transform(num_df): - num_df['sum'] = num_df['num1'] + num_df['num2'] - return num_df - - sum_solid = SolidDefinition.single_output_transform( - name='sum', - description='This computes the sum of two numbers.' - inputs=[dagster_pd.dataframe_csv_input(name='num_df')], - transform_fn=sum_transform, - output=dagster_pd.dataframe_output(), - ) - -Execution ---------- - -These are useless without being able to execute them. In order to -execute a solid, you need to package it up into a pipeline. - -.. code-block:: python - - pipeline = dagster.PipelineDefinition(name='hello_world', solids=[sum_solid]) - -Then you an execute it by providing an environment. You must provide -enough source data to create all the inputs necessary for the pipeline. - -.. code-block:: python - - environment = config.Environment( - sources={ - 'sum' : { - 'num_df' : config.Source(name='CSV', args={'path': 'path/to/input.csv'}) - } - } - ) - - pipeline_result = dagster.execute_pipeline( - dagster.ExecutionContext(), - pipeline, - environment - ) - - print(pipeline_result.result_for_solid('sum').transformed_value) - -Execute pipeline does a purely in-memory transform, materializing -nothing. This is useful in testing and CI/CD contexts. - -Materialization ----------------- - -In order to produce outputs that are available to external systems, you -must materialize them. In this case, that means producing files. In -addition to your environment, you must specify your materializations. - -.. code-block:: python - - materializations = [ - config.Materialization( - solid='sum', - name='CSV', - args={'path': 'path/to/output.csv'}, - ) - ] - - dagster.execute_pipeline( - dagster.ExecutionContext(), - pipeline, - environment, - materializations, - ) - -Dependencies ------------- - -So far we have demonstrated a single stage pipeline, which is obviously -of limited value. - -Imagine we wanted to add another stage which took the sum we produced -and squared that value. (Fancy!) - -.. code-block:: python - - def sum_sq_transform(sum_df): - sum_df['sum_sq'] = sum_df['sum'] * sum_df['sum'] - return sum_df - - # Fully expanded InputDefintion. Should be wrapped in higher-level - # but this is here for explanatory code. - sum_sq_input = InputDefinition( - name='sum_df', - sources=[ - SourceDefinition( - source_type='CSV', - argument_def_dict={'path': ArgumentDefinition(types.Path)}, - source_fn=lambda arg_dict: pd.read_csv(arg_dict['path']), - ), - ], - depends_on=sum_solid, - ) - - sum_sq_solid = SolidDefinition.single_output_transform( - name='sum_sq', - inputs=[sum_sq_input], - transform_fn=sum_sq_transform, - output=dagster_pd.dataframe_output(), - ) - -Note that input specifies that dependency. This means that the input -value passed to the transform can be generated by an upstream dependency -OR by an external source. This allows for the solid to be executable in -isolation or in the context of a pipeline. - -.. code-block:: python - - pipeline = dagster.PipelineDefinition(solids=[sum_solid, sum_sq_solid]) - - environment = config.Environment( - sources={ - 'sum' : { - 'num_df' : config.Source(name='CSV', args={'path': 'path/to/num.csv'}) - } - } - ) - - pipeline_result = dagster.execute_pipeline( - dagster.ExecutionContext(), - pipeline, - environment - ) - -The above executed both solids, even though one input was provided. The -input into sum\_sq\_solid was provided by the upstream result from the -output of sum\_solid. - -You can also execute subsets of the pipeline. Given the above pipeline, -you could specify that you only want to specify the first solid: - -.. code-block:: python - - environment = config.Environment( - sources={ - 'num_df' : config.Source(name='CSV', args={'path': 'path/to/num.csv'}) - } - ) - - pipeline_result = dagster.execute_pipeline( - dagster.ExecutionContext(), - pipeline, - environment, - through=['sum'], - ) - -Or you could specify just the second solid. In that case the environment -would have to be changed. - -.. code-block:: python - - environment = config.Environment( - sources={ - 'sum_sq' : { - 'sum_df' : config.Source(name='CSV', args={'path': 'path/to/sum.csv'}) - } - } - ) - - pipeline_result = dagster.execute_pipeline( - dagster.ExecutionContext(), - pipeline, - environment, - from=['sum_sq'], - through=['sum_sq'], - ) - -Expectations ------------- - -Expectations are another reason to introduce logical seams between data -computations. They are a way to perform data quality tests or -statistical process control on data pipelines. - -TODO: Complete this section when the APIs and functionality are more -fleshed out. diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index d4c7ef0f8e962..970655dc24150 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -70,34 +70,55 @@ class PipelineContextDefinition(object): (such as db connections) off of that context. Thus the pipeline author has complete control over how the author of each individual solid within the pipeline interacts with its operating environment. + + The PipelineContextDefinition is passed to the PipelineDefinition in + a dictionary key'ed by its name so the name is not present in this object. + + Attributes: + config_def: A ConfigurationDefinition. A context definition + defines the configuration necessary to instantiate a context. + This is where the that description goes + context_fn: A callable that either returns *or* yields an ExecutionContext. + (pipeline: PipelineDefinition, config: Any) : ExecutionContext + description: A description of what this context represents ''' @staticmethod def passthrough_context_definition(context): + '''Create a context definition from a pre-existing context. + This can be useful in testing contexts where you may want to + create a context manually and then pass it into a one-off + PipelineDefinition + + Args: + context: ExecutionContext + Returns: + PipelineContextDefinition + ''' + check.inst_param(context, 'context', ExecutionContext) context_definition = PipelineContextDefinition(context_fn=lambda _pipeline, _args: context) return {'default': context_definition} def __init__(self, context_fn, config_def=None, description=None): ''' - Parameters - ---------- - config_def: ConfigDefinition - Define the configuration for the context + Args: + config_def: ConfigDefinition + Define the configuration for the context - context_fn: callable (pipeline: PipelineDefinition, args: dict str => Any - Returns *or* yields an ExecutionContext. + context_fn: callable (pipeline: PipelineDefinition, args: dict str => Any + Returns *or* yields an ExecutionContext. - If it yields a context, the code after the yield executes after pipeline - completion, just like a python context manager. + If it yields a context, the code after the yield executes after pipeline + completion, just like a python context manager. - Environment-specific resources should be placed in the "resources" argument - to an execution context. This argument can be *anything* and it is made - avaiable to every solid in the pipeline. A typical pattern is to have this - resources object be a namedtuple, where each property is an object that - manages a particular resource, e.g. aws, a local filesystem manager, etc. + Environment-specific resources should be placed in the "resources" argument + to an execution context. This argument can be *anything* and it is made + avaiable to every solid in the pipeline. A typical pattern is to have this + resources object be a namedtuple, where each property is an object that + manages a particular resource, e.g. aws, a local filesystem manager, etc. - description: str (optional) + description: str (optional) ''' self.config_def = check.opt_inst_param( config_def, @@ -110,8 +131,8 @@ def __init__(self, context_fn, config_def=None, description=None): def _default_pipeline_context_definitions(): - def _default_context_fn(_pipeline, args): - log_level = level_from_string(args['log_level']) + def _default_context_fn(_pipeline, config_value): + log_level = level_from_string(config_value['log_level']) context = ExecutionContext( loggers=[define_colored_console_logger('dagster', level=log_level)] ) @@ -136,18 +157,23 @@ class DependencyDefinition(namedtuple('_DependencyDefinition', 'solid output des '''Dependency definitions represent an edge in the DAG of solids. This object is used with a dictionary structure (whose keys represent solid/input where the dependency comes from) so this object only contains the target dependency information. - ''' - def __new__(cls, solid, output=DEFAULT_OUTPUT, description=None): - ''' - Parameters: + Attributes: solid: str The name of the solid that is the target of the dependency. This is the solid where the value passed between the solids comes from. - output: str (optional) defaults to 'result' + output: str (optional) The name of the output that is the target of the dependency. description: str (optional) + ''' + + def __new__(cls, solid, output=DEFAULT_OUTPUT, description=None): + ''' + Args: + solid: str + output: str (optional) defaults to 'result' + description: str (optional) ''' return super(DependencyDefinition, cls).__new__( cls, @@ -262,6 +288,13 @@ class PipelineDefinition(object): (of type ExecutionContext) is created and what configuration is necessary to create it. - Dependencies. Solids within a pipeline are arranged as a DAG (directed, acyclic graph). Dependencies determine how the values produced by solids flow through the DAG. + + Attributes: + name: + description: + solids: + dependencies: + dependency_structure: ''' @staticmethod @@ -272,26 +305,23 @@ def create_single_solid_pipeline(pipeline, solid_name, injected_solids=None): Frequently (especially in test contexts) one wants to isolate a single solid for independent testing. - See Also - -------- - PipelineDefinition.create_sub_pipeline + See PipelineDefinition.create_sub_pipeline. - Parameters - ---------- - pipeline: PipelineDefinition - solid_name: str - injected_solids: + Args: + pipeline: PipelineDefinition + solid_name: str + injected_solids: Two dimensional dictionary. (optional) solid_name (str) => index_name (str) => SolidDefinition - When you create a subpipeline, you possible left with solids within that pipeline - who have unmet dependencies. To fulfill these dependencies new solids must be - provided. + When you create a subpipeline, you possible left with solids within that pipeline + who have unmet dependencies. To fulfill these dependencies new solids must be + provided. - returns a PipelineDefinition + Returns: + PipelineDefinition - Examples - -------- + Example: new_pipeline = PipelineDefinition.create_single_solid_pipeline( existing_pipeline, 'A', # existing_pipeline @@ -319,20 +349,18 @@ def create_sub_pipeline(pipeline, from_solids, through_solids, injected_solids=N of solids and then proceed forward through the dependency graph until all the "through" solids are reached. - See Also - -------- - PipelineDefinition.create_single_solid_pipeline + See PipelineDefinition.create_single_solid_pipeline - Parameters - ---------- - pipeline: PipelineDefinintion - from_solids: list of strs - through_solids: list of strs - injected_solids: - Two dimensional dictionary. (optional) - solid_name (str) => index_name (str) => SolidDefinition + Args: + pipeline: PipelineDefinintion + from_solids: list of strs + through_solids: list of strs + injected_solids: + Two dimensional dictionary. (optional) + solid_name (str) => index_name (str) => SolidDefinition - returns a PipelineDefinition + Returns: + PipelineDefinition ''' # FIXME: fix circular reference check.inst_param(pipeline, 'pipeline', PipelineDefinition) @@ -416,13 +444,12 @@ def __init__( self, solids, name=None, description=None, context_definitions=None, dependencies=None ): ''' - Parameters - ---------- - solids: list of SolidDefinition - name: string (optional) - description: string (optional) - context_definitions: dictionary str => PipelineContextDefinition (optional) - dependencies: 2D dictionary str => str => DependencyDefinition (optional) + Args: + solids: list of SolidDefinition + name: string (optional) + description: string (optional) + context_definitions: dictionary str => PipelineContextDefinition (optional) + dependencies: 2D dictionary str => str => DependencyDefinition (optional) ''' self.description = check.opt_str_param(description, 'description') self.name = check.opt_str_param(name, 'name') @@ -496,12 +523,11 @@ class InputDefinition(object): '''An InputDefinition instance represents an argument to a transform defined within a solid. Inputs are values within the dagster type system that are created from previous solids. - Parameters - ---------- - name: str - dagster_type: DagsterType (optional) defaults to types.Any - expectations: list of ExpectationDefinition (optional) - description: str (optional) + Attributes: + name: str + dagster_type: DagsterType (optional) defaults to types.Any + expectations: list of ExpectationDefinition (optional) + description: str (optional) ''' def __init__(self, name, dagster_type=None, expectations=None, description=None): @@ -523,12 +549,11 @@ class OutputDefinition(object): output, and so the user can construct a single OutputDefinition that will have the default name of "result". - Parameters - ---------- - name: str (optional) defaults to "result" - dagster_type: DagsterType (optional) defaults to types.Any - expectations: list of ExpectationDefinition (optional) - description: str (optional) + Attributes: + name: str (optional) defaults to "result" + dagster_type: DagsterType (optional) defaults to types.Any + expectations: list of ExpectationDefinition (optional) + description: str (optional) ''' def __init__(self, name=None, dagster_type=None, expectations=None, description=None): @@ -603,10 +628,9 @@ class Result(namedtuple('_Result', 'value output_name')): An implementator of a SolidDefinition must provide a transform that yields objects of this type. - Parameters - ---------- - value: Any - output_name: str (optional) defaults to "result"''' + Attributes: + value: Any + output_name: str (optional) defaults to "result"''' def __new__(cls, value, output_name=DEFAULT_OUTPUT): return super(Result, cls).__new__( @@ -625,26 +649,24 @@ def __init__(self, config_type=types.Any): '''Solids have config, which determine how they interact with the external world. Example configs would be file paths, database table names, and so forth. - Parameters: - ---------- - config_type: DagsterType (optional defaults to types.Any)''' + Args: + config_type: DagsterType (optional defaults to types.Any)''' self.config_type = check.inst_param(config_type, 'config_type', DagsterType) class SolidDefinition(object): '''A solid is a node of computation within a pipeline. - Parameters: - ---------- - name: str - inputs: list of InputDefinitions - transform_fn: callable with sig ( - context: ExecutionContext, - inputs: str => Any, - config_dict: str => Any) : Iterable - outputs: list of OutputDefinitions - config_def: ConfigDefinition (optional) - description: str (optional) + Attributes: + name: str + inputs: list of InputDefinitions + transform_fn: callable with sig ( + context: ExecutionContext, + inputs: str => Any, + config_value: Any) : Iterable + outputs: list of OutputDefinitions + config_def: ConfigDefinition + description: str ''' def __init__(self, name, inputs, transform_fn, outputs, config_def=None, description=None): diff --git a/python_modules/dagster/docs/index.rst b/python_modules/dagster/docs/index.rst index e37a359468b77..da3d39405bcb2 100644 --- a/python_modules/dagster/docs/index.rst +++ b/python_modules/dagster/docs/index.rst @@ -6,7 +6,7 @@ Dagster's documentation! =================================== -.. include:: ../README.rst +.. include:: ../../../README.rst :start-after: docs-include From 3943e1f88a2096e65dda1098db8ab2074dc65e8e Mon Sep 17 00:00:00 2001 From: Mikhail Novikov Date: Tue, 4 Sep 2018 14:58:00 +0300 Subject: [PATCH 048/103] Make dagit work with new config --- python_modules/dagit/dagit/schema.py | 112 ++++++++++++------ .../dagit/dagit/webapp/package.json | 6 +- python_modules/dagit/dagit/webapp/schema.json | 1 - .../dagit/dagit/webapp/src/Config.tsx | 82 +++++++++++++ .../dagit/dagit/webapp/src/Pipeline.tsx | 17 ++- .../dagit/dagit/webapp/src/Solid.tsx | 40 +++---- .../dagit/webapp/src/SolidTypeSignature.tsx | 6 +- .../dagit/dagit/webapp/src/SpacedCard.tsx | 27 +++-- .../dagit/webapp/src/TypeWithTooltip.tsx | 4 +- .../webapp/src/graph/getFullSolidLayout.ts | 4 +- .../dagit/dagit/webapp/src/index.tsx | 14 +++ .../dagit/dagit/webapp/src/schema.json | 1 + .../dagit/dagit/webapp/src/types/AppQuery.ts | 60 ++++++++-- ...alizationFragment.ts => ConfigFragment.ts} | 26 +++- .../webapp/src/types/PipelineFragment.ts | 60 ++++++++-- .../webapp/src/types/PipelinesFragment.ts | 60 ++++++++-- .../dagit/webapp/src/types/SolidFragment.ts | 28 ++++- .../dagit/webapp/src/types/SourceFragment.ts | 37 ------ ...Fragment.ts => TypeWithTooltipFragment.ts} | 4 +- .../dagit/dagit/webapp/typings.d.ts | 5 + python_modules/dagit/webapp/src/Arguments.tsx | 49 -------- .../webapp/src/types/ArgumentsFragment.ts | 31 ----- 22 files changed, 425 insertions(+), 249 deletions(-) delete mode 100644 python_modules/dagit/dagit/webapp/schema.json create mode 100644 python_modules/dagit/dagit/webapp/src/Config.tsx create mode 100644 python_modules/dagit/dagit/webapp/src/schema.json rename python_modules/dagit/dagit/webapp/src/types/{MaterializationFragment.ts => ConfigFragment.ts} (53%) delete mode 100644 python_modules/dagit/dagit/webapp/src/types/SourceFragment.ts rename python_modules/dagit/dagit/webapp/src/types/{TypeFragment.ts => TypeWithTooltipFragment.ts} (88%) delete mode 100644 python_modules/dagit/webapp/src/Arguments.tsx delete mode 100644 python_modules/dagit/webapp/src/types/ArgumentsFragment.ts diff --git a/python_modules/dagit/dagit/schema.py b/python_modules/dagit/dagit/schema.py index 4c34fee80b6fd..1ab61cde5f63d 100644 --- a/python_modules/dagit/dagit/schema.py +++ b/python_modules/dagit/dagit/schema.py @@ -1,4 +1,5 @@ import graphene +from dagster.core.types import (DagsterScalarType, DagsterCompositeType) class Query(graphene.ObjectType): @@ -17,16 +18,11 @@ def resolve_pipelines(self, info): return pipelines -# (XXX) Some stuff is named, other stuffed is keyed in dict. -# Either everything should be named or everything should be keyed - - class Pipeline(graphene.ObjectType): - # XXX(freiksenet): optional, but probably shouldn't be name = graphene.NonNull(graphene.String) description = graphene.String() solids = graphene.NonNull(graphene.List(lambda: graphene.NonNull(Solid))) - context = graphene.NonNull(graphene.List(lambda: graphene.NonNull(PipelineContext))) + contexts = graphene.NonNull(graphene.List(lambda: graphene.NonNull(PipelineContext))) def __init__(self, pipeline): super(Pipeline, self).__init__(name=pipeline.name, description=pipeline.description) @@ -40,7 +36,7 @@ def resolve_solids(self, info): ) for solid in self._pipeline.solids ] - def resolve_context(self, info): + def resolve_contexts(self, info): return [ PipelineContext(name=name, context=context) for name, context in self._pipeline.context_definitions.items() @@ -50,17 +46,14 @@ def resolve_context(self, info): class PipelineContext(graphene.ObjectType): name = graphene.NonNull(graphene.String) description = graphene.String() - arguments = graphene.NonNull(graphene.List(lambda: graphene.NonNull(Argument))) + config = graphene.NonNull(lambda: Config) def __init__(self, name, context): super(PipelineContext, self).__init__(name=name, description=context.description) self._context = context - def resolve_arguments(self, info): - return [ - Argument(name=name, argument=argument) - for name, argument in self._context.argument_def_dict.items() - ] + def resolve_config(self, info): + return Config(self._context.config_def) class Solid(graphene.ObjectType): @@ -68,7 +61,7 @@ class Solid(graphene.ObjectType): description = graphene.String() inputs = graphene.NonNull(graphene.List(lambda: graphene.NonNull(Input))) outputs = graphene.NonNull(graphene.List(lambda: graphene.NonNull(Output))) - config = graphene.NonNull(graphene.List(lambda: graphene.NonNull(Argument))) + config = graphene.NonNull(lambda: Config) def __init__(self, solid, depends_on=None, depended_by=None): super(Solid, self).__init__(name=solid.name, description=solid.description) @@ -102,10 +95,7 @@ def resolve_outputs(self, info): ] def resolve_config(self, info): - return [ - Argument(name=name, argument=argument) - for name, argument in self._solid.config_def.argument_def_dict.items() - ] + return Config(self._solid.config_def) class Input(graphene.ObjectType): @@ -126,7 +116,7 @@ def __init__(self, input_definition, solid, depends_on): self._depends_on = depends_on def resolve_type(self, info): - return Type(dagster_type=self._input_definition.dagster_type) + return Type.from_dagster_type(dagster_type=self._input_definition.dagster_type) def resolve_expectations(self, info): if self._input_definition.expectations: @@ -161,7 +151,7 @@ def __init__(self, output_definition, solid, depended_by): self._depended_by = depended_by def resolve_type(self, info): - return Type(dagster_type=self._output_definition.dagster_type) + return Type.from_dagster_type(dagster_type=self._output_definition.dagster_type) def resolve_expectations(self, info): if self._output_definition.expectations: @@ -182,42 +172,90 @@ def resolve_depends_on(self, info): ] -class Argument(graphene.ObjectType): +class Expectation(graphene.ObjectType): name = graphene.NonNull(graphene.String) description = graphene.String() - type = graphene.NonNull(lambda: Type) - is_optional = graphene.NonNull(graphene.Boolean) - def __init__(self, name, argument): - super(Argument, self).__init__( - name=name, description=argument.description, is_optional=argument.is_optional + def __init__(self, expectation): + super(Expectation, self).__init__( + name=expectation.name, description=expectation.description ) - self._argument = argument + + +class Config(graphene.ObjectType): + type = graphene.NonNull(lambda: Type) + + def __init__(self, config_def): + super(Config, self).__init__() + self._config_def = config_def def resolve_type(self, info): - return Type(dagster_type=self._argument.dagster_type) + return Type.from_dagster_type(dagster_type=self._config_def.config_type) -class Expectation(graphene.ObjectType): +class Type(graphene.Interface): name = graphene.NonNull(graphene.String) description = graphene.String() - def __init__(self, expectation): - super(Expectation, self).__init__( - name=expectation.name, description=expectation.description + @classmethod + def from_dagster_type(self, dagster_type): + if isinstance(dagster_type, DagsterCompositeType): + return CompositeType(dagster_type) + else: + return RegularType(dagster_type) + + +class RegularType(graphene.ObjectType): + class Meta: + interfaces = [ + Type, + ] + + def __init__(self, dagster_type): + super(RegularType, self).__init__( + name=dagster_type.name, + description=dagster_type.description, ) -class Type(graphene.ObjectType): - name = graphene.NonNull(graphene.String) - description = graphene.String() +class CompositeType(graphene.ObjectType): + fields = graphene.NonNull(graphene.List(graphene.NonNull(lambda: TypeField))) + + class Meta: + interfaces = [ + Type, + ] def __init__(self, dagster_type): - super(Type, self).__init__( + super(CompositeType, self).__init__( name=dagster_type.name, description=dagster_type.description, ) + self._dagster_type = dagster_type + + def resolve_fields(self, info): + return [TypeField(name=k, field=v) for k, v in self._dagster_type.field_dict.items()] + + +class TypeField(graphene.ObjectType): + name = graphene.NonNull(graphene.String) + description = graphene.String() + type = graphene.NonNull(lambda: Type) + default_value = graphene.String() + is_optional = graphene.NonNull(graphene.Boolean) + + def __init__(self, name, field): + super(TypeField, self).__init__( + name=name, + description=field.description, + default_value=str(field.default_value), + is_optional=field.is_optional + ) + self._field = field + + def resolve_type(self, info): + return Type.from_dagster_type(dagster_type=self._field.dagster_type) def create_schema(): - return graphene.Schema(query=Query) + return graphene.Schema(query=Query, types=[RegularType, CompositeType]) diff --git a/python_modules/dagit/dagit/webapp/package.json b/python_modules/dagit/dagit/webapp/package.json index 575322ba7c1c5..2b39224c09fef 100644 --- a/python_modules/dagit/dagit/webapp/package.json +++ b/python_modules/dagit/dagit/webapp/package.json @@ -46,8 +46,8 @@ "build": "react-scripts-ts build", "test": "react-scripts-ts test --env=jsdom", "eject": "react-scripts-ts eject", - "download-schema": "apollo schema:download --endpoint $REACT_APP_GRAPHQL_URI ./schema.json", - "generate-types": "apollo codegen:generate --queries \"./src/**/*.tsx\" --target typescript types --schema ./schema.json", - "watch-types": "apollo codegen:generate --queries \"./src/**/*.tsx\" --target typescript types --schema ./schema.json --watch" + "download-schema": "apollo schema:download --endpoint $REACT_APP_GRAPHQL_URI ./src/schema.json", + "generate-types": "apollo codegen:generate --queries \"./src/**/*.tsx\" --target typescript types --schema ./src/schema.json", + "watch-types": "apollo codegen:generate --queries \"./src/**/*.tsx\" --target typescript types --schema ./src/schema.json --watch" } } diff --git a/python_modules/dagit/dagit/webapp/schema.json b/python_modules/dagit/dagit/webapp/schema.json deleted file mode 100644 index caf47807e1813..0000000000000 --- a/python_modules/dagit/dagit/webapp/schema.json +++ /dev/null @@ -1 +0,0 @@ -{"queryType":{"name":"Query"},"mutationType":null,"subscriptionType":null,"types":[{"kind":"OBJECT","name":"Query","description":null,"fields":[{"name":"pipeline","description":null,"args":[{"name":"name","description":null,"type":{"kind":"SCALAR","name":"String","ofType":null},"defaultValue":null}],"type":{"kind":"OBJECT","name":"Pipeline","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"pipelines","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Pipeline","ofType":null}}}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"SCALAR","name":"String","description":"The `String` scalar type represents textual data, represented as UTF-8 character sequences. The String type is most often used by GraphQL to represent free-form human-readable text.","fields":null,"inputFields":null,"interfaces":null,"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Pipeline","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"solids","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Solid","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"context","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"PipelineContext","ofType":null}}}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Solid","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"inputs","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Input","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"outputs","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Output","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"config","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Argument","ofType":null}}}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Input","description":null,"fields":[{"name":"solid","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Solid","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"type","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Type","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"expectations","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Expectation","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"dependsOn","description":null,"args":[],"type":{"kind":"OBJECT","name":"Output","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Type","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Expectation","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Output","description":null,"fields":[{"name":"solid","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Solid","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"type","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Type","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"expectations","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Expectation","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"dependedBy","description":null,"args":[],"type":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Input","ofType":null}}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Argument","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"type","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Type","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"isOptional","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"SCALAR","name":"Boolean","description":"The `Boolean` scalar type represents `true` or `false`.","fields":null,"inputFields":null,"interfaces":null,"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"PipelineContext","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"arguments","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Argument","ofType":null}}}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"__Schema","description":"A GraphQL Schema defines the capabilities of a GraphQL server. It exposes all available types and directives on the server, as well as the entry points for query, mutation, and subscription operations.","fields":[{"name":"types","description":"A list of all types supported by this server.","args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Type","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"queryType","description":"The type that query operations will be rooted at.","args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Type","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"mutationType","description":"If this server supports mutation, the type that mutation operations will be rooted at.","args":[],"type":{"kind":"OBJECT","name":"__Type","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"subscriptionType","description":"If this server support subscription, the type that subscription operations will be rooted at.","args":[],"type":{"kind":"OBJECT","name":"__Type","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"directives","description":"A list of all directives supported by this server.","args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Directive","ofType":null}}}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"__Type","description":"The fundamental unit of any GraphQL Schema is the type. There are many kinds of types in GraphQL as represented by the `__TypeKind` enum.\n\nDepending on the kind of a type, certain fields describe information about that type. Scalar types provide no information beyond a name and description, while Enum types provide their values. Object and Interface types provide the fields they describe. Abstract types, Union and Interface, provide the Object types possible at runtime. List and NonNull types compose other types.","fields":[{"name":"kind","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"ENUM","name":"__TypeKind","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"name","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"fields","description":null,"args":[{"name":"includeDeprecated","description":null,"type":{"kind":"SCALAR","name":"Boolean","ofType":null},"defaultValue":"false"}],"type":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Field","ofType":null}}},"isDeprecated":false,"deprecationReason":null},{"name":"interfaces","description":null,"args":[],"type":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Type","ofType":null}}},"isDeprecated":false,"deprecationReason":null},{"name":"possibleTypes","description":null,"args":[],"type":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Type","ofType":null}}},"isDeprecated":false,"deprecationReason":null},{"name":"enumValues","description":null,"args":[{"name":"includeDeprecated","description":null,"type":{"kind":"SCALAR","name":"Boolean","ofType":null},"defaultValue":"false"}],"type":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__EnumValue","ofType":null}}},"isDeprecated":false,"deprecationReason":null},{"name":"inputFields","description":null,"args":[],"type":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__InputValue","ofType":null}}},"isDeprecated":false,"deprecationReason":null},{"name":"ofType","description":null,"args":[],"type":{"kind":"OBJECT","name":"__Type","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"ENUM","name":"__TypeKind","description":"An enum describing what kind of type a given `__Type` is.","fields":null,"inputFields":null,"interfaces":null,"enumValues":[{"name":"SCALAR","description":"Indicates this type is a scalar.","isDeprecated":false,"deprecationReason":null},{"name":"OBJECT","description":"Indicates this type is an object. `fields` and `interfaces` are valid fields.","isDeprecated":false,"deprecationReason":null},{"name":"INTERFACE","description":"Indicates this type is an interface. `fields` and `possibleTypes` are valid fields.","isDeprecated":false,"deprecationReason":null},{"name":"UNION","description":"Indicates this type is a union. `possibleTypes` is a valid field.","isDeprecated":false,"deprecationReason":null},{"name":"ENUM","description":"Indicates this type is an enum. `enumValues` is a valid field.","isDeprecated":false,"deprecationReason":null},{"name":"INPUT_OBJECT","description":"Indicates this type is an input object. `inputFields` is a valid field.","isDeprecated":false,"deprecationReason":null},{"name":"LIST","description":"Indicates this type is a list. `ofType` is a valid field.","isDeprecated":false,"deprecationReason":null},{"name":"NON_NULL","description":"Indicates this type is a non-null. `ofType` is a valid field.","isDeprecated":false,"deprecationReason":null}],"possibleTypes":null},{"kind":"OBJECT","name":"__Field","description":"Object and Interface types are described by a list of Fields, each of which has a name, potentially a list of arguments, and a return type.","fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"args","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__InputValue","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"type","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Type","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"isDeprecated","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"deprecationReason","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"__InputValue","description":"Arguments provided to Fields or Directives and the input fields of an InputObject are represented as Input Values which describe their type and optionally a default value.","fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"type","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Type","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"defaultValue","description":"A GraphQL-formatted string representing the default value for this input value.","args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"__EnumValue","description":"One possible value for a given Enum. Enum values are unique values, not a placeholder for a string or numeric value. However an Enum value is returned in a JSON response as a string.","fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"isDeprecated","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"deprecationReason","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"__Directive","description":"A Directive provides a way to describe alternate runtime execution and type validation behavior in a GraphQL document.\n\nIn some cases, you need to provide options to alter GraphQL's execution behavior in ways field arguments will not suffice, such as conditionally including or skipping a field. Directives provide this by describing additional information to the executor.","fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"locations","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"ENUM","name":"__DirectiveLocation","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"args","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__InputValue","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"onOperation","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"isDeprecated":true,"deprecationReason":"Use `locations`."},{"name":"onFragment","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"isDeprecated":true,"deprecationReason":"Use `locations`."},{"name":"onField","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"isDeprecated":true,"deprecationReason":"Use `locations`."}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"ENUM","name":"__DirectiveLocation","description":"A Directive can be adjacent to many parts of the GraphQL language, a __DirectiveLocation describes one such possible adjacencies.","fields":null,"inputFields":null,"interfaces":null,"enumValues":[{"name":"QUERY","description":"Location adjacent to a query operation.","isDeprecated":false,"deprecationReason":null},{"name":"MUTATION","description":"Location adjacent to a mutation operation.","isDeprecated":false,"deprecationReason":null},{"name":"SUBSCRIPTION","description":"Location adjacent to a subscription operation.","isDeprecated":false,"deprecationReason":null},{"name":"FIELD","description":"Location adjacent to a field.","isDeprecated":false,"deprecationReason":null},{"name":"FRAGMENT_DEFINITION","description":"Location adjacent to a fragment definition.","isDeprecated":false,"deprecationReason":null},{"name":"FRAGMENT_SPREAD","description":"Location adjacent to a fragment spread.","isDeprecated":false,"deprecationReason":null},{"name":"INLINE_FRAGMENT","description":"Location adjacent to an inline fragment.","isDeprecated":false,"deprecationReason":null},{"name":"SCHEMA","description":"Location adjacent to a schema definition.","isDeprecated":false,"deprecationReason":null},{"name":"SCALAR","description":"Location adjacent to a scalar definition.","isDeprecated":false,"deprecationReason":null},{"name":"OBJECT","description":"Location adjacent to an object type definition.","isDeprecated":false,"deprecationReason":null},{"name":"FIELD_DEFINITION","description":"Location adjacent to a field definition.","isDeprecated":false,"deprecationReason":null},{"name":"ARGUMENT_DEFINITION","description":"Location adjacent to an argument definition.","isDeprecated":false,"deprecationReason":null},{"name":"INTERFACE","description":"Location adjacent to an interface definition.","isDeprecated":false,"deprecationReason":null},{"name":"UNION","description":"Location adjacent to a union definition.","isDeprecated":false,"deprecationReason":null},{"name":"ENUM","description":"Location adjacent to an enum definition.","isDeprecated":false,"deprecationReason":null},{"name":"ENUM_VALUE","description":"Location adjacent to an enum value definition.","isDeprecated":false,"deprecationReason":null},{"name":"INPUT_OBJECT","description":"Location adjacent to an input object type definition.","isDeprecated":false,"deprecationReason":null},{"name":"INPUT_FIELD_DEFINITION","description":"Location adjacent to an input object field definition.","isDeprecated":false,"deprecationReason":null}],"possibleTypes":null}],"directives":[{"name":"include","description":"Directs the executor to include this field or fragment only when the `if` argument is true.","locations":["FIELD","FRAGMENT_SPREAD","INLINE_FRAGMENT"],"args":[{"name":"if","description":"Included when true.","type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"defaultValue":null}]},{"name":"skip","description":"Directs the executor to skip this field or fragment when the `if` argument is true.","locations":["FIELD","FRAGMENT_SPREAD","INLINE_FRAGMENT"],"args":[{"name":"if","description":"Skipped when true.","type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"defaultValue":null}]}]} \ No newline at end of file diff --git a/python_modules/dagit/dagit/webapp/src/Config.tsx b/python_modules/dagit/dagit/webapp/src/Config.tsx new file mode 100644 index 0000000000000..7b4dca300404b --- /dev/null +++ b/python_modules/dagit/dagit/webapp/src/Config.tsx @@ -0,0 +1,82 @@ +import * as React from "react"; +import gql from "graphql-tag"; +import styled from "styled-components"; +import { UL, H6 } from "@blueprintjs/core"; +import SpacedCard from "./SpacedCard"; +import TypeWithTooltip from "./TypeWithTooltip"; +import Description from "./Description"; +import { ConfigFragment } from "./types/ConfigFragment"; + +interface ConfigProps { + config: ConfigFragment; +} + +export default class Config extends React.Component { + static fragments = { + ConfigFragment: gql` + fragment ConfigFragment on Config { + type { + __typename + name + description + ... on CompositeType { + fields { + name + description + isOptional + defaultValue + type { + name + description + ...TypeWithTooltipFragment + } + } + ...TypeWithTooltipFragment + } + } + } + + ${TypeWithTooltip.fragments.TypeWithTooltipFragment} + ` + }; + + renderFields() { + if (this.props.config.type.__typename === "CompositeType") { + return ( +
    + {this.props.config.type.fields.map((field, i: number) => ( +
  • + {field.name} {field.isOptional ? "(optional)" : null}{" "} + + + + +
  • + ))} +
+ ); + } else { + return null; + } + } + + public render() { + return ( + +
Config
+ + {this.renderFields()} +
+ ); + } +} + +const ConfigCard = styled(SpacedCard)` + width: 400px; + margin-bottom: 10px; +`; + +const DescriptionWrapper = styled.div` + max-width: 400px; + margin-bottom: 10px; +`; diff --git a/python_modules/dagit/dagit/webapp/src/Pipeline.tsx b/python_modules/dagit/dagit/webapp/src/Pipeline.tsx index b3345175e6cf8..c1c963888f0e1 100644 --- a/python_modules/dagit/dagit/webapp/src/Pipeline.tsx +++ b/python_modules/dagit/dagit/webapp/src/Pipeline.tsx @@ -6,7 +6,7 @@ import { Switch, Route, match } from "react-router"; import { Link } from "react-router-dom"; import { Card, H2, H5, Text, Code, UL, H6 } from "@blueprintjs/core"; import SpacedCard from "./SpacedCard"; -import Arguments from "./Arguments"; +import Config from "./Config"; import Solid from "./Solid"; import PipelineGraph from "./graph/PipelineGraph"; import { Breadcrumbs, Breadcrumb } from "./Breadcrumbs"; @@ -31,11 +31,11 @@ export default class Pipeline extends React.Component { ...SolidFragment ...SolidListItemFragment } - context { + contexts { name description - arguments { - ...ArgumentsFragment + config { + ...ConfigFragment } } ...PipelineGraphFragment @@ -44,7 +44,7 @@ export default class Pipeline extends React.Component { ${Solid.fragments.SolidFragment} ${PipelineGraph.fragments.PipelineGraphFragment} ${SolidListItem.fragments.SolidListItemFragment} - ${Arguments.fragments.ArgumentsFragment} + ${Config.fragments.ConfigFragment} ` }; @@ -101,16 +101,15 @@ export default class Pipeline extends React.Component { } renderContext() { - return this.props.pipeline.context.map((context: any, i: number) => ( - + return this.props.pipeline.contexts.map((context: any, i: number) => ( +
{context.name}
-
Arguments
- +
)); } diff --git a/python_modules/dagit/dagit/webapp/src/Solid.tsx b/python_modules/dagit/dagit/webapp/src/Solid.tsx index 5ea9ce9cd0161..8c790c45cd509 100644 --- a/python_modules/dagit/dagit/webapp/src/Solid.tsx +++ b/python_modules/dagit/dagit/webapp/src/Solid.tsx @@ -3,7 +3,7 @@ import gql from "graphql-tag"; import styled from "styled-components"; import { Link } from "react-router-dom"; import { H5, H6, Text, Colors, Code, UL } from "@blueprintjs/core"; -import Arguments from "./Arguments"; +import Config from "./Config"; import SpacedCard from "./SpacedCard"; import SolidTypeSignature from "./SolidTypeSignature"; import TypeWithTooltip from "./TypeWithTooltip"; @@ -22,13 +22,13 @@ export default class Solid extends React.Component { name description config { - ...ArgumentsFragment + ...ConfigFragment } inputs { name description type { - ...TypeFragment + ...TypeWithTooltipFragment } expectations { name @@ -45,7 +45,7 @@ export default class Solid extends React.Component { name description type { - ...TypeFragment + ...TypeWithTooltipFragment } expectations { name @@ -58,25 +58,12 @@ export default class Solid extends React.Component { } } - ${TypeWithTooltip.fragments.TypeFragment} + ${TypeWithTooltip.fragments.TypeWithTooltipFragment} ${SolidTypeSignature.fragments.SolidTypeSignatureFragment} - ${Arguments.fragments.ArgumentsFragment} + ${Config.fragments.ConfigFragment} ` }; - renderConfig() { - if (this.props.solid.config.length > 0) { - return ( - <> -
Config
- - - ); - } else { - return null; - } - } - renderInputs() { return this.props.solid.inputs.map((input, i: number) => ( @@ -131,6 +118,17 @@ export default class Solid extends React.Component { )); } + renderSeparator() { + if ( + this.props.solid.inputs.length > 0 && + this.props.solid.outputs.length > 0 + ) { + return ; + } else { + return null; + } + } + public render() { return ( @@ -143,10 +141,10 @@ export default class Solid extends React.Component { - {this.renderConfig()} + {this.renderInputs()} - + {this.renderSeparator()} {this.renderOutputs()} diff --git a/python_modules/dagit/dagit/webapp/src/SolidTypeSignature.tsx b/python_modules/dagit/dagit/webapp/src/SolidTypeSignature.tsx index 77029f0a51122..346e4fbf6a13d 100644 --- a/python_modules/dagit/dagit/webapp/src/SolidTypeSignature.tsx +++ b/python_modules/dagit/dagit/webapp/src/SolidTypeSignature.tsx @@ -19,18 +19,18 @@ export default class SolidTypeSignature extends React.Component< outputs { name type { - ...TypeFragment + ...TypeWithTooltipFragment } } inputs { name type { - ...TypeFragment + ...TypeWithTooltipFragment } } } - ${TypeWithTooltip.fragments.TypeFragment} + ${TypeWithTooltip.fragments.TypeWithTooltipFragment} ` }; diff --git a/python_modules/dagit/dagit/webapp/src/SpacedCard.tsx b/python_modules/dagit/dagit/webapp/src/SpacedCard.tsx index 85b14a483818e..5b83c6417605d 100644 --- a/python_modules/dagit/dagit/webapp/src/SpacedCard.tsx +++ b/python_modules/dagit/dagit/webapp/src/SpacedCard.tsx @@ -2,30 +2,33 @@ import * as React from "react"; import styled, { css } from "styled-components"; import { Card, ICardProps } from "@blueprintjs/core"; -interface ISpacedCardProps { +type ISpacedCardProps = ICardProps & { horizontal?: boolean; -} +}; -const determineMargin = (props: ISpacedCardProps) => - props.horizontal ? horizontal : vertical; +export default class SpacedCard extends React.Component { + render() { + const { horizontal, ...rest } = this.props; + if (horizontal) { + return {this.props.children}; + } else { + return {this.props.children}; + } + } +} -const horizontal = css` +const HorizontalCard = styled(Card)` margin-right: 10px; &:last-child { margin-right: 0; } `; -const vertical = css` + +const VerticalCard = styled(Card)` margin-bottom: 10px; &:last-child { margin-bottom: 0; } `; - -const SpacedCard = styled(Card)` - ${determineMargin}; -`; - -export default SpacedCard; diff --git a/python_modules/dagit/dagit/webapp/src/TypeWithTooltip.tsx b/python_modules/dagit/dagit/webapp/src/TypeWithTooltip.tsx index 863fca8c2dd10..fc1186758df95 100644 --- a/python_modules/dagit/dagit/webapp/src/TypeWithTooltip.tsx +++ b/python_modules/dagit/dagit/webapp/src/TypeWithTooltip.tsx @@ -13,8 +13,8 @@ export default class TypeWithTooltip extends React.Component< {} > { static fragments = { - TypeFragment: gql` - fragment TypeFragment on Type { + TypeWithTooltipFragment: gql` + fragment TypeWithTooltipFragment on Type { name description } diff --git a/python_modules/dagit/dagit/webapp/src/graph/getFullSolidLayout.ts b/python_modules/dagit/dagit/webapp/src/graph/getFullSolidLayout.ts index 351d686f8fac4..88c4cc81ddb0a 100644 --- a/python_modules/dagit/dagit/webapp/src/graph/getFullSolidLayout.ts +++ b/python_modules/dagit/dagit/webapp/src/graph/getFullSolidLayout.ts @@ -73,8 +73,8 @@ export function getDagrePipelineLayout( g.setGraph({ rankdir: "LR", align: "UL", - marginx: 0, - marginy: 0 + marginx: 100, + marginy: 100 }); g.setDefaultEdgeLabel(function() { return {}; diff --git a/python_modules/dagit/dagit/webapp/src/index.tsx b/python_modules/dagit/dagit/webapp/src/index.tsx index 174618100becb..f77ca0d25b330 100644 --- a/python_modules/dagit/dagit/webapp/src/index.tsx +++ b/python_modules/dagit/dagit/webapp/src/index.tsx @@ -2,12 +2,26 @@ import * as React from "react"; import * as ReactDOM from "react-dom"; import { injectGlobal } from "styled-components"; import ApolloClient from "apollo-boost"; +import { + InMemoryCache, + IntrospectionFragmentMatcher +} from "apollo-cache-inmemory"; import { ApolloProvider } from "react-apollo"; import App from "./App"; import "@blueprintjs/core/lib/css/blueprint.css"; import "@blueprintjs/icons/lib/css/blueprint-icons.css"; +import introspectionQueryResultData from "./schema.json"; + +const fragmentMatcher = new IntrospectionFragmentMatcher({ + introspectionQueryResultData: { + __schema: introspectionQueryResultData + } +}); + +const cache = new InMemoryCache({ fragmentMatcher }); const client = new ApolloClient({ + cache, uri: process.env.REACT_APP_GRAPHQL_URI || "/graphql" }); diff --git a/python_modules/dagit/dagit/webapp/src/schema.json b/python_modules/dagit/dagit/webapp/src/schema.json new file mode 100644 index 0000000000000..d288aa6af3b2f --- /dev/null +++ b/python_modules/dagit/dagit/webapp/src/schema.json @@ -0,0 +1 @@ +{"queryType":{"name":"Query"},"mutationType":null,"subscriptionType":null,"types":[{"kind":"OBJECT","name":"Query","description":null,"fields":[{"name":"pipeline","description":null,"args":[{"name":"name","description":null,"type":{"kind":"SCALAR","name":"String","ofType":null},"defaultValue":null}],"type":{"kind":"OBJECT","name":"Pipeline","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"pipelines","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Pipeline","ofType":null}}}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"SCALAR","name":"String","description":"The `String` scalar type represents textual data, represented as UTF-8 character sequences. The String type is most often used by GraphQL to represent free-form human-readable text.","fields":null,"inputFields":null,"interfaces":null,"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Pipeline","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"solids","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Solid","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"contexts","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"PipelineContext","ofType":null}}}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Solid","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"inputs","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Input","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"outputs","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Output","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"config","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Config","ofType":null}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Input","description":null,"fields":[{"name":"solid","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Solid","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"type","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"INTERFACE","name":"Type","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"expectations","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Expectation","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"dependsOn","description":null,"args":[],"type":{"kind":"OBJECT","name":"Output","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"INTERFACE","name":"Type","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":null,"enumValues":null,"possibleTypes":[{"kind":"OBJECT","name":"RegularType","ofType":null},{"kind":"OBJECT","name":"CompositeType","ofType":null}]},{"kind":"OBJECT","name":"Expectation","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Output","description":null,"fields":[{"name":"solid","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Solid","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"type","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"INTERFACE","name":"Type","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"expectations","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Expectation","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"dependedBy","description":null,"args":[],"type":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Input","ofType":null}}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"Config","description":null,"fields":[{"name":"type","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"INTERFACE","name":"Type","ofType":null}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"PipelineContext","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"config","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"Config","ofType":null}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"__Schema","description":"A GraphQL Schema defines the capabilities of a GraphQL server. It exposes all available types and directives on the server, as well as the entry points for query, mutation, and subscription operations.","fields":[{"name":"types","description":"A list of all types supported by this server.","args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Type","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"queryType","description":"The type that query operations will be rooted at.","args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Type","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"mutationType","description":"If this server supports mutation, the type that mutation operations will be rooted at.","args":[],"type":{"kind":"OBJECT","name":"__Type","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"subscriptionType","description":"If this server support subscription, the type that subscription operations will be rooted at.","args":[],"type":{"kind":"OBJECT","name":"__Type","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"directives","description":"A list of all directives supported by this server.","args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Directive","ofType":null}}}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"__Type","description":"The fundamental unit of any GraphQL Schema is the type. There are many kinds of types in GraphQL as represented by the `__TypeKind` enum.\n\nDepending on the kind of a type, certain fields describe information about that type. Scalar types provide no information beyond a name and description, while Enum types provide their values. Object and Interface types provide the fields they describe. Abstract types, Union and Interface, provide the Object types possible at runtime. List and NonNull types compose other types.","fields":[{"name":"kind","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"ENUM","name":"__TypeKind","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"name","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"fields","description":null,"args":[{"name":"includeDeprecated","description":null,"type":{"kind":"SCALAR","name":"Boolean","ofType":null},"defaultValue":"false"}],"type":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Field","ofType":null}}},"isDeprecated":false,"deprecationReason":null},{"name":"interfaces","description":null,"args":[],"type":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Type","ofType":null}}},"isDeprecated":false,"deprecationReason":null},{"name":"possibleTypes","description":null,"args":[],"type":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Type","ofType":null}}},"isDeprecated":false,"deprecationReason":null},{"name":"enumValues","description":null,"args":[{"name":"includeDeprecated","description":null,"type":{"kind":"SCALAR","name":"Boolean","ofType":null},"defaultValue":"false"}],"type":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__EnumValue","ofType":null}}},"isDeprecated":false,"deprecationReason":null},{"name":"inputFields","description":null,"args":[],"type":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__InputValue","ofType":null}}},"isDeprecated":false,"deprecationReason":null},{"name":"ofType","description":null,"args":[],"type":{"kind":"OBJECT","name":"__Type","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"ENUM","name":"__TypeKind","description":"An enum describing what kind of type a given `__Type` is.","fields":null,"inputFields":null,"interfaces":null,"enumValues":[{"name":"SCALAR","description":"Indicates this type is a scalar.","isDeprecated":false,"deprecationReason":null},{"name":"OBJECT","description":"Indicates this type is an object. `fields` and `interfaces` are valid fields.","isDeprecated":false,"deprecationReason":null},{"name":"INTERFACE","description":"Indicates this type is an interface. `fields` and `possibleTypes` are valid fields.","isDeprecated":false,"deprecationReason":null},{"name":"UNION","description":"Indicates this type is a union. `possibleTypes` is a valid field.","isDeprecated":false,"deprecationReason":null},{"name":"ENUM","description":"Indicates this type is an enum. `enumValues` is a valid field.","isDeprecated":false,"deprecationReason":null},{"name":"INPUT_OBJECT","description":"Indicates this type is an input object. `inputFields` is a valid field.","isDeprecated":false,"deprecationReason":null},{"name":"LIST","description":"Indicates this type is a list. `ofType` is a valid field.","isDeprecated":false,"deprecationReason":null},{"name":"NON_NULL","description":"Indicates this type is a non-null. `ofType` is a valid field.","isDeprecated":false,"deprecationReason":null}],"possibleTypes":null},{"kind":"SCALAR","name":"Boolean","description":"The `Boolean` scalar type represents `true` or `false`.","fields":null,"inputFields":null,"interfaces":null,"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"__Field","description":"Object and Interface types are described by a list of Fields, each of which has a name, potentially a list of arguments, and a return type.","fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"args","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__InputValue","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"type","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Type","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"isDeprecated","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"deprecationReason","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"__InputValue","description":"Arguments provided to Fields or Directives and the input fields of an InputObject are represented as Input Values which describe their type and optionally a default value.","fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"type","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__Type","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"defaultValue","description":"A GraphQL-formatted string representing the default value for this input value.","args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"__EnumValue","description":"One possible value for a given Enum. Enum values are unique values, not a placeholder for a string or numeric value. However an Enum value is returned in a JSON response as a string.","fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"isDeprecated","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"deprecationReason","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"__Directive","description":"A Directive provides a way to describe alternate runtime execution and type validation behavior in a GraphQL document.\n\nIn some cases, you need to provide options to alter GraphQL's execution behavior in ways field arguments will not suffice, such as conditionally including or skipping a field. Directives provide this by describing additional information to the executor.","fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"locations","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"ENUM","name":"__DirectiveLocation","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"args","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"__InputValue","ofType":null}}}},"isDeprecated":false,"deprecationReason":null},{"name":"onOperation","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"isDeprecated":true,"deprecationReason":"Use `locations`."},{"name":"onFragment","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"isDeprecated":true,"deprecationReason":"Use `locations`."},{"name":"onField","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"isDeprecated":true,"deprecationReason":"Use `locations`."}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null},{"kind":"ENUM","name":"__DirectiveLocation","description":"A Directive can be adjacent to many parts of the GraphQL language, a __DirectiveLocation describes one such possible adjacencies.","fields":null,"inputFields":null,"interfaces":null,"enumValues":[{"name":"QUERY","description":"Location adjacent to a query operation.","isDeprecated":false,"deprecationReason":null},{"name":"MUTATION","description":"Location adjacent to a mutation operation.","isDeprecated":false,"deprecationReason":null},{"name":"SUBSCRIPTION","description":"Location adjacent to a subscription operation.","isDeprecated":false,"deprecationReason":null},{"name":"FIELD","description":"Location adjacent to a field.","isDeprecated":false,"deprecationReason":null},{"name":"FRAGMENT_DEFINITION","description":"Location adjacent to a fragment definition.","isDeprecated":false,"deprecationReason":null},{"name":"FRAGMENT_SPREAD","description":"Location adjacent to a fragment spread.","isDeprecated":false,"deprecationReason":null},{"name":"INLINE_FRAGMENT","description":"Location adjacent to an inline fragment.","isDeprecated":false,"deprecationReason":null},{"name":"SCHEMA","description":"Location adjacent to a schema definition.","isDeprecated":false,"deprecationReason":null},{"name":"SCALAR","description":"Location adjacent to a scalar definition.","isDeprecated":false,"deprecationReason":null},{"name":"OBJECT","description":"Location adjacent to an object type definition.","isDeprecated":false,"deprecationReason":null},{"name":"FIELD_DEFINITION","description":"Location adjacent to a field definition.","isDeprecated":false,"deprecationReason":null},{"name":"ARGUMENT_DEFINITION","description":"Location adjacent to an argument definition.","isDeprecated":false,"deprecationReason":null},{"name":"INTERFACE","description":"Location adjacent to an interface definition.","isDeprecated":false,"deprecationReason":null},{"name":"UNION","description":"Location adjacent to a union definition.","isDeprecated":false,"deprecationReason":null},{"name":"ENUM","description":"Location adjacent to an enum definition.","isDeprecated":false,"deprecationReason":null},{"name":"ENUM_VALUE","description":"Location adjacent to an enum value definition.","isDeprecated":false,"deprecationReason":null},{"name":"INPUT_OBJECT","description":"Location adjacent to an input object type definition.","isDeprecated":false,"deprecationReason":null},{"name":"INPUT_FIELD_DEFINITION","description":"Location adjacent to an input object field definition.","isDeprecated":false,"deprecationReason":null}],"possibleTypes":null},{"kind":"OBJECT","name":"RegularType","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[{"kind":"INTERFACE","name":"Type","ofType":null}],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"CompositeType","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"fields","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"LIST","name":null,"ofType":{"kind":"NON_NULL","name":null,"ofType":{"kind":"OBJECT","name":"TypeField","ofType":null}}}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[{"kind":"INTERFACE","name":"Type","ofType":null}],"enumValues":null,"possibleTypes":null},{"kind":"OBJECT","name":"TypeField","description":null,"fields":[{"name":"name","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"String","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"description","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"type","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"INTERFACE","name":"Type","ofType":null}},"isDeprecated":false,"deprecationReason":null},{"name":"defaultValue","description":null,"args":[],"type":{"kind":"SCALAR","name":"String","ofType":null},"isDeprecated":false,"deprecationReason":null},{"name":"isOptional","description":null,"args":[],"type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"isDeprecated":false,"deprecationReason":null}],"inputFields":null,"interfaces":[],"enumValues":null,"possibleTypes":null}],"directives":[{"name":"include","description":"Directs the executor to include this field or fragment only when the `if` argument is true.","locations":["FIELD","FRAGMENT_SPREAD","INLINE_FRAGMENT"],"args":[{"name":"if","description":"Included when true.","type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"defaultValue":null}]},{"name":"skip","description":"Directs the executor to skip this field or fragment when the `if` argument is true.","locations":["FIELD","FRAGMENT_SPREAD","INLINE_FRAGMENT"],"args":[{"name":"if","description":"Skipped when true.","type":{"kind":"NON_NULL","name":null,"ofType":{"kind":"SCALAR","name":"Boolean","ofType":null}},"defaultValue":null}]}]} \ No newline at end of file diff --git a/python_modules/dagit/dagit/webapp/src/types/AppQuery.ts b/python_modules/dagit/dagit/webapp/src/types/AppQuery.ts index fbc2706651ec1..ae6e58cd8a467 100644 --- a/python_modules/dagit/dagit/webapp/src/types/AppQuery.ts +++ b/python_modules/dagit/dagit/webapp/src/types/AppQuery.ts @@ -51,16 +51,36 @@ export interface AppQuery_pipelines_solids_inputs { dependsOn: AppQuery_pipelines_solids_inputs_dependsOn | null; } -export interface AppQuery_pipelines_solids_config_type { +export interface AppQuery_pipelines_solids_config_type_RegularType { + __typename: "RegularType"; name: string; description: string | null; } -export interface AppQuery_pipelines_solids_config { +export interface AppQuery_pipelines_solids_config_type_CompositeType_fields_type { + name: string; + description: string | null; +} + +export interface AppQuery_pipelines_solids_config_type_CompositeType_fields { name: string; description: string | null; - type: AppQuery_pipelines_solids_config_type; isOptional: boolean; + defaultValue: string | null; + type: AppQuery_pipelines_solids_config_type_CompositeType_fields_type; +} + +export interface AppQuery_pipelines_solids_config_type_CompositeType { + __typename: "CompositeType"; + name: string; + description: string | null; + fields: AppQuery_pipelines_solids_config_type_CompositeType_fields[]; +} + +export type AppQuery_pipelines_solids_config_type = AppQuery_pipelines_solids_config_type_RegularType | AppQuery_pipelines_solids_config_type_CompositeType; + +export interface AppQuery_pipelines_solids_config { + type: AppQuery_pipelines_solids_config_type; } export interface AppQuery_pipelines_solids { @@ -68,32 +88,52 @@ export interface AppQuery_pipelines_solids { inputs: AppQuery_pipelines_solids_inputs[]; name: string; description: string | null; - config: AppQuery_pipelines_solids_config[]; + config: AppQuery_pipelines_solids_config; } -export interface AppQuery_pipelines_context_arguments_type { +export interface AppQuery_pipelines_contexts_config_type_RegularType { + __typename: "RegularType"; name: string; description: string | null; } -export interface AppQuery_pipelines_context_arguments { +export interface AppQuery_pipelines_contexts_config_type_CompositeType_fields_type { + name: string; + description: string | null; +} + +export interface AppQuery_pipelines_contexts_config_type_CompositeType_fields { name: string; description: string | null; - type: AppQuery_pipelines_context_arguments_type; isOptional: boolean; + defaultValue: string | null; + type: AppQuery_pipelines_contexts_config_type_CompositeType_fields_type; +} + +export interface AppQuery_pipelines_contexts_config_type_CompositeType { + __typename: "CompositeType"; + name: string; + description: string | null; + fields: AppQuery_pipelines_contexts_config_type_CompositeType_fields[]; +} + +export type AppQuery_pipelines_contexts_config_type = AppQuery_pipelines_contexts_config_type_RegularType | AppQuery_pipelines_contexts_config_type_CompositeType; + +export interface AppQuery_pipelines_contexts_config { + type: AppQuery_pipelines_contexts_config_type; } -export interface AppQuery_pipelines_context { +export interface AppQuery_pipelines_contexts { name: string; description: string | null; - arguments: AppQuery_pipelines_context_arguments[]; + config: AppQuery_pipelines_contexts_config; } export interface AppQuery_pipelines { name: string; description: string | null; solids: AppQuery_pipelines_solids[]; - context: AppQuery_pipelines_context[]; + contexts: AppQuery_pipelines_contexts[]; } export interface AppQuery { diff --git a/python_modules/dagit/dagit/webapp/src/types/MaterializationFragment.ts b/python_modules/dagit/dagit/webapp/src/types/ConfigFragment.ts similarity index 53% rename from python_modules/dagit/dagit/webapp/src/types/MaterializationFragment.ts rename to python_modules/dagit/dagit/webapp/src/types/ConfigFragment.ts index bc17dcf983582..b0af02d9d8c4a 100644 --- a/python_modules/dagit/dagit/webapp/src/types/MaterializationFragment.ts +++ b/python_modules/dagit/dagit/webapp/src/types/ConfigFragment.ts @@ -4,25 +4,39 @@ // This file was automatically generated and should not be edited. // ==================================================== -// GraphQL fragment: MaterializationFragment +// GraphQL fragment: ConfigFragment // ==================================================== -export interface MaterializationFragment_arguments_type { +export interface ConfigFragment_type_RegularType { + __typename: "RegularType"; name: string; description: string | null; } -export interface MaterializationFragment_arguments { +export interface ConfigFragment_type_CompositeType_fields_type { + name: string; + description: string | null; +} + +export interface ConfigFragment_type_CompositeType_fields { name: string; description: string | null; - type: MaterializationFragment_arguments_type; isOptional: boolean; + defaultValue: string | null; + type: ConfigFragment_type_CompositeType_fields_type; } -export interface MaterializationFragment { +export interface ConfigFragment_type_CompositeType { + __typename: "CompositeType"; name: string; description: string | null; - arguments: MaterializationFragment_arguments[]; + fields: ConfigFragment_type_CompositeType_fields[]; +} + +export type ConfigFragment_type = ConfigFragment_type_RegularType | ConfigFragment_type_CompositeType; + +export interface ConfigFragment { + type: ConfigFragment_type; } /* tslint:disable */ diff --git a/python_modules/dagit/dagit/webapp/src/types/PipelineFragment.ts b/python_modules/dagit/dagit/webapp/src/types/PipelineFragment.ts index d9fc4d53120cc..a6a6cdb5d9769 100644 --- a/python_modules/dagit/dagit/webapp/src/types/PipelineFragment.ts +++ b/python_modules/dagit/dagit/webapp/src/types/PipelineFragment.ts @@ -51,16 +51,36 @@ export interface PipelineFragment_solids_inputs { dependsOn: PipelineFragment_solids_inputs_dependsOn | null; } -export interface PipelineFragment_solids_config_type { +export interface PipelineFragment_solids_config_type_RegularType { + __typename: "RegularType"; name: string; description: string | null; } -export interface PipelineFragment_solids_config { +export interface PipelineFragment_solids_config_type_CompositeType_fields_type { + name: string; + description: string | null; +} + +export interface PipelineFragment_solids_config_type_CompositeType_fields { name: string; description: string | null; - type: PipelineFragment_solids_config_type; isOptional: boolean; + defaultValue: string | null; + type: PipelineFragment_solids_config_type_CompositeType_fields_type; +} + +export interface PipelineFragment_solids_config_type_CompositeType { + __typename: "CompositeType"; + name: string; + description: string | null; + fields: PipelineFragment_solids_config_type_CompositeType_fields[]; +} + +export type PipelineFragment_solids_config_type = PipelineFragment_solids_config_type_RegularType | PipelineFragment_solids_config_type_CompositeType; + +export interface PipelineFragment_solids_config { + type: PipelineFragment_solids_config_type; } export interface PipelineFragment_solids { @@ -68,32 +88,52 @@ export interface PipelineFragment_solids { inputs: PipelineFragment_solids_inputs[]; name: string; description: string | null; - config: PipelineFragment_solids_config[]; + config: PipelineFragment_solids_config; } -export interface PipelineFragment_context_arguments_type { +export interface PipelineFragment_contexts_config_type_RegularType { + __typename: "RegularType"; name: string; description: string | null; } -export interface PipelineFragment_context_arguments { +export interface PipelineFragment_contexts_config_type_CompositeType_fields_type { + name: string; + description: string | null; +} + +export interface PipelineFragment_contexts_config_type_CompositeType_fields { name: string; description: string | null; - type: PipelineFragment_context_arguments_type; isOptional: boolean; + defaultValue: string | null; + type: PipelineFragment_contexts_config_type_CompositeType_fields_type; +} + +export interface PipelineFragment_contexts_config_type_CompositeType { + __typename: "CompositeType"; + name: string; + description: string | null; + fields: PipelineFragment_contexts_config_type_CompositeType_fields[]; +} + +export type PipelineFragment_contexts_config_type = PipelineFragment_contexts_config_type_RegularType | PipelineFragment_contexts_config_type_CompositeType; + +export interface PipelineFragment_contexts_config { + type: PipelineFragment_contexts_config_type; } -export interface PipelineFragment_context { +export interface PipelineFragment_contexts { name: string; description: string | null; - arguments: PipelineFragment_context_arguments[]; + config: PipelineFragment_contexts_config; } export interface PipelineFragment { name: string; description: string | null; solids: PipelineFragment_solids[]; - context: PipelineFragment_context[]; + contexts: PipelineFragment_contexts[]; } /* tslint:disable */ diff --git a/python_modules/dagit/dagit/webapp/src/types/PipelinesFragment.ts b/python_modules/dagit/dagit/webapp/src/types/PipelinesFragment.ts index 2ab384ed98a20..8310e093406d2 100644 --- a/python_modules/dagit/dagit/webapp/src/types/PipelinesFragment.ts +++ b/python_modules/dagit/dagit/webapp/src/types/PipelinesFragment.ts @@ -51,16 +51,36 @@ export interface PipelinesFragment_solids_inputs { dependsOn: PipelinesFragment_solids_inputs_dependsOn | null; } -export interface PipelinesFragment_solids_config_type { +export interface PipelinesFragment_solids_config_type_RegularType { + __typename: "RegularType"; name: string; description: string | null; } -export interface PipelinesFragment_solids_config { +export interface PipelinesFragment_solids_config_type_CompositeType_fields_type { + name: string; + description: string | null; +} + +export interface PipelinesFragment_solids_config_type_CompositeType_fields { name: string; description: string | null; - type: PipelinesFragment_solids_config_type; isOptional: boolean; + defaultValue: string | null; + type: PipelinesFragment_solids_config_type_CompositeType_fields_type; +} + +export interface PipelinesFragment_solids_config_type_CompositeType { + __typename: "CompositeType"; + name: string; + description: string | null; + fields: PipelinesFragment_solids_config_type_CompositeType_fields[]; +} + +export type PipelinesFragment_solids_config_type = PipelinesFragment_solids_config_type_RegularType | PipelinesFragment_solids_config_type_CompositeType; + +export interface PipelinesFragment_solids_config { + type: PipelinesFragment_solids_config_type; } export interface PipelinesFragment_solids { @@ -68,32 +88,52 @@ export interface PipelinesFragment_solids { inputs: PipelinesFragment_solids_inputs[]; name: string; description: string | null; - config: PipelinesFragment_solids_config[]; + config: PipelinesFragment_solids_config; } -export interface PipelinesFragment_context_arguments_type { +export interface PipelinesFragment_contexts_config_type_RegularType { + __typename: "RegularType"; name: string; description: string | null; } -export interface PipelinesFragment_context_arguments { +export interface PipelinesFragment_contexts_config_type_CompositeType_fields_type { + name: string; + description: string | null; +} + +export interface PipelinesFragment_contexts_config_type_CompositeType_fields { name: string; description: string | null; - type: PipelinesFragment_context_arguments_type; isOptional: boolean; + defaultValue: string | null; + type: PipelinesFragment_contexts_config_type_CompositeType_fields_type; +} + +export interface PipelinesFragment_contexts_config_type_CompositeType { + __typename: "CompositeType"; + name: string; + description: string | null; + fields: PipelinesFragment_contexts_config_type_CompositeType_fields[]; +} + +export type PipelinesFragment_contexts_config_type = PipelinesFragment_contexts_config_type_RegularType | PipelinesFragment_contexts_config_type_CompositeType; + +export interface PipelinesFragment_contexts_config { + type: PipelinesFragment_contexts_config_type; } -export interface PipelinesFragment_context { +export interface PipelinesFragment_contexts { name: string; description: string | null; - arguments: PipelinesFragment_context_arguments[]; + config: PipelinesFragment_contexts_config; } export interface PipelinesFragment { name: string; description: string | null; solids: PipelinesFragment_solids[]; - context: PipelinesFragment_context[]; + contexts: PipelinesFragment_contexts[]; } /* tslint:disable */ diff --git a/python_modules/dagit/dagit/webapp/src/types/SolidFragment.ts b/python_modules/dagit/dagit/webapp/src/types/SolidFragment.ts index 599c55342195a..eb84c66b8ac5a 100644 --- a/python_modules/dagit/dagit/webapp/src/types/SolidFragment.ts +++ b/python_modules/dagit/dagit/webapp/src/types/SolidFragment.ts @@ -51,16 +51,36 @@ export interface SolidFragment_inputs { dependsOn: SolidFragment_inputs_dependsOn | null; } -export interface SolidFragment_config_type { +export interface SolidFragment_config_type_RegularType { + __typename: "RegularType"; name: string; description: string | null; } -export interface SolidFragment_config { +export interface SolidFragment_config_type_CompositeType_fields_type { + name: string; + description: string | null; +} + +export interface SolidFragment_config_type_CompositeType_fields { name: string; description: string | null; - type: SolidFragment_config_type; isOptional: boolean; + defaultValue: string | null; + type: SolidFragment_config_type_CompositeType_fields_type; +} + +export interface SolidFragment_config_type_CompositeType { + __typename: "CompositeType"; + name: string; + description: string | null; + fields: SolidFragment_config_type_CompositeType_fields[]; +} + +export type SolidFragment_config_type = SolidFragment_config_type_RegularType | SolidFragment_config_type_CompositeType; + +export interface SolidFragment_config { + type: SolidFragment_config_type; } export interface SolidFragment { @@ -68,7 +88,7 @@ export interface SolidFragment { inputs: SolidFragment_inputs[]; name: string; description: string | null; - config: SolidFragment_config[]; + config: SolidFragment_config; } /* tslint:disable */ diff --git a/python_modules/dagit/dagit/webapp/src/types/SourceFragment.ts b/python_modules/dagit/dagit/webapp/src/types/SourceFragment.ts deleted file mode 100644 index d58b34f54d6a7..0000000000000 --- a/python_modules/dagit/dagit/webapp/src/types/SourceFragment.ts +++ /dev/null @@ -1,37 +0,0 @@ - - -/* tslint:disable */ -// This file was automatically generated and should not be edited. - -// ==================================================== -// GraphQL fragment: SourceFragment -// ==================================================== - -export interface SourceFragment_arguments_type { - name: string; - description: string | null; -} - -export interface SourceFragment_arguments { - name: string; - description: string | null; - type: SourceFragment_arguments_type; - isOptional: boolean; -} - -export interface SourceFragment { - name: string; - description: string | null; - arguments: SourceFragment_arguments[]; -} - -/* tslint:disable */ -// This file was automatically generated and should not be edited. - -//============================================================== -// START Enums and Input Objects -//============================================================== - -//============================================================== -// END Enums and Input Objects -//============================================================== \ No newline at end of file diff --git a/python_modules/dagit/dagit/webapp/src/types/TypeFragment.ts b/python_modules/dagit/dagit/webapp/src/types/TypeWithTooltipFragment.ts similarity index 88% rename from python_modules/dagit/dagit/webapp/src/types/TypeFragment.ts rename to python_modules/dagit/dagit/webapp/src/types/TypeWithTooltipFragment.ts index 32444ac7735e2..d6862de356d19 100644 --- a/python_modules/dagit/dagit/webapp/src/types/TypeFragment.ts +++ b/python_modules/dagit/dagit/webapp/src/types/TypeWithTooltipFragment.ts @@ -4,10 +4,10 @@ // This file was automatically generated and should not be edited. // ==================================================== -// GraphQL fragment: TypeFragment +// GraphQL fragment: TypeWithTooltipFragment // ==================================================== -export interface TypeFragment { +export interface TypeWithTooltipFragment { name: string; description: string | null; } diff --git a/python_modules/dagit/dagit/webapp/typings.d.ts b/python_modules/dagit/dagit/webapp/typings.d.ts index 7c28ace371dd2..064a649b39b1a 100644 --- a/python_modules/dagit/dagit/webapp/typings.d.ts +++ b/python_modules/dagit/dagit/webapp/typings.d.ts @@ -5,3 +5,8 @@ declare module "@vx/shape"; declare module "@vx/legend"; declare module "@vx/scale"; declare module "@vx/responsive"; + +declare module "*.json" { + const value: any; + export default value; +} diff --git a/python_modules/dagit/webapp/src/Arguments.tsx b/python_modules/dagit/webapp/src/Arguments.tsx deleted file mode 100644 index c0702292706fb..0000000000000 --- a/python_modules/dagit/webapp/src/Arguments.tsx +++ /dev/null @@ -1,49 +0,0 @@ -import * as React from "react"; -import gql from "graphql-tag"; -import styled from "styled-components"; -import { UL } from "@blueprintjs/core"; -import TypeWithTooltip from "./TypeWithTooltip"; -import Description from "./Description"; -import { ArgumentsFragment } from "./types/ArgumentsFragment"; - -interface ArgumentsProps { - arguments: Array; -} - -export default class Arguments extends React.Component { - static fragments = { - ArgumentsFragment: gql` - fragment ArgumentsFragment on Argument { - name - description - type { - ...TypeFragment - } - isOptional - } - - ${TypeWithTooltip.fragments.TypeFragment} - ` - }; - - public render() { - return ( -
    - {this.props.arguments.map((argument: any, i: number) => ( -
  • - {argument.name} {argument.isOptional ? "(optional)" : null}{" "} - - - - -
  • - ))} -
- ); - } -} - -const DescriptionWrapper = styled.div` - max-width: 400px; - margin-bottom: 10px; -`; diff --git a/python_modules/dagit/webapp/src/types/ArgumentsFragment.ts b/python_modules/dagit/webapp/src/types/ArgumentsFragment.ts deleted file mode 100644 index a0aed84ee9063..0000000000000 --- a/python_modules/dagit/webapp/src/types/ArgumentsFragment.ts +++ /dev/null @@ -1,31 +0,0 @@ - - -/* tslint:disable */ -// This file was automatically generated and should not be edited. - -// ==================================================== -// GraphQL fragment: ArgumentsFragment -// ==================================================== - -export interface ArgumentsFragment_type { - name: string; - description: string | null; -} - -export interface ArgumentsFragment { - name: string; - description: string | null; - type: ArgumentsFragment_type; - isOptional: boolean; -} - -/* tslint:disable */ -// This file was automatically generated and should not be edited. - -//============================================================== -// START Enums and Input Objects -//============================================================== - -//============================================================== -// END Enums and Input Objects -//============================================================== \ No newline at end of file From 75eeb74ff7e40a99fb7dcf50e2653d07b4bf5b2a Mon Sep 17 00:00:00 2001 From: Mikhail Novikov Date: Tue, 4 Sep 2018 16:13:30 +0300 Subject: [PATCH 049/103] Add apidocs --- .gitignore | 1 + python_modules/dagster/docs/apidocs/index.rst | 38 +++++++++++++++++++ python_modules/dagster/docs/index.rst | 9 +++++ 3 files changed, 48 insertions(+) create mode 100644 python_modules/dagster/docs/apidocs/index.rst diff --git a/.gitignore b/.gitignore index 7646446a33297..aeb716b02c4b2 100644 --- a/.gitignore +++ b/.gitignore @@ -109,3 +109,4 @@ tags *.db docs/_build +python_modules/dagster/docs/_build diff --git a/python_modules/dagster/docs/apidocs/index.rst b/python_modules/dagster/docs/apidocs/index.rst new file mode 100644 index 0000000000000..b5185320cca63 --- /dev/null +++ b/python_modules/dagster/docs/apidocs/index.rst @@ -0,0 +1,38 @@ +API Reference +=================================== + +dagster.core.definitions +----------------------------------- + +.. automodule:: dagster.core.definitions + :members: + +dagster.core.execution +----------------------------------- + +.. automodule:: dagster.core.execution + :members: + +dagster.core.execution_context +----------------------------------- + +.. automodule:: dagster.core.execution_context + :members: + +dagster.core.errors +----------------------------------- + +.. automodule:: dagster.core.errors + :members: + +dagster.core.types +----------------------------------- + +.. automodule:: dagster.core.types + :members: + +dagster.core.decorators +----------------------------------- + +.. automodule:: dagster.core.decorators + :members: diff --git a/python_modules/dagster/docs/index.rst b/python_modules/dagster/docs/index.rst index da3d39405bcb2..b0b3d68ed0e95 100644 --- a/python_modules/dagster/docs/index.rst +++ b/python_modules/dagster/docs/index.rst @@ -20,6 +20,15 @@ Other docs developing/index +API Reference +================ + +.. toctree:: + :maxdepth: 2 + :caption: Contents: + + apidocs/index + Indices and tables ================== From d1f472487fda11ce8b3f6144e9405f68a11c03dc Mon Sep 17 00:00:00 2001 From: Mikhail Novikov Date: Tue, 4 Sep 2018 16:19:41 +0300 Subject: [PATCH 050/103] RTD requirements --- .read-the-docs-requirements.txt | 1 + 1 file changed, 1 insertion(+) create mode 100644 .read-the-docs-requirements.txt diff --git a/.read-the-docs-requirements.txt b/.read-the-docs-requirements.txt new file mode 100644 index 0000000000000..de2033a61af48 --- /dev/null +++ b/.read-the-docs-requirements.txt @@ -0,0 +1 @@ +./python_modules/dagster From 580ae7f6708a95c48e54599b3b2af69760801618 Mon Sep 17 00:00:00 2001 From: Mikhail Novikov Date: Tue, 4 Sep 2018 16:55:31 +0300 Subject: [PATCH 051/103] Enable napoleon --- python_modules/dagster/docs/conf.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python_modules/dagster/docs/conf.py b/python_modules/dagster/docs/conf.py index 25e2157f109ed..318956764edcb 100644 --- a/python_modules/dagster/docs/conf.py +++ b/python_modules/dagster/docs/conf.py @@ -154,3 +154,5 @@ ] # -- Extension configuration ------------------------------------------------- + +extensions = ['sphinx.ext.napoleon'] From c1fc6b395575397882b74bc752deccbd5ea1883b Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Tue, 4 Sep 2018 12:36:04 -0700 Subject: [PATCH 052/103] Restructure docs and being docstring conversion Provides top-level organization to documentation into a few sections: - Motivations - Principles - Topic Guides - Tutorials Set up API refernece to only take things in the __all__ variable and then converted a few classes to the proper format. --- python_modules/dagster/dagster/__init__.py | 11 + .../dagster/dagster/core/definitions.py | 340 +++++++++++------- python_modules/dagster/dagster/core/types.py | 20 ++ python_modules/dagster/docs/apidocs/index.rst | 41 +-- python_modules/dagster/docs/index.rst | 22 +- python_modules/dagster/docs/motivations.rst | 3 + python_modules/dagster/docs/principles.rst | 63 ++++ python_modules/dagster/docs/topic_guides.rst | 0 python_modules/dagster/docs/tutorials.rst | 0 9 files changed, 334 insertions(+), 166 deletions(-) create mode 100644 python_modules/dagster/docs/motivations.rst create mode 100644 python_modules/dagster/docs/principles.rst create mode 100644 python_modules/dagster/docs/topic_guides.rst create mode 100644 python_modules/dagster/docs/tutorials.rst diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index f8cb4d035c1b6..05b55b09b7547 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -32,3 +32,14 @@ ) import dagster.core.types as types + +__all__ = [ + 'ConfigDefinition', + 'DependencyDefinition', + 'ExpectationDefinition', + 'ExpectationResult', + 'Field', + 'InputDefinition', + 'OutputDefinition', + 'PipelineDefinition', +] diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index 970655dc24150..c562f23544ebc 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -2,7 +2,6 @@ defaultdict, namedtuple, ) -import copy import keyword import re from toposort import toposort_flatten @@ -75,12 +74,11 @@ class PipelineContextDefinition(object): a dictionary key'ed by its name so the name is not present in this object. Attributes: - config_def: A ConfigurationDefinition. A context definition - defines the configuration necessary to instantiate a context. - This is where the that description goes - context_fn: A callable that either returns *or* yields an ExecutionContext. + config_def (ConfigurationDefinition): + context_fn (callable): + A callable that either returns *or* yields an ExecutionContext. Type sig: (pipeline: PipelineDefinition, config: Any) : ExecutionContext - description: A description of what this context represents + description (str): A description of what this context represents ''' @staticmethod @@ -159,22 +157,18 @@ class DependencyDefinition(namedtuple('_DependencyDefinition', 'solid output des comes from) so this object only contains the target dependency information. Attributes: - solid: str + solid (str): The name of the solid that is the target of the dependency. This is the solid where the value passed between the solids comes from. - output: str (optional) + output (str): The name of the output that is the target of the dependency. - description: str (optional) + Defaults to "result", the default output name of solids with a single output. + description (str): + Description of this dependency. Optional. ''' def __new__(cls, solid, output=DEFAULT_OUTPUT, description=None): - ''' - Args: - solid: str - output: str (optional) defaults to 'result' - description: str (optional) - ''' return super(DependencyDefinition, cls).__new__( cls, check.str_param(solid, 'solid'), @@ -208,7 +202,7 @@ def check_opt_two_dim_str_dict(ddict, param_name, value_type): return ddict -def create_handle_dict(solid_dict, dep_dict): +def _create_handle_dict(solid_dict, dep_dict): check.dict_param(solid_dict, 'solid_dict', key_type=str, value_type=SolidDefinition) check_two_dim_str_dict(dep_dict, 'dep_dict', DependencyDefinition) @@ -228,7 +222,7 @@ def create_handle_dict(solid_dict, dep_dict): class DependencyStructure(object): @staticmethod def from_definitions(solids, dep_dict): - return DependencyStructure(create_handle_dict(_build_named_dict(solids), dep_dict)) + return DependencyStructure(_create_handle_dict(_build_named_dict(solids), dep_dict)) def __init__(self, handle_dict): self._handle_dict = check.inst_param(handle_dict, 'handle_dict', InputToOutputHandleDict) @@ -277,26 +271,90 @@ def _build_named_dict(things): class PipelineDefinition(object): - ''' A instance of a PipelineDefinition represents a pipeline in dagster. + '''A instance of a PipelineDefinition represents a pipeline in dagster. A pipeline is comprised of: - - Solids. Each solid represents a functional unit of data computation. - - Context Definitions. Pipelines can be designed to execute in a number of - different operating environments (e.g. prod, dev, unittest) that require - different configuration and setup. A context definition defines how a context - (of type ExecutionContext) is created and what configuration is necessary to create it. - - Dependencies. Solids within a pipeline are arranged as a DAG (directed, acyclic graph). - Dependencies determine how the values produced by solids flow through the DAG. + - Solids: + Each solid represents a functional unit of data computation. + + - Context Definitions: + Pipelines can be designed to execute in a number of different operating environments + (e.g. prod, dev, unittest) that require different configuration and setup. A context + definition defines how a context (of type ExecutionContext) is created and what + configuration is necessary to create it. + + - Dependencies: + Solids within a pipeline are arranged as a DAG (directed, acyclic graph). Dependencies + determine how the values produced by solids flow through the DAG. Attributes: - name: - description: - solids: - dependencies: - dependency_structure: + name (str): + Name of the pipeline. Must be unique per-repository. + description (str): + Description of the pipeline. Optional. + solids (List[SolidDefinition]): + List of the solids in this pipeline. + dependencies (Dict[str, Dict[str, DependencyDefinition]]) : + Dependencies that constitute the structure of the pipeline. This is a two-dimensional + array that maps solid_name => input_name => DependencyDefiniion instance + context_definitions (Dict[str, PipelineContextDefinition]): + The context definitions available for consumers of this pipelines. For example, a + unit-testing environment and a production environment probably have very different + configuration and requirements. There would be one context definition per + environment. + dependency_structure (DependencyStructure): + Used mostly internally. This has the same information as the dependencies data + structure, but indexed for fast usage. ''' + def __init__( + self, solids, name=None, description=None, context_definitions=None, dependencies=None + ): + ''' + Args: + solids (List[SolidDefinition]): Solids in the pipeline + name (str): Name. This is optional, mostly for situations that require ephemeral + pipeline definitions for fast scaffolding or testing. + description (str): Description of the pipline. + context_definitions (Dict[str, PipelineContextDefinition]): See class description. + dependencies: (Dict[str, Dict[str, DependencyDefinition]]): See class description. + ''' + self.description = check.opt_str_param(description, 'description') + self.name = check.opt_str_param(name, 'name') + + if context_definitions is None: + context_definitions = _default_pipeline_context_definitions() + + self.context_definitions = check.dict_param( + context_definitions, + 'context_definitions', + key_type=str, + value_type=PipelineContextDefinition, + ) + + for solid in solids: + if not isinstance(solid, SolidDefinition) and callable(solid): + raise DagsterInvalidDefinitionError( + '''You have passed a lambda or function {func} into + a pipeline that is not a solid. You have likely forgetten to annotate this function + with an @solid decorator located in dagster.core.decorators + '''.format(func=solid.__name__) + ) + + self._solid_dict = _build_named_dict(solids) + + dependencies = check_two_dim_str_dict( + dependencies, + 'dependencies', + DependencyDefinition, + ) if dependencies else {} + + self.__validate_dependences(dependencies) + dependency_structure = DependencyStructure.from_definitions(solids, dependencies) + self.__validate_dependency_structure(name, solids, dependency_structure) + self.dependency_structure = dependency_structure + @staticmethod def create_single_solid_pipeline(pipeline, solid_name, injected_solids=None): ''' @@ -308,29 +366,33 @@ def create_single_solid_pipeline(pipeline, solid_name, injected_solids=None): See PipelineDefinition.create_sub_pipeline. Args: - pipeline: PipelineDefinition - solid_name: str - injected_solids: - Two dimensional dictionary. (optional) - solid_name (str) => index_name (str) => SolidDefinition - + pipeline (PipelineDefinition): PipelineDefinition that we will subset + solid_name (str): Name of the solid to isolate + injected_solids (Dict[str, Dict[str, SolidDefinition]]): When you create a subpipeline, you possible left with solids within that pipeline who have unmet dependencies. To fulfill these dependencies new solids must be provided. Returns: - PipelineDefinition + PipelineDefinition: The new pipeline with only the passed-in solid and the injected + solids. Example: - new_pipeline = PipelineDefinition.create_single_solid_pipeline( - existing_pipeline, - 'A', # existing_pipeline - { - 'A': { - 'A_input': new_solid_instance, + + .. code-block:: python + + new_pipeline = PipelineDefinition.create_single_solid_pipeline( + existing_pipeline, + 'A', # name of solid within existing_pipeline + { + 'A': { + # new_solid_instance is a solid that outputs something + # that is compatible with A_input + 'A_input': new_solid_instance, + }, }, - }, - ) + ) + ''' return PipelineDefinition.create_sub_pipeline( pipeline, @@ -352,17 +414,23 @@ def create_sub_pipeline(pipeline, from_solids, through_solids, injected_solids=N See PipelineDefinition.create_single_solid_pipeline Args: - pipeline: PipelineDefinintion - from_solids: list of strs - through_solids: list of strs - injected_solids: - Two dimensional dictionary. (optional) - solid_name (str) => index_name (str) => SolidDefinition + pipeline (PipelineDefinition): PipelineDefinition that we will subset + from_solids (List[str]): + List solids to "start" from. Inputs into these solids will have to satisfied + via the injected solids parameter. + through_solids (List[str]): + List of solids to execute "through". Solids depending on these solids + transitively will not be included in the returned PipelineDefinition. + injected_solids (Dict[str, Dict[str, SolidDefinition]]): + When you create a subpipeline, you possible left with solids within that pipeline + who have unmet dependencies. To fulfill these dependencies new solids must be + provided. Returns: - PipelineDefinition + PipelineDefinition: + The new pipeline definition that contains all the solids from the "from_solids" + through the "through_solids", plus the injected solids. ''' - # FIXME: fix circular reference check.inst_param(pipeline, 'pipeline', PipelineDefinition) check.list_param(from_solids, 'from_solids', of_type=str) check.list_param(through_solids, 'through_solids', of_type=str) @@ -370,6 +438,7 @@ def create_sub_pipeline(pipeline, from_solids, through_solids, injected_solids=N injected_solids, 'injected_solids', SolidDefinition ) + # FIXME: fix circular reference subgraph = ExecutionGraph.from_pipeline_subset( pipeline, from_solids, @@ -440,79 +509,96 @@ def __validate_dependency_structure(self, name, solids, dependency_structure): '{input_def.name}'.format(input_def=input_def) ) - def __init__( - self, solids, name=None, description=None, context_definitions=None, dependencies=None - ): - ''' - Args: - solids: list of SolidDefinition - name: string (optional) - description: string (optional) - context_definitions: dictionary str => PipelineContextDefinition (optional) - dependencies: 2D dictionary str => str => DependencyDefinition (optional) - ''' - self.description = check.opt_str_param(description, 'description') - self.name = check.opt_str_param(name, 'name') - - if context_definitions is None: - context_definitions = _default_pipeline_context_definitions() - - self.context_definitions = check.dict_param( - context_definitions, - 'context_definitions', - key_type=str, - value_type=PipelineContextDefinition, - ) - - for solid in solids: - if not isinstance(solid, SolidDefinition) and callable(solid): - raise DagsterInvalidDefinitionError( - '''You have passed a lambda or function {func} into - a pipeline that is not a solid. You have likely forgetten to annotate this function - with an @solid decorator located in dagster.core.decorators - '''.format(func=solid.__name__) - ) - - self._solid_dict = _build_named_dict(solids) - - dependencies = check_two_dim_str_dict( - dependencies, - 'dependencies', - DependencyDefinition, - ) if dependencies else {} - - self.__validate_dependences(dependencies) - dependency_structure = DependencyStructure.from_definitions(solids, dependencies) - self.__validate_dependency_structure(name, solids, dependency_structure) - self.dependency_structure = dependency_structure - @property def solids(self): - '''Return the solids in the pipeline.''' + '''Return the solids in the pipeline. + + Returns: + List[SolidDefinition]: List of solids. + ''' return list(self._solid_dict.values()) def has_solid(self, name): + '''Return whether or not the solid is in the piepline + + Args: + name (str): Name of solid + + Returns: + bool: True if the solid is in the pipeline + ''' check.str_param(name, 'name') return name in self._solid_dict def solid_named(self, name): - '''Return the solid named "name". Throws if it does not exist.''' + '''Return the solid named "name". Throws if it does not exist. + + Args: + name (str): Name of solid + + Returns: + SolidDefinition: SolidDefinition with correct name. + ''' check.str_param(name, 'name') return self._solid_dict[name] class ExpectationResult(object): - def __init__(self, success, solid=None, message=None, result_context=None): + ''' + When Expectations are evaluated in the callback passed to ExpectationDefinitions, + the user must return an ExpectationResult object from the callback. + + Attributes: + + success (bool): Whether the expectation passed or not. + message (str): Information about the computation. Typically only used in the failure case. + result_context (Any): Arbitrary information about the expectation result. + ''' + + def __init__(self, success, message=None, result_context=None): self.success = check.bool_param(success, 'success') - self.solid = check.opt_inst_param(solid, SolidDefinition, 'solid') self.message = check.opt_str_param(message, 'message') self.result_context = check.opt_dict_param(result_context, 'result_context') - def copy(self): - return copy.deepcopy(self) - class ExpectationDefinition(object): + ''' + Expectations represent a data quality test. It performs an arbitrary computation + to see if a given input or output satisfies the expectation. + + Attributes: + + name (str): The name of the expectation. Names should be unique per-solid. + expectation_fn (callable): + This is the implementation of expectation computation. It should be a callback + of the form. + + (context: ExecutionContext, info: ExpectationExecutionInfo, value: Any) + : ExpectationResult + + "value" conforms to the type check performed within the Dagster type system. + + e.g. If the expectation is declare on an input of type dagster_pd.DataFrame, you can + assume that value is a pandas.DataFrame + + description (str): Description of expectation. Optional. + + Example: + + .. code-block:: python + + InputDefinition('some_input', types.Int, expectations=[ + ExpectationDefinition( + name='is_positive', + expectation_fn=lambda( + _context, + _info, + value, + ): ExpectationResult(success=value > 0), + ) + ]) + ''' + def __init__(self, name, expectation_fn, description=None): self.name = check_valid_name(name) self.expectation_fn = check.callable_param(expectation_fn, 'expectation_fn') @@ -524,18 +610,18 @@ class InputDefinition(object): Inputs are values within the dagster type system that are created from previous solids. Attributes: - name: str - dagster_type: DagsterType (optional) defaults to types.Any - expectations: list of ExpectationDefinition (optional) - description: str (optional) + name (str): Name of the input. + dagster_type (DagsterType): Type of the input. Defaults to types.Any + expectations (List[ExpectationDefinition]): + List of expectations that applies to the value passed to the solid. + description (str): Description of the input. Optional. ''' - def __init__(self, name, dagster_type=None, expectations=None, description=None): + def __init__(self, name, dagster_type=types.Any, expectations=None, description=None): + '' self.name = check_valid_name(name) - self.dagster_type = check.opt_inst_param( - dagster_type, 'dagster_type', types.DagsterType, types.Any - ) + self.dagster_type = check.inst_param(dagster_type, 'dagster_type', types.DagsterType) self.expectations = check.opt_list_param( expectations, 'expectations', of_type=ExpectationDefinition @@ -550,10 +636,10 @@ class OutputDefinition(object): the default name of "result". Attributes: - name: str (optional) defaults to "result" - dagster_type: DagsterType (optional) defaults to types.Any - expectations: list of ExpectationDefinition (optional) - description: str (optional) + name (str): Name of the output. Defaults to "result". + dagster_type (DagsterType): Type of the output. Defaults to types.Any. + expectations List[ExpectationDefinition]: Expectations for this output. + description (str): Description of the output. Optional. ''' def __init__(self, name=None, dagster_type=None, expectations=None, description=None): @@ -641,16 +727,26 @@ def __new__(cls, value, output_name=DEFAULT_OUTPUT): class ConfigDefinition(object): + '''Represents the configuration of an entity in Dagster + + Broadly defined, configs determine how computations within dagster interact with + the external world. Example values that would end up in configs would be file paths, + database table names, and so forth. + + Attributes: + + config_type (DagsterType): Type of the config. + ''' + @staticmethod def config_dict(field_dict): return ConfigDefinition(types.ConfigDictionary(field_dict)) def __init__(self, config_type=types.Any): - '''Solids have config, which determine how they interact with the external world. - Example configs would be file paths, database table names, and so forth. + '''Construct a ConfigDefinition Args: - config_type: DagsterType (optional defaults to types.Any)''' + config_type (DagsterType): Type the determines shape and values of config''' self.config_type = check.inst_param(config_type, 'config_type', DagsterType) @@ -825,7 +921,7 @@ def augment(self, injected_solids): for input_handle, output_handle in self.dependency_structure.items(): handle_dict[input_handle] = output_handle - for input_handle, output_handle in create_handle_dict(solid_dict, new_deps).items(): + for input_handle, output_handle in _create_handle_dict(solid_dict, new_deps).items(): handle_dict[input_handle] = output_handle return ExecutionGraph(self.pipeline, solids, DependencyStructure(handle_dict)) diff --git a/python_modules/dagster/dagster/core/types.py b/python_modules/dagster/dagster/core/types.py index 9cace20ae3361..66118d12f9f6a 100644 --- a/python_modules/dagster/dagster/core/types.py +++ b/python_modules/dagster/dagster/core/types.py @@ -9,6 +9,9 @@ def __init__(self, name, description=None): self.name = check.str_param(name, 'name') self.description = check.opt_str_param(description, 'description') + def __repr__(self): + return 'DagsterType({name})'.format(name=self.name) + def evaluate_value(self, _value): check.not_implemented('Must implement in subclass') @@ -107,6 +110,18 @@ class __FieldValueSentinel: class Field: + ''' + A Field in a DagsterCompositeType. + + Attributes: + dagster_type (DagsterType): The type of the field. + default_value (Any): + If the Field is optional, a default value can be provided when the field value + is not specified. + is_optional (bool): Is the field optional. + description (str): Description of the field. + ''' + def __init__( self, dagster_type, @@ -128,6 +143,11 @@ def __init__( @property def default_provided(self): + '''Was a default value provided + + Returns: + bool: Yes or no + ''' return self.default_value != FIELD_NO_DEFAULT_PROVIDED diff --git a/python_modules/dagster/docs/apidocs/index.rst b/python_modules/dagster/docs/apidocs/index.rst index b5185320cca63..e3cc3b4253ece 100644 --- a/python_modules/dagster/docs/apidocs/index.rst +++ b/python_modules/dagster/docs/apidocs/index.rst @@ -1,38 +1,9 @@ API Reference -=================================== +============= -dagster.core.definitions ------------------------------------ +Top-level Dagster API +--------------------- -.. automodule:: dagster.core.definitions - :members: - -dagster.core.execution ------------------------------------ - -.. automodule:: dagster.core.execution - :members: - -dagster.core.execution_context ------------------------------------ - -.. automodule:: dagster.core.execution_context - :members: - -dagster.core.errors ------------------------------------ - -.. automodule:: dagster.core.errors - :members: - -dagster.core.types ------------------------------------ - -.. automodule:: dagster.core.types - :members: - -dagster.core.decorators ------------------------------------ - -.. automodule:: dagster.core.decorators - :members: +.. automodule:: dagster + :members: + :special-members: __init__ diff --git a/python_modules/dagster/docs/index.rst b/python_modules/dagster/docs/index.rst index b0b3d68ed0e95..31c65d6568add 100644 --- a/python_modules/dagster/docs/index.rst +++ b/python_modules/dagster/docs/index.rst @@ -1,21 +1,25 @@ -.. Dagster documentation master file, created by - sphinx-quickstart on Mon Jun 11 11:00:41 2018. - You can adapt this file completely to your liking, but it should at least - contain the root `toctree` directive. +.. image:: https://user-images.githubusercontent.com/28738937/44878798-b6e17e00-ac5c-11e8-8d25-2e47e5a53418.png + :align: center -Dagster's documentation! -=================================== -.. include:: ../../../README.rst - :start-after: docs-include +Welcome to Dagster, an opinionated programming model for data pipelines. +Documentation Structure: +.. toctree:: + :maxdepth: 3 + + motivations + principles + tutorials + topic_guides + apidocs/index Other docs ========== .. toctree:: - :maxdepth: 2 + :maxdepth: 3 :caption: Contents: developing/index diff --git a/python_modules/dagster/docs/motivations.rst b/python_modules/dagster/docs/motivations.rst new file mode 100644 index 0000000000000..ded2427383fff --- /dev/null +++ b/python_modules/dagster/docs/motivations.rst @@ -0,0 +1,3 @@ +------------------- +Motivations +------------------- diff --git a/python_modules/dagster/docs/principles.rst b/python_modules/dagster/docs/principles.rst new file mode 100644 index 0000000000000..64a250560a6aa --- /dev/null +++ b/python_modules/dagster/docs/principles.rst @@ -0,0 +1,63 @@ +Priniciples +----------- + +As noted above Dagster has a point of view and values regarding how data pipelines should be built and structured. We list them in no particular order: + +Functional +^^^^^^^^^^ +We believe that data pipelines should be organized as DAGs of functional, idempotent computations. +These computations injest input, do computation, and produce output, either with no side effects +or well-known, un-externalized side effects. Given the same inputs and configuration, the +computation should always produce the same output. These computations should also be +parameterizable, so that they can execute in different environments. + + * See https://bit.ly/2LxDgnr for an excellent overview of functional programing in batch computations. + +Self-describing +^^^^^^^^^^^^^^^ +Pipelines should be self-describing with rich metadata and types. Users should be able to approach +a pipeline, and use tooling to inspect the pipelines for their structure and capabilities. This +metadata should be co-located with actual code of the pipeline. Documentation and code is delivered +as a single artifact. + +Compute-agnostic +^^^^^^^^^^^^^^^^ +Heterogeneity in data pipelines is the norm, rather than the exception. Data pipelines are written +by many people in different personas -- data engineers, machine-learning engineers, data +scientists, analysts and so on -- who have different needs and tools, and are particular about +those tools. + +Dagster has opinions about the structure and best practices of data pipelines. It has no opinions +about what libraries and engines use to do actual compute. Dagster pipelines can be comprised of +any python computation, which could be Pandas, Spark, or it could in turn invoke SQL or any +other DSL or library deemed appropriate to the task. + +Testable +^^^^^^^^ +Testing data pipelines is notoriously difficult. Because it is so difficult it is often never +done, or done poorly. Dagster pipelines are designed to be tested. They have explicit support +for pipeline authors to manage and maintain multiple operating environments -- for example, unit +testing, integration testing, and production environments, among others. In addition dagster can +execute arbitrary subsets and nodes of the pipeline, critical testability. (This capability +happens to be useful in operational contexts as well). + +Verifiable data quality +^^^^^^^^^^^^^^^^^^^^^^^ +Testing code is important in data pipelines, but it is not sufficient. Data quality tests -- run +during every meaningful stage of computation in production -- are critical to reduce the +maintenance burden of data pipelines. Pipeline authors generally do not have control of their +input data, and make many implicit assumptions about that data. The data formats can also change +over time. In order to control this entropy, Dagster encourages users to computationally verify +assumptions (known as expectations) about the data as part of the piplien process. This way if +those assumptions are broken, the breakage can be reported quickly, easily, and with rich metadata +and diagnostic information. These expectations can also serve as contracts between teams. + + * See https://bit.ly/2mxDS1R for a primer on pipeline tests for data quality. + +Gradual, optional typing +^^^^^^^^^^^^^^^^^^^^^^^^ + +Dagster contains a type system to describe the values flowing through the pipeline and the +configuration of the pipeline. This allows nodes in a pipeline know if they are properly +arranged and configuration prior to execution before execution, and serves as value +documentation and runtime error checking. diff --git a/python_modules/dagster/docs/topic_guides.rst b/python_modules/dagster/docs/topic_guides.rst new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/python_modules/dagster/docs/tutorials.rst b/python_modules/dagster/docs/tutorials.rst new file mode 100644 index 0000000000000..e69de29bb2d1d From dc41f7136dd68df29808e74640fd67c7a43b811a Mon Sep 17 00:00:00 2001 From: Mikhail Novikov Date: Thu, 6 Sep 2018 11:33:34 +0300 Subject: [PATCH 053/103] Fixed wrong type reference --- python_modules/dagit/dagit/webapp/src/TypeWithTooltip.tsx | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python_modules/dagit/dagit/webapp/src/TypeWithTooltip.tsx b/python_modules/dagit/dagit/webapp/src/TypeWithTooltip.tsx index fc1186758df95..ad25f65650d09 100644 --- a/python_modules/dagit/dagit/webapp/src/TypeWithTooltip.tsx +++ b/python_modules/dagit/dagit/webapp/src/TypeWithTooltip.tsx @@ -2,10 +2,10 @@ import * as React from "react"; import gql from "graphql-tag"; import styled from "styled-components"; import { Code, Position, Tooltip, Text } from "@blueprintjs/core"; -import { TypeFragment } from "./types/TypeFragment"; +import { TypeWithTooltipFragment } from "./types/TypeWithTooltipFragment"; interface ITypeWithTooltipProps { - type: TypeFragment; + type: TypeWithTooltipFragment; } export default class TypeWithTooltip extends React.Component< From c0b4fb5bea81ac61b54a67d809ea933c747a1940 Mon Sep 17 00:00:00 2001 From: Mikhail Novikov Date: Thu, 6 Sep 2018 14:46:56 +0300 Subject: [PATCH 054/103] Add dagit webapp tests --- .circleci/config.yml | 22 + .../dagit/dagit/webapp/package.json | 29 +- .../dagit/dagit/webapp/src/App.test.tsx | 9 - python_modules/dagit/dagit/webapp/src/App.tsx | 6 +- .../dagit/dagit/webapp/src/AppCache.tsx | 16 + .../dagit/dagit/webapp/src/Loading.tsx | 5 +- .../dagit/webapp/src/__tests__/App.test.tsx | 230 ++++++++ .../webapp/src/__tests__/MockedProvider.tsx | 62 ++ .../__tests__/__snapshots__/App.test.tsx.snap | 75 +++ .../dagit/webapp/src/__tests__/test-links.tsx | 180 ++++++ .../dagit/dagit/webapp/src/index.tsx | 16 +- .../dagit/dagit/webapp/tsconfig.test.json | 2 +- python_modules/dagit/dagit/webapp/yarn.lock | 530 +++++++++++++++++- 13 files changed, 1137 insertions(+), 45 deletions(-) delete mode 100644 python_modules/dagit/dagit/webapp/src/App.test.tsx create mode 100644 python_modules/dagit/dagit/webapp/src/AppCache.tsx create mode 100644 python_modules/dagit/dagit/webapp/src/__tests__/App.test.tsx create mode 100644 python_modules/dagit/dagit/webapp/src/__tests__/MockedProvider.tsx create mode 100644 python_modules/dagit/dagit/webapp/src/__tests__/__snapshots__/App.test.tsx.snap create mode 100644 python_modules/dagit/dagit/webapp/src/__tests__/test-links.tsx diff --git a/.circleci/config.yml b/.circleci/config.yml index 20219952ed687..cbf91aed9495a 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -10,6 +10,7 @@ workflows: - dagit-python-3.6 - dagit-python-3.5 - dagit-python-2.7 + - dagit-webapp jobs: dagster-python-3.6: &dagster-template @@ -84,3 +85,24 @@ jobs: - image: circleci/python:2.7.15 environment: TOXENV: py27 + + dagit-webapp: + docker: + - image: circleci/node:10.6 + working_directory: ~/repo/python_modules/dagit/dagit/webapp + steps: + - checkout + + - run: + name: Install Dependencies + command: | + yarn + + - run: + name: Run Dagit Webapp Tests + command: | + yarn test + + - store_artifacts: + path: test-reports + destination: test-reports diff --git a/python_modules/dagit/dagit/webapp/package.json b/python_modules/dagit/dagit/webapp/package.json index 2b39224c09fef..32b1e28882755 100644 --- a/python_modules/dagit/dagit/webapp/package.json +++ b/python_modules/dagit/dagit/webapp/package.json @@ -27,27 +27,50 @@ "react-markdown": "^3.4.1", "react-router": "^4.3.1", "react-router-dom": "^4.3.1", - "styled-components": "^3.3.3" + "styled-components": "^3.3.3", + "ts-jest": "^23.1.4" }, "devDependencies": { "@types/jest": "^23.3.1", "@types/node": "^10.5.5", "@types/react": "^16.4.7", "@types/react-dom": "^16.0.6", + "@types/react-test-renderer": "^16.0.2", "apollo": "^1.6.0", + "babel-jest": "^23.4.2", "babel-plugin-graphql-tag": "^1.6.0", "graphql": "^0.13.2", + "jest": "^23.5.0", "prettier": "^1.14.0", "react-scripts-ts": "2.16.0", - "typescript": "^3.0.1" + "react-test-renderer": "^16.4.2", + "typescript": "^3.0.1", + "typescript-babel-jest": "^1.0.5" }, "scripts": { "start": "react-scripts-ts start", "build": "react-scripts-ts build", - "test": "react-scripts-ts test --env=jsdom", + "test": "jest", "eject": "react-scripts-ts eject", "download-schema": "apollo schema:download --endpoint $REACT_APP_GRAPHQL_URI ./src/schema.json", "generate-types": "apollo codegen:generate --queries \"./src/**/*.tsx\" --target typescript types --schema ./src/schema.json", "watch-types": "apollo codegen:generate --queries \"./src/**/*.tsx\" --target typescript types --schema ./src/schema.json --watch" + }, + "jest": { + "roots": [ + "/src" + ], + "transform": { + "^.+\\.(ts|tsx)$": "ts-jest" + }, + "moduleFileExtensions": [ + "js", + "jsx", + "json", + "ts", + "tsx" + ], + "testRegex": "/__tests__/.*\\.test\\.(ts|tsx)$", + "testURL": "http://localhost" } } diff --git a/python_modules/dagit/dagit/webapp/src/App.test.tsx b/python_modules/dagit/dagit/webapp/src/App.test.tsx deleted file mode 100644 index e0f09ab57a46b..0000000000000 --- a/python_modules/dagit/dagit/webapp/src/App.test.tsx +++ /dev/null @@ -1,9 +0,0 @@ -import * as React from 'react'; -import * as ReactDOM from 'react-dom'; -import App from './App'; - -it('renders without crashing', () => { - const div = document.createElement('div'); - ReactDOM.render(, div); - ReactDOM.unmountComponentAtNode(div); -}); diff --git a/python_modules/dagit/dagit/webapp/src/App.tsx b/python_modules/dagit/dagit/webapp/src/App.tsx index 9e5a9f4bfe93b..7b140f61758cc 100644 --- a/python_modules/dagit/dagit/webapp/src/App.tsx +++ b/python_modules/dagit/dagit/webapp/src/App.tsx @@ -8,7 +8,7 @@ import Loading from "./Loading"; import Pipelines from "./Pipelines"; import { AppQuery } from "./types/AppQuery"; -class App extends React.Component { +export default class App extends React.Component { public render() { return ( @@ -36,7 +36,7 @@ class App extends React.Component { } } -const APP_QUERY = gql` +export const APP_QUERY = gql` query AppQuery { pipelines { ...PipelinesFragment @@ -45,5 +45,3 @@ const APP_QUERY = gql` ${Pipelines.fragments.PipelinesFragment} `; - -export default App; diff --git a/python_modules/dagit/dagit/webapp/src/AppCache.tsx b/python_modules/dagit/dagit/webapp/src/AppCache.tsx new file mode 100644 index 0000000000000..4763a0ad0487a --- /dev/null +++ b/python_modules/dagit/dagit/webapp/src/AppCache.tsx @@ -0,0 +1,16 @@ +import { + InMemoryCache, + IntrospectionFragmentMatcher +} from "apollo-cache-inmemory"; +// this is a require cause otherwise it breaks +const introspectionQueryResultData = require("./schema.json"); + +const fragmentMatcher = new IntrospectionFragmentMatcher({ + introspectionQueryResultData: { + __schema: introspectionQueryResultData + } +}); + +const AppCache = new InMemoryCache({ fragmentMatcher }); + +export default AppCache; diff --git a/python_modules/dagit/dagit/webapp/src/Loading.tsx b/python_modules/dagit/dagit/webapp/src/Loading.tsx index 0d6ce4372e3dc..070d1d72ef5b8 100644 --- a/python_modules/dagit/dagit/webapp/src/Loading.tsx +++ b/python_modules/dagit/dagit/webapp/src/Loading.tsx @@ -21,7 +21,10 @@ export default class Loading extends React.Component< ); - } else { + } else if (this.props.queryResult.error) { + throw this.props.queryResult.error; + } + { return this.props.children(this.props.queryResult.data as TData); } } diff --git a/python_modules/dagit/dagit/webapp/src/__tests__/App.test.tsx b/python_modules/dagit/dagit/webapp/src/__tests__/App.test.tsx new file mode 100644 index 0000000000000..db786606a9f08 --- /dev/null +++ b/python_modules/dagit/dagit/webapp/src/__tests__/App.test.tsx @@ -0,0 +1,230 @@ +import * as React from "react"; +import * as TestRenderer from "react-test-renderer"; +import App, { APP_QUERY } from "../App"; +import AppCache from "../AppCache"; +import { MockedProvider } from "./MockedProvider"; + +const mocks = [ + { + request: { + operationName: "AppQuery", + query: APP_QUERY + }, + result: { + data: { + pipelines: [ + { + __typename: "Pipeline", + name: "pandas_hello_world", + description: null, + solids: [ + { + __typename: "Solid", + outputs: [ + { + __typename: "Output", + name: "result", + type: { + __typename: "RegularType", + name: "PandasDataFrame", + description: + "Two-dimensional size-mutable, potentially heterogeneous\ntabular data structure with labeled axes (rows and columns). See http://pandas.pydata.org/" + }, + description: null, + expectations: [] + } + ], + inputs: [], + name: "load_num_csv", + description: null, + config: { + __typename: "Config", + type: { + __typename: "CompositeType", + name: "ConfigDictionary", + description: + "Configuration dictionary.\n Typed-checked but then passed to implementations as a python dict", + fields: [ + { + __typename: "Field", + name: "path", + description: null, + isOptional: false, + defaultValue: + "", + type: { + __typename: "RegularType", + name: "Path", + description: + "\nA string the represents a path. It is very useful for some tooling\nto know that a string indeed represents a file path. That way they\ncan, for example, make the paths relative to a different location\nfor a particular execution environment.\n" + } + } + ] + } + } + }, + { + __typename: "Solid", + outputs: [ + { + __typename: "Output", + name: "result", + type: { + __typename: "RegularType", + name: "PandasDataFrame", + description: + "Two-dimensional size-mutable, potentially heterogeneous\ntabular data structure with labeled axes (rows and columns). See http://pandas.pydata.org/" + }, + description: null, + expectations: [] + } + ], + inputs: [ + { + __typename: "Input", + name: "num", + type: { + __typename: "RegularType", + name: "PandasDataFrame", + description: + "Two-dimensional size-mutable, potentially heterogeneous\ntabular data structure with labeled axes (rows and columns). See http://pandas.pydata.org/" + }, + description: null, + expectations: [], + dependsOn: { + __typename: "Output", + name: "result", + solid: { + __typename: "Solid", + name: "load_num_csv" + } + } + } + ], + name: "sum_solid", + description: null, + config: { + __typename: "Config", + type: { + __typename: "RegularType", + name: "Any", + description: + "The type that allows any value, including no value." + } + } + }, + { + __typename: "Solid", + outputs: [ + { + __typename: "Output", + name: "result", + type: { + __typename: "RegularType", + name: "PandasDataFrame", + description: + "Two-dimensional size-mutable, potentially heterogeneous\ntabular data structure with labeled axes (rows and columns). See http://pandas.pydata.org/" + }, + description: null, + expectations: [] + } + ], + inputs: [ + { + __typename: "Input", + name: "sum_df", + type: { + __typename: "RegularType", + name: "PandasDataFrame", + description: + "Two-dimensional size-mutable, potentially heterogeneous\ntabular data structure with labeled axes (rows and columns). See http://pandas.pydata.org/" + }, + description: null, + expectations: [], + dependsOn: { + __typename: "Output", + name: "result", + solid: { + __typename: "Solid", + name: "sum_solid" + } + } + } + ], + name: "sum_sq_solid", + description: null, + config: { + __typename: "Config", + type: { + __typename: "RegularType", + name: "Any", + description: + "The type that allows any value, including no value." + } + } + } + ], + contexts: [ + { + __typename: "Context", + name: "default", + description: null, + config: { + __typename: "Config", + type: { + __typename: "CompositeType", + name: "ConfigDictionary", + description: + "Configuration dictionary.\n Typed-checked but then passed to implementations as a python dict", + fields: [ + { + __typename: "Field", + name: "log_level", + description: null, + isOptional: true, + defaultValue: "ERROR", + type: { + __typename: "RegularType", + name: "String", + description: "A string." + } + } + ] + } + } + } + ] + } + ] + } + } + } +]; + +function createNodeMock(element) { + if (element.type === "div") { + return { + querySelector() { + return null; + }, + querySelectorAll() { + return []; + } + }; + } + return null; +} + +it("renders without error", async () => { + const component = TestRenderer.create( + + + , + { createNodeMock } + ); + + await new Promise(resolve => setTimeout(resolve, 1000)); + + const tree = component.toJSON(); + expect(tree).toMatchSnapshot(); +}); diff --git a/python_modules/dagit/dagit/webapp/src/__tests__/MockedProvider.tsx b/python_modules/dagit/dagit/webapp/src/__tests__/MockedProvider.tsx new file mode 100644 index 0000000000000..c2863459347c9 --- /dev/null +++ b/python_modules/dagit/dagit/webapp/src/__tests__/MockedProvider.tsx @@ -0,0 +1,62 @@ +import * as React from "react"; +import ApolloClient from "apollo-client"; +import { DefaultOptions } from "apollo-client/ApolloClient"; +import { InMemoryCache as Cache } from "apollo-cache-inmemory"; + +import { ApolloProvider } from "react-apollo"; +import { MockedResponse, MockLink } from "./test-links"; +import { ApolloCache } from "apollo-cache"; + +export interface MockedProviderProps { + mocks?: Array; + addTypename?: boolean; + defaultOptions?: DefaultOptions; + cache?: ApolloCache; +} + +export interface MockedProviderState { + client: ApolloClient; +} + +export class MockedProvider extends React.Component< + MockedProviderProps, + MockedProviderState +> { + public static defaultProps: MockedProviderProps = { + addTypename: true + }; + + constructor(props: MockedProviderProps) { + super(props); + + const { mocks, addTypename, defaultOptions, cache } = this.props; + const client = new ApolloClient({ + cache: cache || new Cache({ addTypename }), + defaultOptions, + link: new MockLink(mocks || [], addTypename) + }); + + this.state = { client }; + } + + public render() { + return ( + + {this.props.children} + + ); + } + + public componentWillUnmount() { + if (!this.state.client.queryManager) { + return; + } + const scheduler = this.state.client.queryManager.scheduler; + Object.keys(scheduler.registeredQueries).forEach(queryId => { + scheduler.stopPollingQuery(queryId); + }); + Object.keys(scheduler.intervalQueries).forEach((interval: any) => { + scheduler.fetchQueriesOnInterval(interval); + }); + } +} diff --git a/python_modules/dagit/dagit/webapp/src/__tests__/__snapshots__/App.test.tsx.snap b/python_modules/dagit/dagit/webapp/src/__tests__/__snapshots__/App.test.tsx.snap new file mode 100644 index 0000000000000..dfcdcd053db66 --- /dev/null +++ b/python_modules/dagit/dagit/webapp/src/__tests__/__snapshots__/App.test.tsx.snap @@ -0,0 +1,75 @@ +// Jest Snapshot v1, https://goo.gl/fbAQLP + +exports[`renders without error 1`] = ` +Array [ +
+
+
+ Dagit +
+
+
+
, +
+
+
+
+
+
+ +
+
+
+

+ No pipeline selected +

+
+ Select a pipeline in the sidebar on the left +
+
+
, +] +`; diff --git a/python_modules/dagit/dagit/webapp/src/__tests__/test-links.tsx b/python_modules/dagit/dagit/webapp/src/__tests__/test-links.tsx new file mode 100644 index 0000000000000..8d0362e50fb98 --- /dev/null +++ b/python_modules/dagit/dagit/webapp/src/__tests__/test-links.tsx @@ -0,0 +1,180 @@ +import { + Operation, + GraphQLRequest, + ApolloLink, + FetchResult, + Observable + // Observer, +} from "apollo-link"; + +import { print } from "graphql/language/printer"; +import { addTypenameToDocument } from "apollo-utilities"; +import { isEqual } from "lodash"; + +export interface MockedResponse { + request: GraphQLRequest; + result?: FetchResult; + error?: Error; + delay?: number; + newData?: () => FetchResult; +} + +export interface MockedSubscriptionResult { + result?: FetchResult; + error?: Error; + delay?: number; +} + +export interface MockedSubscription { + request: GraphQLRequest; +} + +export class MockLink extends ApolloLink { + public addTypename: Boolean = true; + private mockedResponsesByKey: { [key: string]: MockedResponse[] } = {}; + + constructor( + mockedResponses: ReadonlyArray, + addTypename: Boolean = true + ) { + super(); + this.addTypename = addTypename; + if (mockedResponses) + mockedResponses.forEach(mockedResponse => { + this.addMockedResponse(mockedResponse); + }); + } + + public addMockedResponse(mockedResponse: MockedResponse) { + const key = requestToKey(mockedResponse.request, this.addTypename); + let mockedResponses = this.mockedResponsesByKey[key]; + if (!mockedResponses) { + mockedResponses = []; + this.mockedResponsesByKey[key] = mockedResponses; + } + mockedResponses.push(mockedResponse); + } + + public request(operation: Operation) { + const key = requestToKey(operation, this.addTypename); + let responseIndex; + const response = (this.mockedResponsesByKey[key] || []).find( + (res, index) => { + const requestVariables = operation.variables || {}; + const mockedResponseVariables = res.request.variables || {}; + if (!isEqual(requestVariables, mockedResponseVariables)) { + return false; + } + responseIndex = index; + return true; + } + ); + + if (!response || typeof responseIndex === "undefined") { + throw new Error( + `No more mocked responses for the query: ${print( + operation.query + )}, variables: ${JSON.stringify(operation.variables)}` + ); + } + + this.mockedResponsesByKey[key].splice(responseIndex, 1); + + const { result, error, delay, newData } = response; + + if (newData) { + response.result = newData(); + this.mockedResponsesByKey[key].push(response); + } + + if (!result && !error) { + throw new Error( + `Mocked response should contain either result or error: ${key}` + ); + } + + return new Observable(observer => { + let timer = setTimeout(() => { + if (error) { + observer.error(error); + } else { + if (result) observer.next(result); + observer.complete(); + } + }, delay ? delay : 0); + + return () => { + clearTimeout(timer); + }; + }); + } +} + +export class MockSubscriptionLink extends ApolloLink { + // private observer: Observer; + public unsubscribers: any[] = []; + public setups: any[] = []; + + private observer: any; + + constructor() { + super(); + } + + public request(_req: any) { + return new Observable(observer => { + this.setups.forEach(x => x()); + this.observer = observer; + return () => { + this.unsubscribers.forEach(x => x()); + }; + }); + } + + public simulateResult(result: MockedSubscriptionResult) { + setTimeout(() => { + const { observer } = this; + if (!observer) throw new Error("subscription torn down"); + if (result.result && observer.next) observer.next(result.result); + if (result.error && observer.error) observer.error(result.error); + }, result.delay || 0); + } + + public onSetup(listener: any): void { + this.setups = this.setups.concat([listener]); + } + + public onUnsubscribe(listener: any): void { + this.unsubscribers = this.unsubscribers.concat([listener]); + } +} + +function requestToKey(request: GraphQLRequest, addTypename: Boolean): string { + const queryString = + request.query && + print(addTypename ? addTypenameToDocument(request.query) : request.query); + + const requestKey = { query: queryString }; + + return JSON.stringify(requestKey); +} + +// Pass in multiple mocked responses, so that you can test flows that end up +// making multiple queries to the server +// NOTE: The last arg can optionally be an `addTypename` arg +export function mockSingleLink(...mockedResponses: Array): ApolloLink { + // to pull off the potential typename. If this isn't a boolean, we'll just set it true later + let maybeTypename = mockedResponses[mockedResponses.length - 1]; + let mocks = mockedResponses.slice(0, mockedResponses.length - 1); + + if (typeof maybeTypename !== "boolean") { + mocks = mockedResponses; + maybeTypename = true; + } + + return new MockLink(mocks, maybeTypename); +} + +export function mockObservableLink(): MockSubscriptionLink { + return new MockSubscriptionLink(); +} diff --git a/python_modules/dagit/dagit/webapp/src/index.tsx b/python_modules/dagit/dagit/webapp/src/index.tsx index f77ca0d25b330..7a45381a127f7 100644 --- a/python_modules/dagit/dagit/webapp/src/index.tsx +++ b/python_modules/dagit/dagit/webapp/src/index.tsx @@ -2,26 +2,14 @@ import * as React from "react"; import * as ReactDOM from "react-dom"; import { injectGlobal } from "styled-components"; import ApolloClient from "apollo-boost"; -import { - InMemoryCache, - IntrospectionFragmentMatcher -} from "apollo-cache-inmemory"; import { ApolloProvider } from "react-apollo"; import App from "./App"; +import AppCache from "./AppCache"; import "@blueprintjs/core/lib/css/blueprint.css"; import "@blueprintjs/icons/lib/css/blueprint-icons.css"; -import introspectionQueryResultData from "./schema.json"; - -const fragmentMatcher = new IntrospectionFragmentMatcher({ - introspectionQueryResultData: { - __schema: introspectionQueryResultData - } -}); - -const cache = new InMemoryCache({ fragmentMatcher }); const client = new ApolloClient({ - cache, + cache: AppCache, uri: process.env.REACT_APP_GRAPHQL_URI || "/graphql" }); diff --git a/python_modules/dagit/dagit/webapp/tsconfig.test.json b/python_modules/dagit/dagit/webapp/tsconfig.test.json index 65ffdd493929c..2c7b284162f4c 100644 --- a/python_modules/dagit/dagit/webapp/tsconfig.test.json +++ b/python_modules/dagit/dagit/webapp/tsconfig.test.json @@ -3,4 +3,4 @@ "compilerOptions": { "module": "commonjs" } -} \ No newline at end of file +} diff --git a/python_modules/dagit/dagit/webapp/yarn.lock b/python_modules/dagit/dagit/webapp/yarn.lock index 79e89c671bbfe..524f931bc68ab 100644 --- a/python_modules/dagit/dagit/webapp/yarn.lock +++ b/python_modules/dagit/dagit/webapp/yarn.lock @@ -259,6 +259,12 @@ "@types/history" "*" "@types/react" "*" +"@types/react-test-renderer@^16.0.2": + version "16.0.2" + resolved "https://registry.yarnpkg.com/@types/react-test-renderer/-/react-test-renderer-16.0.2.tgz#2715eae40df08f80b3620c498598c8615693b6c6" + dependencies: + "@types/react" "*" + "@types/react@*", "@types/react@^16.3.14", "@types/react@^16.4.7": version "16.4.7" resolved "https://registry.yarnpkg.com/@types/react/-/react-16.4.7.tgz#f33f6d759a7e1833befa15224d68942d178a5a3f" @@ -673,6 +679,16 @@ apollo@^1.6.0: react-dom "^16.4.0" rimraf "^2.6.2" +app-root-path@2.0.1: + version "2.0.1" + resolved "https://registry.yarnpkg.com/app-root-path/-/app-root-path-2.0.1.tgz#cd62dcf8e4fd5a417efc664d2e5b10653c651b46" + +append-transform@^0.4.0: + version "0.4.0" + resolved "https://registry.yarnpkg.com/append-transform/-/append-transform-0.4.0.tgz#d76ebf8ca94d276e247a36bad44a4b74ab611991" + dependencies: + default-require-extensions "^1.0.0" + append-transform@^1.0.0: version "1.0.0" resolved "https://registry.yarnpkg.com/append-transform/-/append-transform-1.0.0.tgz#046a52ae582a228bd72f58acfbe2967c678759ab" @@ -1015,6 +1031,14 @@ babel-helpers@^6.24.1: babel-runtime "^6.22.0" babel-template "^6.24.1" +babel-jest@20.0.3: + version "20.0.3" + resolved "https://registry.yarnpkg.com/babel-jest/-/babel-jest-20.0.3.tgz#e4a03b13dc10389e140fc645d09ffc4ced301671" + dependencies: + babel-core "^6.0.0" + babel-plugin-istanbul "^4.0.0" + babel-preset-jest "^20.0.3" + babel-jest@^22.1.0, babel-jest@^22.4.4: version "22.4.4" resolved "https://registry.yarnpkg.com/babel-jest/-/babel-jest-22.4.4.tgz#977259240420e227444ebe49e226a61e49ea659d" @@ -1022,6 +1046,13 @@ babel-jest@^22.1.0, babel-jest@^22.4.4: babel-plugin-istanbul "^4.1.5" babel-preset-jest "^22.4.4" +babel-jest@^23.4.2: + version "23.4.2" + resolved "https://registry.yarnpkg.com/babel-jest/-/babel-jest-23.4.2.tgz#f276de67798a5d68f2d6e87ff518c2f6e1609877" + dependencies: + babel-plugin-istanbul "^4.1.6" + babel-preset-jest "^23.2.0" + babel-literal-to-ast@^0.1.2: version "0.1.2" resolved "https://registry.yarnpkg.com/babel-literal-to-ast/-/babel-literal-to-ast-0.1.2.tgz#f50aa963711935adf19da0d5a4defc827ad5f403" @@ -1067,7 +1098,7 @@ babel-plugin-graphql-tag@^1.6.0: babylon "^6.18.0" debug "^2.6.8" -babel-plugin-istanbul@^4.1.4, babel-plugin-istanbul@^4.1.5: +babel-plugin-istanbul@^4.0.0, babel-plugin-istanbul@^4.1.4, babel-plugin-istanbul@^4.1.5, babel-plugin-istanbul@^4.1.6: version "4.1.6" resolved "https://registry.yarnpkg.com/babel-plugin-istanbul/-/babel-plugin-istanbul-4.1.6.tgz#36c59b2192efce81c5b378321b74175add1c9a45" dependencies: @@ -1076,10 +1107,18 @@ babel-plugin-istanbul@^4.1.4, babel-plugin-istanbul@^4.1.5: istanbul-lib-instrument "^1.10.1" test-exclude "^4.2.1" +babel-plugin-jest-hoist@^20.0.3: + version "20.0.3" + resolved "https://registry.yarnpkg.com/babel-plugin-jest-hoist/-/babel-plugin-jest-hoist-20.0.3.tgz#afedc853bd3f8dc3548ea671fbe69d03cc2c1767" + babel-plugin-jest-hoist@^22.4.4: version "22.4.4" resolved "https://registry.yarnpkg.com/babel-plugin-jest-hoist/-/babel-plugin-jest-hoist-22.4.4.tgz#b9851906eab34c7bf6f8c895a2b08bea1a844c0b" +babel-plugin-jest-hoist@^23.2.0: + version "23.2.0" + resolved "https://registry.yarnpkg.com/babel-plugin-jest-hoist/-/babel-plugin-jest-hoist-23.2.0.tgz#e61fae05a1ca8801aadee57a6d66b8cefaf44167" + babel-plugin-syntax-async-functions@^6.8.0: version "6.13.0" resolved "https://registry.yarnpkg.com/babel-plugin-syntax-async-functions/-/babel-plugin-syntax-async-functions-6.13.0.tgz#cad9cad1191b5ad634bf30ae0872391e0647be95" @@ -1421,6 +1460,12 @@ babel-preset-flow@^6.23.0: dependencies: babel-plugin-transform-flow-strip-types "^6.22.0" +babel-preset-jest@^20.0.3: + version "20.0.3" + resolved "https://registry.yarnpkg.com/babel-preset-jest/-/babel-preset-jest-20.0.3.tgz#cbacaadecb5d689ca1e1de1360ebfc66862c178a" + dependencies: + babel-plugin-jest-hoist "^20.0.3" + babel-preset-jest@^22.0.1, babel-preset-jest@^22.4.4: version "22.4.4" resolved "https://registry.yarnpkg.com/babel-preset-jest/-/babel-preset-jest-22.4.4.tgz#ec9fbd8bcd7dfd24b8b5320e0e688013235b7c39" @@ -1428,6 +1473,13 @@ babel-preset-jest@^22.0.1, babel-preset-jest@^22.4.4: babel-plugin-jest-hoist "^22.4.4" babel-plugin-syntax-object-rest-spread "^6.13.0" +babel-preset-jest@^23.2.0: + version "23.2.0" + resolved "https://registry.yarnpkg.com/babel-preset-jest/-/babel-preset-jest-23.2.0.tgz#8ec7a03a138f001a1a8fb1e8113652bf1a55da46" + dependencies: + babel-plugin-jest-hoist "^23.2.0" + babel-plugin-syntax-object-rest-spread "^6.13.0" + babel-preset-react-app@^3.1.1: version "3.1.2" resolved "https://registry.yarnpkg.com/babel-preset-react-app/-/babel-preset-react-app-3.1.2.tgz#49ba3681b917c4e5c73a5249d3ef4c48fae064e2" @@ -1486,7 +1538,7 @@ babel-template@^6.16.0, babel-template@^6.24.1, babel-template@^6.26.0: babylon "^6.18.0" lodash "^4.17.4" -babel-traverse@^6.0.20, babel-traverse@^6.18.0, babel-traverse@^6.24.1, babel-traverse@^6.26.0: +babel-traverse@^6.0.0, babel-traverse@^6.0.20, babel-traverse@^6.18.0, babel-traverse@^6.24.1, babel-traverse@^6.26.0: version "6.26.0" resolved "https://registry.yarnpkg.com/babel-traverse/-/babel-traverse-6.26.0.tgz#46a9cbd7edcc62c8e5c064e2d2d8d0f4035766ee" dependencies: @@ -1500,7 +1552,7 @@ babel-traverse@^6.0.20, babel-traverse@^6.18.0, babel-traverse@^6.24.1, babel-tr invariant "^2.2.2" lodash "^4.17.4" -babel-types@^6.0.19, babel-types@^6.18.0, babel-types@^6.19.0, babel-types@^6.24.1, babel-types@^6.26.0: +babel-types@^6.0.0, babel-types@^6.0.19, babel-types@^6.18.0, babel-types@^6.19.0, babel-types@^6.24.1, babel-types@^6.26.0: version "6.26.0" resolved "https://registry.yarnpkg.com/babel-types/-/babel-types-6.26.0.tgz#a3b073f94ab49eb6fa55cd65227a334380632497" dependencies: @@ -1651,7 +1703,7 @@ browser-process-hrtime@^0.1.2: version "0.1.2" resolved "https://registry.yarnpkg.com/browser-process-hrtime/-/browser-process-hrtime-0.1.2.tgz#425d68a58d3447f02a04aa894187fce8af8b7b8e" -browser-resolve@^1.11.2: +browser-resolve@^1.11.2, browser-resolve@^1.11.3: version "1.11.3" resolved "https://registry.yarnpkg.com/browser-resolve/-/browser-resolve-1.11.3.tgz#9b7cbb3d0f510e4cb86bdbd796124d28b5890af6" dependencies: @@ -2126,6 +2178,10 @@ clone@^1.0.2: version "1.0.4" resolved "https://registry.yarnpkg.com/clone/-/clone-1.0.4.tgz#da309cc263df15994c688ca902179ca3c7cd7c7e" +closest-file-data@^0.1.4: + version "0.1.4" + resolved "https://registry.yarnpkg.com/closest-file-data/-/closest-file-data-0.1.4.tgz#975f87c132f299d24a0375b9f63ca3fb88f72b3a" + co@^4.6.0: version "4.6.0" resolved "https://registry.yarnpkg.com/co/-/co-4.6.0.tgz#6ea6bdf3d853ae54ccb8e47bfa0bf3f9031fb184" @@ -2729,6 +2785,12 @@ deepmerge@^2.0.1: version "2.1.1" resolved "https://registry.yarnpkg.com/deepmerge/-/deepmerge-2.1.1.tgz#e862b4e45ea0555072bf51e7fd0d9845170ae768" +default-require-extensions@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/default-require-extensions/-/default-require-extensions-1.0.0.tgz#f37ea15d3e13ffd9b437d33e1a75b5fb97874cb8" + dependencies: + strip-bom "^2.0.0" + default-require-extensions@^2.0.0: version "2.0.0" resolved "https://registry.yarnpkg.com/default-require-extensions/-/default-require-extensions-2.0.0.tgz#f5f8fbb18a7d6d50b21f641f649ebb522cfe24f7" @@ -3293,6 +3355,17 @@ expect@^22.4.0: jest-message-util "^22.4.3" jest-regex-util "^22.4.3" +expect@^23.5.0: + version "23.5.0" + resolved "https://registry.yarnpkg.com/expect/-/expect-23.5.0.tgz#18999a0eef8f8acf99023fde766d9c323c2562ed" + dependencies: + ansi-styles "^3.2.0" + jest-diff "^23.5.0" + jest-get-type "^22.1.0" + jest-matcher-utils "^23.5.0" + jest-message-util "^23.4.0" + jest-regex-util "^23.3.0" + express@^4.13.3: version "4.16.3" resolved "https://registry.yarnpkg.com/express/-/express-4.16.3.tgz#6af8a502350db3246ecc4becf6b5a34d22f7ed53" @@ -3637,6 +3710,14 @@ fs-extra@4.0.3: jsonfile "^4.0.0" universalify "^0.1.0" +fs-extra@6.0.1, fs-extra@^6.0.1: + version "6.0.1" + resolved "https://registry.yarnpkg.com/fs-extra/-/fs-extra-6.0.1.tgz#8abc128f7946e310135ddc93b98bddb410e7a34b" + dependencies: + graceful-fs "^4.1.2" + jsonfile "^4.0.0" + universalify "^0.1.0" + fs-extra@^0.30.0: version "0.30.0" resolved "https://registry.yarnpkg.com/fs-extra/-/fs-extra-0.30.0.tgz#f233ffcc08d4da7d432daa449776989db1df93f0" @@ -3647,14 +3728,6 @@ fs-extra@^0.30.0: path-is-absolute "^1.0.0" rimraf "^2.2.8" -fs-extra@^6.0.1: - version "6.0.1" - resolved "https://registry.yarnpkg.com/fs-extra/-/fs-extra-6.0.1.tgz#8abc128f7946e310135ddc93b98bddb410e7a34b" - dependencies: - graceful-fs "^4.1.2" - jsonfile "^4.0.0" - universalify "^0.1.0" - fs-minipass@^1.2.5: version "1.2.5" resolved "https://registry.yarnpkg.com/fs-minipass/-/fs-minipass-1.2.5.tgz#06c277218454ec288df77ada54a03b8702aacb9d" @@ -4742,16 +4815,42 @@ istanbul-api@^1.1.14: mkdirp "^0.5.1" once "^1.4.0" +istanbul-api@^1.3.1: + version "1.3.7" + resolved "https://registry.yarnpkg.com/istanbul-api/-/istanbul-api-1.3.7.tgz#a86c770d2b03e11e3f778cd7aedd82d2722092aa" + dependencies: + async "^2.1.4" + fileset "^2.0.2" + istanbul-lib-coverage "^1.2.1" + istanbul-lib-hook "^1.2.2" + istanbul-lib-instrument "^1.10.2" + istanbul-lib-report "^1.1.5" + istanbul-lib-source-maps "^1.2.6" + istanbul-reports "^1.5.1" + js-yaml "^3.7.0" + mkdirp "^0.5.1" + once "^1.4.0" + istanbul-lib-coverage@^1.1.1, istanbul-lib-coverage@^1.1.2, istanbul-lib-coverage@^1.2.0: version "1.2.0" resolved "https://registry.yarnpkg.com/istanbul-lib-coverage/-/istanbul-lib-coverage-1.2.0.tgz#f7d8f2e42b97e37fe796114cb0f9d68b5e3a4341" +istanbul-lib-coverage@^1.2.1: + version "1.2.1" + resolved "https://registry.yarnpkg.com/istanbul-lib-coverage/-/istanbul-lib-coverage-1.2.1.tgz#ccf7edcd0a0bb9b8f729feeb0930470f9af664f0" + istanbul-lib-hook@^1.2.0: version "1.2.1" resolved "https://registry.yarnpkg.com/istanbul-lib-hook/-/istanbul-lib-hook-1.2.1.tgz#f614ec45287b2a8fc4f07f5660af787575601805" dependencies: append-transform "^1.0.0" +istanbul-lib-hook@^1.2.2: + version "1.2.2" + resolved "https://registry.yarnpkg.com/istanbul-lib-hook/-/istanbul-lib-hook-1.2.2.tgz#bc6bf07f12a641fbf1c85391d0daa8f0aea6bf86" + dependencies: + append-transform "^0.4.0" + istanbul-lib-instrument@^1.10.1, istanbul-lib-instrument@^1.8.0: version "1.10.1" resolved "https://registry.yarnpkg.com/istanbul-lib-instrument/-/istanbul-lib-instrument-1.10.1.tgz#724b4b6caceba8692d3f1f9d0727e279c401af7b" @@ -4764,6 +4863,18 @@ istanbul-lib-instrument@^1.10.1, istanbul-lib-instrument@^1.8.0: istanbul-lib-coverage "^1.2.0" semver "^5.3.0" +istanbul-lib-instrument@^1.10.2: + version "1.10.2" + resolved "https://registry.yarnpkg.com/istanbul-lib-instrument/-/istanbul-lib-instrument-1.10.2.tgz#1f55ed10ac3c47f2bdddd5307935126754d0a9ca" + dependencies: + babel-generator "^6.18.0" + babel-template "^6.16.0" + babel-traverse "^6.18.0" + babel-types "^6.18.0" + babylon "^6.18.0" + istanbul-lib-coverage "^1.2.1" + semver "^5.3.0" + istanbul-lib-report@^1.1.4: version "1.1.4" resolved "https://registry.yarnpkg.com/istanbul-lib-report/-/istanbul-lib-report-1.1.4.tgz#e886cdf505c4ebbd8e099e4396a90d0a28e2acb5" @@ -4773,6 +4884,15 @@ istanbul-lib-report@^1.1.4: path-parse "^1.0.5" supports-color "^3.1.2" +istanbul-lib-report@^1.1.5: + version "1.1.5" + resolved "https://registry.yarnpkg.com/istanbul-lib-report/-/istanbul-lib-report-1.1.5.tgz#f2a657fc6282f96170aaf281eb30a458f7f4170c" + dependencies: + istanbul-lib-coverage "^1.2.1" + mkdirp "^0.5.1" + path-parse "^1.0.5" + supports-color "^3.1.2" + istanbul-lib-source-maps@^1.2.1: version "1.2.3" resolved "https://registry.yarnpkg.com/istanbul-lib-source-maps/-/istanbul-lib-source-maps-1.2.3.tgz#20fb54b14e14b3fb6edb6aca3571fd2143db44e6" @@ -4793,12 +4913,28 @@ istanbul-lib-source-maps@^1.2.4: rimraf "^2.6.1" source-map "^0.5.3" +istanbul-lib-source-maps@^1.2.6: + version "1.2.6" + resolved "https://registry.yarnpkg.com/istanbul-lib-source-maps/-/istanbul-lib-source-maps-1.2.6.tgz#37b9ff661580f8fca11232752ee42e08c6675d8f" + dependencies: + debug "^3.1.0" + istanbul-lib-coverage "^1.2.1" + mkdirp "^0.5.1" + rimraf "^2.6.1" + source-map "^0.5.3" + istanbul-reports@^1.3.0: version "1.3.0" resolved "https://registry.yarnpkg.com/istanbul-reports/-/istanbul-reports-1.3.0.tgz#2f322e81e1d9520767597dca3c20a0cce89a3554" dependencies: handlebars "^4.0.3" +istanbul-reports@^1.5.1: + version "1.5.1" + resolved "https://registry.yarnpkg.com/istanbul-reports/-/istanbul-reports-1.5.1.tgz#97e4dbf3b515e8c484caea15d6524eebd3ff4e1a" + dependencies: + handlebars "^4.0.3" + isurl@^1.0.0-alpha5: version "1.0.0" resolved "https://registry.yarnpkg.com/isurl/-/isurl-1.0.0.tgz#b27f4f49f3cdaa3ea44a0a5b7f3462e6edc39d67" @@ -4820,6 +4956,12 @@ jest-changed-files@^22.2.0: dependencies: throat "^4.0.0" +jest-changed-files@^23.4.2: + version "23.4.2" + resolved "https://registry.yarnpkg.com/jest-changed-files/-/jest-changed-files-23.4.2.tgz#1eed688370cd5eebafe4ae93d34bb3b64968fe83" + dependencies: + throat "^4.0.0" + jest-cli@^22.4.2: version "22.4.4" resolved "https://registry.yarnpkg.com/jest-cli/-/jest-cli-22.4.4.tgz#68cd2a2aae983adb1e6638248ca21082fd6d9e90" @@ -4859,6 +5001,47 @@ jest-cli@^22.4.2: which "^1.2.12" yargs "^10.0.3" +jest-cli@^23.5.0: + version "23.5.0" + resolved "https://registry.yarnpkg.com/jest-cli/-/jest-cli-23.5.0.tgz#d316b8e34a38a610a1efc4f0403d8ef8a55e4492" + dependencies: + ansi-escapes "^3.0.0" + chalk "^2.0.1" + exit "^0.1.2" + glob "^7.1.2" + graceful-fs "^4.1.11" + import-local "^1.0.0" + is-ci "^1.0.10" + istanbul-api "^1.3.1" + istanbul-lib-coverage "^1.2.0" + istanbul-lib-instrument "^1.10.1" + istanbul-lib-source-maps "^1.2.4" + jest-changed-files "^23.4.2" + jest-config "^23.5.0" + jest-environment-jsdom "^23.4.0" + jest-get-type "^22.1.0" + jest-haste-map "^23.5.0" + jest-message-util "^23.4.0" + jest-regex-util "^23.3.0" + jest-resolve-dependencies "^23.5.0" + jest-runner "^23.5.0" + jest-runtime "^23.5.0" + jest-snapshot "^23.5.0" + jest-util "^23.4.0" + jest-validate "^23.5.0" + jest-watcher "^23.4.0" + jest-worker "^23.2.0" + micromatch "^2.3.11" + node-notifier "^5.2.1" + prompts "^0.1.9" + realpath-native "^1.0.0" + rimraf "^2.5.4" + slash "^1.0.0" + string-length "^2.0.0" + strip-ansi "^4.0.0" + which "^1.2.12" + yargs "^11.0.0" + jest-config@^22.0.1, jest-config@^22.4.4: version "22.4.4" resolved "https://registry.yarnpkg.com/jest-config/-/jest-config-22.4.4.tgz#72a521188720597169cd8b4ff86934ef5752d86a" @@ -4875,6 +5058,25 @@ jest-config@^22.0.1, jest-config@^22.4.4: jest-validate "^22.4.4" pretty-format "^22.4.0" +jest-config@^23.5.0: + version "23.5.0" + resolved "https://registry.yarnpkg.com/jest-config/-/jest-config-23.5.0.tgz#3770fba03f7507ee15f3b8867c742e48f31a9773" + dependencies: + babel-core "^6.0.0" + babel-jest "^23.4.2" + chalk "^2.0.1" + glob "^7.1.1" + jest-environment-jsdom "^23.4.0" + jest-environment-node "^23.4.0" + jest-get-type "^22.1.0" + jest-jasmine2 "^23.5.0" + jest-regex-util "^23.3.0" + jest-resolve "^23.5.0" + jest-util "^23.4.0" + jest-validate "^23.5.0" + micromatch "^2.3.11" + pretty-format "^23.5.0" + jest-diff@^22.4.0, jest-diff@^22.4.3: version "22.4.3" resolved "https://registry.yarnpkg.com/jest-diff/-/jest-diff-22.4.3.tgz#e18cc3feff0aeef159d02310f2686d4065378030" @@ -4884,12 +5086,34 @@ jest-diff@^22.4.0, jest-diff@^22.4.3: jest-get-type "^22.4.3" pretty-format "^22.4.3" +jest-diff@^23.5.0: + version "23.5.0" + resolved "https://registry.yarnpkg.com/jest-diff/-/jest-diff-23.5.0.tgz#250651a433dd0050290a07642946cc9baaf06fba" + dependencies: + chalk "^2.0.1" + diff "^3.2.0" + jest-get-type "^22.1.0" + pretty-format "^23.5.0" + jest-docblock@^22.4.0, jest-docblock@^22.4.3: version "22.4.3" resolved "https://registry.yarnpkg.com/jest-docblock/-/jest-docblock-22.4.3.tgz#50886f132b42b280c903c592373bb6e93bb68b19" dependencies: detect-newline "^2.1.0" +jest-docblock@^23.2.0: + version "23.2.0" + resolved "https://registry.yarnpkg.com/jest-docblock/-/jest-docblock-23.2.0.tgz#f085e1f18548d99fdd69b20207e6fd55d91383a7" + dependencies: + detect-newline "^2.1.0" + +jest-each@^23.5.0: + version "23.5.0" + resolved "https://registry.yarnpkg.com/jest-each/-/jest-each-23.5.0.tgz#77f7e2afe6132a80954b920006e78239862b10ba" + dependencies: + chalk "^2.0.1" + pretty-format "^23.5.0" + jest-environment-jsdom@^22.4.1: version "22.4.3" resolved "https://registry.yarnpkg.com/jest-environment-jsdom/-/jest-environment-jsdom-22.4.3.tgz#d67daa4155e33516aecdd35afd82d4abf0fa8a1e" @@ -4898,6 +5122,14 @@ jest-environment-jsdom@^22.4.1: jest-util "^22.4.3" jsdom "^11.5.1" +jest-environment-jsdom@^23.4.0: + version "23.4.0" + resolved "https://registry.yarnpkg.com/jest-environment-jsdom/-/jest-environment-jsdom-23.4.0.tgz#056a7952b3fea513ac62a140a2c368c79d9e6023" + dependencies: + jest-mock "^23.2.0" + jest-util "^23.4.0" + jsdom "^11.5.1" + jest-environment-node@^22.4.1: version "22.4.3" resolved "https://registry.yarnpkg.com/jest-environment-node/-/jest-environment-node-22.4.3.tgz#54c4eaa374c83dd52a9da8759be14ebe1d0b9129" @@ -4905,6 +5137,13 @@ jest-environment-node@^22.4.1: jest-mock "^22.4.3" jest-util "^22.4.3" +jest-environment-node@^23.4.0: + version "23.4.0" + resolved "https://registry.yarnpkg.com/jest-environment-node/-/jest-environment-node-23.4.0.tgz#57e80ed0841dea303167cce8cd79521debafde10" + dependencies: + jest-mock "^23.2.0" + jest-util "^23.4.0" + jest-get-type@^22.1.0, jest-get-type@^22.4.3: version "22.4.3" resolved "https://registry.yarnpkg.com/jest-get-type/-/jest-get-type-22.4.3.tgz#e3a8504d8479342dd4420236b322869f18900ce4" @@ -4921,6 +5160,19 @@ jest-haste-map@^22.4.2: micromatch "^2.3.11" sane "^2.0.0" +jest-haste-map@^23.5.0: + version "23.5.0" + resolved "https://registry.yarnpkg.com/jest-haste-map/-/jest-haste-map-23.5.0.tgz#d4ca618188bd38caa6cb20349ce6610e194a8065" + dependencies: + fb-watchman "^2.0.0" + graceful-fs "^4.1.11" + invariant "^2.2.4" + jest-docblock "^23.2.0" + jest-serializer "^23.0.1" + jest-worker "^23.2.0" + micromatch "^2.3.11" + sane "^2.0.0" + jest-jasmine2@^22.4.4: version "22.4.4" resolved "https://registry.yarnpkg.com/jest-jasmine2/-/jest-jasmine2-22.4.4.tgz#c55f92c961a141f693f869f5f081a79a10d24e23" @@ -4937,12 +5189,35 @@ jest-jasmine2@^22.4.4: jest-util "^22.4.1" source-map-support "^0.5.0" +jest-jasmine2@^23.5.0: + version "23.5.0" + resolved "https://registry.yarnpkg.com/jest-jasmine2/-/jest-jasmine2-23.5.0.tgz#05fe7f1788e650eeb5a03929e6461ea2e9f3db53" + dependencies: + babel-traverse "^6.0.0" + chalk "^2.0.1" + co "^4.6.0" + expect "^23.5.0" + is-generator-fn "^1.0.0" + jest-diff "^23.5.0" + jest-each "^23.5.0" + jest-matcher-utils "^23.5.0" + jest-message-util "^23.4.0" + jest-snapshot "^23.5.0" + jest-util "^23.4.0" + pretty-format "^23.5.0" + jest-leak-detector@^22.4.0: version "22.4.3" resolved "https://registry.yarnpkg.com/jest-leak-detector/-/jest-leak-detector-22.4.3.tgz#2b7b263103afae8c52b6b91241a2de40117e5b35" dependencies: pretty-format "^22.4.3" +jest-leak-detector@^23.5.0: + version "23.5.0" + resolved "https://registry.yarnpkg.com/jest-leak-detector/-/jest-leak-detector-23.5.0.tgz#14ac2a785bd625160a2ea968fd5d98b7dcea3e64" + dependencies: + pretty-format "^23.5.0" + jest-matcher-utils@^22.4.0, jest-matcher-utils@^22.4.3: version "22.4.3" resolved "https://registry.yarnpkg.com/jest-matcher-utils/-/jest-matcher-utils-22.4.3.tgz#4632fe428ebc73ebc194d3c7b65d37b161f710ff" @@ -4951,6 +5226,14 @@ jest-matcher-utils@^22.4.0, jest-matcher-utils@^22.4.3: jest-get-type "^22.4.3" pretty-format "^22.4.3" +jest-matcher-utils@^23.5.0: + version "23.5.0" + resolved "https://registry.yarnpkg.com/jest-matcher-utils/-/jest-matcher-utils-23.5.0.tgz#0e2ea67744cab78c9ab15011c4d888bdd3e49e2a" + dependencies: + chalk "^2.0.1" + jest-get-type "^22.1.0" + pretty-format "^23.5.0" + jest-message-util@^22.4.0, jest-message-util@^22.4.3: version "22.4.3" resolved "https://registry.yarnpkg.com/jest-message-util/-/jest-message-util-22.4.3.tgz#cf3d38aafe4befddbfc455e57d65d5239e399eb7" @@ -4961,20 +5244,45 @@ jest-message-util@^22.4.0, jest-message-util@^22.4.3: slash "^1.0.0" stack-utils "^1.0.1" +jest-message-util@^23.4.0: + version "23.4.0" + resolved "https://registry.yarnpkg.com/jest-message-util/-/jest-message-util-23.4.0.tgz#17610c50942349508d01a3d1e0bda2c079086a9f" + dependencies: + "@babel/code-frame" "^7.0.0-beta.35" + chalk "^2.0.1" + micromatch "^2.3.11" + slash "^1.0.0" + stack-utils "^1.0.1" + jest-mock@^22.4.3: version "22.4.3" resolved "https://registry.yarnpkg.com/jest-mock/-/jest-mock-22.4.3.tgz#f63ba2f07a1511772cdc7979733397df770aabc7" +jest-mock@^23.2.0: + version "23.2.0" + resolved "https://registry.yarnpkg.com/jest-mock/-/jest-mock-23.2.0.tgz#ad1c60f29e8719d47c26e1138098b6d18b261134" + jest-regex-util@^22.1.0, jest-regex-util@^22.4.3: version "22.4.3" resolved "https://registry.yarnpkg.com/jest-regex-util/-/jest-regex-util-22.4.3.tgz#a826eb191cdf22502198c5401a1fc04de9cef5af" +jest-regex-util@^23.3.0: + version "23.3.0" + resolved "https://registry.yarnpkg.com/jest-regex-util/-/jest-regex-util-23.3.0.tgz#5f86729547c2785c4002ceaa8f849fe8ca471bc5" + jest-resolve-dependencies@^22.1.0: version "22.4.3" resolved "https://registry.yarnpkg.com/jest-resolve-dependencies/-/jest-resolve-dependencies-22.4.3.tgz#e2256a5a846732dc3969cb72f3c9ad7725a8195e" dependencies: jest-regex-util "^22.4.3" +jest-resolve-dependencies@^23.5.0: + version "23.5.0" + resolved "https://registry.yarnpkg.com/jest-resolve-dependencies/-/jest-resolve-dependencies-23.5.0.tgz#10c4d135beb9d2256de1fedc7094916c3ad74af7" + dependencies: + jest-regex-util "^23.3.0" + jest-snapshot "^23.5.0" + jest-resolve@^22.4.2: version "22.4.3" resolved "https://registry.yarnpkg.com/jest-resolve/-/jest-resolve-22.4.3.tgz#0ce9d438c8438229aa9b916968ec6b05c1abb4ea" @@ -4982,6 +5290,14 @@ jest-resolve@^22.4.2: browser-resolve "^1.11.2" chalk "^2.0.1" +jest-resolve@^23.5.0: + version "23.5.0" + resolved "https://registry.yarnpkg.com/jest-resolve/-/jest-resolve-23.5.0.tgz#3b8e7f67e84598f0caf63d1530bd8534a189d0e6" + dependencies: + browser-resolve "^1.11.3" + chalk "^2.0.1" + realpath-native "^1.0.0" + jest-runner@^22.4.4: version "22.4.4" resolved "https://registry.yarnpkg.com/jest-runner/-/jest-runner-22.4.4.tgz#dfca7b7553e0fa617e7b1291aeb7ce83e540a907" @@ -4998,6 +5314,24 @@ jest-runner@^22.4.4: jest-worker "^22.2.2" throat "^4.0.0" +jest-runner@^23.5.0: + version "23.5.0" + resolved "https://registry.yarnpkg.com/jest-runner/-/jest-runner-23.5.0.tgz#570f7a044da91648b5bb9b6baacdd511076c71d7" + dependencies: + exit "^0.1.2" + graceful-fs "^4.1.11" + jest-config "^23.5.0" + jest-docblock "^23.2.0" + jest-haste-map "^23.5.0" + jest-jasmine2 "^23.5.0" + jest-leak-detector "^23.5.0" + jest-message-util "^23.4.0" + jest-runtime "^23.5.0" + jest-util "^23.4.0" + jest-worker "^23.2.0" + source-map-support "^0.5.6" + throat "^4.0.0" + jest-runtime@^22.4.4: version "22.4.4" resolved "https://registry.yarnpkg.com/jest-runtime/-/jest-runtime-22.4.4.tgz#9ba7792fc75582a5be0f79af6f8fe8adea314048" @@ -5023,10 +5357,40 @@ jest-runtime@^22.4.4: write-file-atomic "^2.1.0" yargs "^10.0.3" +jest-runtime@^23.5.0: + version "23.5.0" + resolved "https://registry.yarnpkg.com/jest-runtime/-/jest-runtime-23.5.0.tgz#eb503525a196dc32f2f9974e3482d26bdf7b63ce" + dependencies: + babel-core "^6.0.0" + babel-plugin-istanbul "^4.1.6" + chalk "^2.0.1" + convert-source-map "^1.4.0" + exit "^0.1.2" + fast-json-stable-stringify "^2.0.0" + graceful-fs "^4.1.11" + jest-config "^23.5.0" + jest-haste-map "^23.5.0" + jest-message-util "^23.4.0" + jest-regex-util "^23.3.0" + jest-resolve "^23.5.0" + jest-snapshot "^23.5.0" + jest-util "^23.4.0" + jest-validate "^23.5.0" + micromatch "^2.3.11" + realpath-native "^1.0.0" + slash "^1.0.0" + strip-bom "3.0.0" + write-file-atomic "^2.1.0" + yargs "^11.0.0" + jest-serializer@^22.4.3: version "22.4.3" resolved "https://registry.yarnpkg.com/jest-serializer/-/jest-serializer-22.4.3.tgz#a679b81a7f111e4766235f4f0c46d230ee0f7436" +jest-serializer@^23.0.1: + version "23.0.1" + resolved "https://registry.yarnpkg.com/jest-serializer/-/jest-serializer-23.0.1.tgz#a3776aeb311e90fe83fab9e533e85102bd164165" + jest-snapshot@^22.4.0: version "22.4.3" resolved "https://registry.yarnpkg.com/jest-snapshot/-/jest-snapshot-22.4.3.tgz#b5c9b42846ffb9faccb76b841315ba67887362d2" @@ -5038,6 +5402,21 @@ jest-snapshot@^22.4.0: natural-compare "^1.4.0" pretty-format "^22.4.3" +jest-snapshot@^23.5.0: + version "23.5.0" + resolved "https://registry.yarnpkg.com/jest-snapshot/-/jest-snapshot-23.5.0.tgz#cc368ebd8513e1175e2a7277f37a801b7358ae79" + dependencies: + babel-types "^6.0.0" + chalk "^2.0.1" + jest-diff "^23.5.0" + jest-matcher-utils "^23.5.0" + jest-message-util "^23.4.0" + jest-resolve "^23.5.0" + mkdirp "^0.5.1" + natural-compare "^1.4.0" + pretty-format "^23.5.0" + semver "^5.5.0" + jest-util@^22.4.1, jest-util@^22.4.3: version "22.4.3" resolved "https://registry.yarnpkg.com/jest-util/-/jest-util-22.4.3.tgz#c70fec8eec487c37b10b0809dc064a7ecf6aafac" @@ -5050,6 +5429,19 @@ jest-util@^22.4.1, jest-util@^22.4.3: mkdirp "^0.5.1" source-map "^0.6.0" +jest-util@^23.4.0: + version "23.4.0" + resolved "https://registry.yarnpkg.com/jest-util/-/jest-util-23.4.0.tgz#4d063cb927baf0a23831ff61bec2cbbf49793561" + dependencies: + callsites "^2.0.0" + chalk "^2.0.1" + graceful-fs "^4.1.11" + is-ci "^1.0.10" + jest-message-util "^23.4.0" + mkdirp "^0.5.1" + slash "^1.0.0" + source-map "^0.6.0" + jest-validate@^22.4.4: version "22.4.4" resolved "https://registry.yarnpkg.com/jest-validate/-/jest-validate-22.4.4.tgz#1dd0b616ef46c995de61810d85f57119dbbcec4d" @@ -5060,12 +5452,35 @@ jest-validate@^22.4.4: leven "^2.1.0" pretty-format "^22.4.0" +jest-validate@^23.5.0: + version "23.5.0" + resolved "https://registry.yarnpkg.com/jest-validate/-/jest-validate-23.5.0.tgz#f5df8f761cf43155e1b2e21d6e9de8a2852d0231" + dependencies: + chalk "^2.0.1" + jest-get-type "^22.1.0" + leven "^2.1.0" + pretty-format "^23.5.0" + +jest-watcher@^23.4.0: + version "23.4.0" + resolved "https://registry.yarnpkg.com/jest-watcher/-/jest-watcher-23.4.0.tgz#d2e28ce74f8dad6c6afc922b92cabef6ed05c91c" + dependencies: + ansi-escapes "^3.0.0" + chalk "^2.0.1" + string-length "^2.0.0" + jest-worker@^22.2.2, jest-worker@^22.4.3: version "22.4.3" resolved "https://registry.yarnpkg.com/jest-worker/-/jest-worker-22.4.3.tgz#5c421417cba1c0abf64bf56bd5fb7968d79dd40b" dependencies: merge-stream "^1.0.1" +jest-worker@^23.2.0: + version "23.2.0" + resolved "https://registry.yarnpkg.com/jest-worker/-/jest-worker-23.2.0.tgz#faf706a8da36fae60eb26957257fa7b5d8ea02b9" + dependencies: + merge-stream "^1.0.1" + jest@22.4.2: version "22.4.2" resolved "https://registry.yarnpkg.com/jest/-/jest-22.4.2.tgz#34012834a49bf1bdd3bc783850ab44e4499afc20" @@ -5073,6 +5488,13 @@ jest@22.4.2: import-local "^1.0.0" jest-cli "^22.4.2" +jest@^23.5.0: + version "23.5.0" + resolved "https://registry.yarnpkg.com/jest/-/jest-23.5.0.tgz#80de353d156ea5ea4a7332f7962ac79135fbc62e" + dependencies: + import-local "^1.0.0" + jest-cli "^23.5.0" + js-base64@^2.1.9: version "2.4.8" resolved "https://registry.yarnpkg.com/js-base64/-/js-base64-2.4.8.tgz#57a9b130888f956834aa40c5b165ba59c758f033" @@ -5257,6 +5679,10 @@ klaw@^1.0.0: optionalDependencies: graceful-fs "^4.1.9" +kleur@^2.0.1: + version "2.0.2" + resolved "https://registry.yarnpkg.com/kleur/-/kleur-2.0.2.tgz#b704f4944d95e255d038f0cb05fb8a602c55a300" + latest-version@^3.0.0: version "3.1.0" resolved "https://registry.yarnpkg.com/latest-version/-/latest-version-3.1.0.tgz#a205383fea322b33b5ae3b18abee0dc2f356ee15" @@ -6743,6 +7169,13 @@ pretty-format@^22.4.0, pretty-format@^22.4.3: ansi-regex "^3.0.0" ansi-styles "^3.2.0" +pretty-format@^23.5.0: + version "23.5.0" + resolved "https://registry.yarnpkg.com/pretty-format/-/pretty-format-23.5.0.tgz#0f9601ad9da70fe690a269cd3efca732c210687c" + dependencies: + ansi-regex "^3.0.0" + ansi-styles "^3.2.0" + private@^0.1.6, private@^0.1.8: version "0.1.8" resolved "https://registry.yarnpkg.com/private/-/private-0.1.8.tgz#2381edb3689f7a53d653190060fcf822d2f368ff" @@ -6771,6 +7204,13 @@ promise@^7.1.1: dependencies: asap "~2.0.3" +prompts@^0.1.9: + version "0.1.14" + resolved "https://registry.yarnpkg.com/prompts/-/prompts-0.1.14.tgz#a8e15c612c5c9ec8f8111847df3337c9cbd443b2" + dependencies: + kleur "^2.0.1" + sisteransi "^0.1.1" + prop-types@^15.5.10, prop-types@^15.5.4, prop-types@^15.5.7, prop-types@^15.6.0, prop-types@^15.6.1, prop-types@^15.6.2: version "15.6.2" resolved "https://registry.yarnpkg.com/prop-types/-/prop-types-15.6.2.tgz#05d5ca77b4453e985d60fc7ff8c859094a497102" @@ -6968,7 +7408,7 @@ react-error-overlay@^4.0.0: version "4.0.0" resolved "https://registry.yarnpkg.com/react-error-overlay/-/react-error-overlay-4.0.0.tgz#d198408a85b4070937a98667f500c832f86bd5d4" -react-is@^16.3.1: +react-is@^16.3.1, react-is@^16.4.2: version "16.4.2" resolved "https://registry.yarnpkg.com/react-is/-/react-is-16.4.2.tgz#84891b56c2b6d9efdee577cc83501dfc5ecead88" @@ -7065,6 +7505,15 @@ react-scripts-ts@2.16.0: optionalDependencies: fsevents "^1.1.3" +react-test-renderer@^16.4.2: + version "16.4.2" + resolved "https://registry.yarnpkg.com/react-test-renderer/-/react-test-renderer-16.4.2.tgz#4e03eca9359bb3210d4373f7547d1364218ef74e" + dependencies: + fbjs "^0.8.16" + object-assign "^4.1.1" + prop-types "^15.6.0" + react-is "^16.4.2" + react-transition-group@^2.2.1: version "2.4.0" resolved "https://registry.yarnpkg.com/react-transition-group/-/react-transition-group-2.4.0.tgz#1d9391fabfd82e016f26fabd1eec329dbd922b5a" @@ -7695,6 +8144,10 @@ signal-exit@^3.0.0, signal-exit@^3.0.2: version "3.0.2" resolved "https://registry.yarnpkg.com/signal-exit/-/signal-exit-3.0.2.tgz#b5fdc08f1287ea1178628e415e25132b73646c6d" +sisteransi@^0.1.1: + version "0.1.1" + resolved "https://registry.yarnpkg.com/sisteransi/-/sisteransi-0.1.1.tgz#5431447d5f7d1675aac667ccd0b865a4994cb3ce" + slash@^1.0.0: version "1.0.0" resolved "https://registry.yarnpkg.com/slash/-/slash-1.0.0.tgz#c41f2f6c39fc16d1cd17ad4b5d896114ae470d55" @@ -7801,6 +8254,13 @@ source-map-support@^0.5.0: buffer-from "^1.0.0" source-map "^0.6.0" +source-map-support@^0.5.6: + version "0.5.9" + resolved "https://registry.yarnpkg.com/source-map-support/-/source-map-support-0.5.9.tgz#41bc953b2534267ea2d605bccfa7bfa3111ced5f" + dependencies: + buffer-from "^1.0.0" + source-map "^0.6.0" + source-map-url@^0.4.0: version "0.4.0" resolved "https://registry.yarnpkg.com/source-map-url/-/source-map-url-0.4.0.tgz#3e935d7ddd73631b97659956d55128e87b5084a3" @@ -8322,6 +8782,15 @@ ts-jest@22.0.1: source-map-support "^0.5.0" yargs "^10.0.3" +ts-jest@^23.1.4: + version "23.1.4" + resolved "https://registry.yarnpkg.com/ts-jest/-/ts-jest-23.1.4.tgz#66ac1d8d3fbf8f9a98432b11aa377aa850664b2b" + dependencies: + closest-file-data "^0.1.4" + fs-extra "6.0.1" + json5 "^0.5.0" + lodash "^4.17.10" + ts-loader@^2.3.7: version "2.3.7" resolved "https://registry.yarnpkg.com/ts-loader/-/ts-loader-2.3.7.tgz#a9028ced473bee12f28a75f9c5b139979d33f2fc" @@ -8420,6 +8889,18 @@ typedarray@^0.0.6: version "0.0.6" resolved "https://registry.yarnpkg.com/typedarray/-/typedarray-0.0.6.tgz#867ac74e3864187b1d3d47d996a78ec5c8830777" +typescript-babel-jest@^1.0.5: + version "1.0.5" + resolved "https://registry.yarnpkg.com/typescript-babel-jest/-/typescript-babel-jest-1.0.5.tgz#5f5acffb7495cb050601f056e4ec07ac52c69445" + dependencies: + app-root-path "2.0.1" + babel-jest "20.0.3" + typescript "^2.4.1" + +typescript@^2.4.1: + version "2.9.2" + resolved "https://registry.yarnpkg.com/typescript/-/typescript-2.9.2.tgz#1cbf61d05d6b96269244eb6a3bce4bd914e0f00c" + typescript@^3.0.1: version "3.0.1" resolved "https://registry.yarnpkg.com/typescript/-/typescript-3.0.1.tgz#43738f29585d3a87575520a4b93ab6026ef11fdb" @@ -9065,6 +9546,12 @@ yargs-parser@^8.1.0: dependencies: camelcase "^4.1.0" +yargs-parser@^9.0.2: + version "9.0.2" + resolved "https://registry.yarnpkg.com/yargs-parser/-/yargs-parser-9.0.2.tgz#9ccf6a43460fe4ed40a9bb68f48d43b8a68cc077" + dependencies: + camelcase "^4.1.0" + yargs@^10.0.3: version "10.1.2" resolved "https://registry.yarnpkg.com/yargs/-/yargs-10.1.2.tgz#454d074c2b16a51a43e2fb7807e4f9de69ccb5c5" @@ -9082,6 +9569,23 @@ yargs@^10.0.3: y18n "^3.2.1" yargs-parser "^8.1.0" +yargs@^11.0.0: + version "11.1.0" + resolved "http://registry.npmjs.org/yargs/-/yargs-11.1.0.tgz#90b869934ed6e871115ea2ff58b03f4724ed2d77" + dependencies: + cliui "^4.0.0" + decamelize "^1.1.1" + find-up "^2.1.0" + get-caller-file "^1.0.1" + os-locale "^2.0.0" + require-directory "^2.1.1" + require-main-filename "^1.0.1" + set-blocking "^2.0.0" + string-width "^2.0.0" + which-module "^2.0.0" + y18n "^3.2.1" + yargs-parser "^9.0.2" + yargs@^6.6.0: version "6.6.0" resolved "https://registry.yarnpkg.com/yargs/-/yargs-6.6.0.tgz#782ec21ef403345f830a808ca3d513af56065208" From c415855f53636cee77da1656abdda4f0762f3a21 Mon Sep 17 00:00:00 2001 From: Mikhail Novikov Date: Thu, 6 Sep 2018 14:51:47 +0300 Subject: [PATCH 055/103] OKAY CIRCLE --- .circleci/config.yml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index cbf91aed9495a..1bcb31e2a5822 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -89,18 +89,20 @@ jobs: dagit-webapp: docker: - image: circleci/node:10.6 - working_directory: ~/repo/python_modules/dagit/dagit/webapp + working_directory: ~/repo steps: - checkout - run: name: Install Dependencies command: | + cd python_modules/dagit/dagit/webapp yarn - run: name: Run Dagit Webapp Tests command: | + cd python_modules/dagit/dagit/webapp yarn test - store_artifacts: From 04ab5675d10028df858208b47d467239f3be4327 Mon Sep 17 00:00:00 2001 From: Mikhail Novikov Date: Thu, 6 Sep 2018 14:57:12 +0300 Subject: [PATCH 056/103] Run typescript too --- .circleci/config.yml | 10 ++++++++-- python_modules/dagit/dagit/webapp/package.json | 4 +++- .../dagit/dagit/webapp/src/__tests__/App.test.tsx | 2 +- 3 files changed, 12 insertions(+), 4 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 1bcb31e2a5822..a4a624b112b8e 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -100,10 +100,16 @@ jobs: yarn - run: - name: Run Dagit Webapp Tests + name: Run Typescript command: | cd python_modules/dagit/dagit/webapp - yarn test + yarn run ts + + - run: + name: Run Tests + command: | + cd python_modules/dagit/dagit/webapp + yarn run jest - store_artifacts: path: test-reports diff --git a/python_modules/dagit/dagit/webapp/package.json b/python_modules/dagit/dagit/webapp/package.json index 32b1e28882755..456dd7d09db4e 100644 --- a/python_modules/dagit/dagit/webapp/package.json +++ b/python_modules/dagit/dagit/webapp/package.json @@ -50,8 +50,10 @@ "scripts": { "start": "react-scripts-ts start", "build": "react-scripts-ts build", - "test": "jest", "eject": "react-scripts-ts eject", + "test": "npm run ts && npm run jest", + "jest": "jest", + "ts": "tsc -p .", "download-schema": "apollo schema:download --endpoint $REACT_APP_GRAPHQL_URI ./src/schema.json", "generate-types": "apollo codegen:generate --queries \"./src/**/*.tsx\" --target typescript types --schema ./src/schema.json", "watch-types": "apollo codegen:generate --queries \"./src/**/*.tsx\" --target typescript types --schema ./src/schema.json --watch" diff --git a/python_modules/dagit/dagit/webapp/src/__tests__/App.test.tsx b/python_modules/dagit/dagit/webapp/src/__tests__/App.test.tsx index db786606a9f08..e05c6838ac83a 100644 --- a/python_modules/dagit/dagit/webapp/src/__tests__/App.test.tsx +++ b/python_modules/dagit/dagit/webapp/src/__tests__/App.test.tsx @@ -201,7 +201,7 @@ const mocks = [ } ]; -function createNodeMock(element) { +function createNodeMock(element: any) { if (element.type === "div") { return { querySelector() { From 52dba79c2806803fea00e52034d7f9a2a2d64470 Mon Sep 17 00:00:00 2001 From: Taylor Miller Date: Thu, 6 Sep 2018 14:18:39 -0600 Subject: [PATCH 057/103] * some minor typos in docs --- python_modules/dagster/docs/principles.rst | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/python_modules/dagster/docs/principles.rst b/python_modules/dagster/docs/principles.rst index 64a250560a6aa..ef13964d64a0a 100644 --- a/python_modules/dagster/docs/principles.rst +++ b/python_modules/dagster/docs/principles.rst @@ -38,8 +38,8 @@ Testing data pipelines is notoriously difficult. Because it is so difficult it i done, or done poorly. Dagster pipelines are designed to be tested. They have explicit support for pipeline authors to manage and maintain multiple operating environments -- for example, unit testing, integration testing, and production environments, among others. In addition dagster can -execute arbitrary subsets and nodes of the pipeline, critical testability. (This capability -happens to be useful in operational contexts as well). +execute arbitrary subsets and nodes of the pipeline which is critical for testability. (This +capability happens to be useful in operational contexts as well). Verifiable data quality ^^^^^^^^^^^^^^^^^^^^^^^ @@ -48,7 +48,7 @@ during every meaningful stage of computation in production -- are critical to re maintenance burden of data pipelines. Pipeline authors generally do not have control of their input data, and make many implicit assumptions about that data. The data formats can also change over time. In order to control this entropy, Dagster encourages users to computationally verify -assumptions (known as expectations) about the data as part of the piplien process. This way if +assumptions (known as expectations) about the data as part of the pipeline process. This way if those assumptions are broken, the breakage can be reported quickly, easily, and with rich metadata and diagnostic information. These expectations can also serve as contracts between teams. @@ -59,5 +59,5 @@ Gradual, optional typing Dagster contains a type system to describe the values flowing through the pipeline and the configuration of the pipeline. This allows nodes in a pipeline know if they are properly -arranged and configuration prior to execution before execution, and serves as value +arranged and configuration prior to execution before execution, and serves as valuable documentation and runtime error checking. From 724669e422e2fa8be26e7dcf82a6759489bd1c75 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Thu, 6 Sep 2018 16:12:08 -0700 Subject: [PATCH 058/103] Various, sundry fixes while debugging Added some logging and did some refactoring --- .../dagster/dagster/core/compute_nodes.py | 88 +++++++++++++------ .../core/core_tests/test_decorators.py | 1 - .../core/core_tests/test_pipeline_errors.py | 1 - .../dagster/dagster/core/execution.py | 8 ++ 4 files changed, 70 insertions(+), 28 deletions(-) diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index b99c99ac0d411..e774dad46f719 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -72,7 +72,13 @@ def __new__(cls, compute_node, output_name): def __str__(self): return ( 'ComputeNodeOutputHandle' - '(guid="{cn.compute_node.guid}", output_name="{cn.output_name}")'.format(cn=self) + '(cn="{cn.compute_node.friendly_name}", output_name="{cn.output_name}")'.format(cn=self) + ) + + def __repr__(self): + return ( + 'ComputeNodeOutputHandle' + '(cn="{cn.compute_node.friendly_name}", output_name="{cn.output_name}")'.format(cn=self) ) def __hash__(self): @@ -172,6 +178,34 @@ class ComputeNodeTag(Enum): EXPECTATION_INPUT = 'expectation_input' +def _yield_transform_results(context, compute_node, values_dict, config_dict): + gen = compute_node.solid.transform_fn(context, values_dict, config_dict) + + if isinstance(gen, Result): + raise DagsterInvariantViolationError( + ( + 'Transform for solid {solid_name} returned a Result rather than ' + + 'yielding it. The transform_fn of the core SolidDefinition must yield ' + + 'its results' + ).format(solid_name=compute_node.solid.name) + ) + + if gen is None: + return + + for result in gen: + if not isinstance(result, Result): + raise DagsterInvariantViolationError( + ( + 'Transform for solid {solid_name} yielded {result} rather an ' + + 'an instance of the Result class.' + ).format( + result=repr(result), + solid_name=compute_node.solid.name, + ) + ) + yield result + def _execute_core_transform(context, compute_node, values_dict, config_dict): ''' Execute the user-specified transform for the solid. Wrap in an error boundary and do @@ -183,34 +217,19 @@ def _execute_core_transform(context, compute_node, values_dict, config_dict): check.dict_param(config_dict, 'config_dict', key_type=str) error_str = 'Error occured during core transform' + + context.debug( + 'Executing core transform for solid {solid}.'.format( + solid=compute_node.solid.name + ) + ) + with _user_code_error_boundary(context, error_str), \ time_execution_scope() as timer_result, \ context.value('solid', compute_node.solid.name): - gen = compute_node.solid.transform_fn(context, values_dict, config_dict) - - if isinstance(gen, Result): - raise DagsterInvariantViolationError( - ( - 'Transform for solid {solid_name} returned a Result rather than ' + - 'yielding it. The transform_fn of the core SolidDefinition must yield ' + - 'its results' - ).format(solid_name=compute_node.solid.name, ) - ) - - if gen is not None: - for result in gen: - if not isinstance(result, Result): - raise DagsterInvariantViolationError( - ( - 'Transform for solid {solid_name} yielded {result} rather an ' + - 'an instance of the Result class.' - ).format( - result=repr(result), - solid_name=compute_node.solid.name, - ) - ) - yield result + for result in _yield_transform_results(context, compute_node, values_dict, config_dict): + yield result context.metric('core_transform_time_ms', timer_result.millis) @@ -379,8 +398,24 @@ def execute_compute_nodes(context, compute_nodes): check.list_param(compute_nodes, 'compute_nodes', of_type=ComputeNode) intermediate_results = {} + context.debug('Entering exeute_compute_nodes loop. Order: {order}'.format( + order=[cn.friendly_name for cn in compute_nodes] + )) + for compute_node in compute_nodes: if not _all_inputs_covered(compute_node, intermediate_results): + result_keys = set(intermediate_results.keys()) + expected_outputs = [ni.prev_output_handle for ni in compute_node.node_inputs] + + context.debug( + 'Not all inputs covered for {compute_name}. Not executing.'.format( + compute_name=compute_node.friendly_name + ) + '\nKeys in result: {result_keys}.'.format( + result_keys=result_keys, + ) + '\nOutputs need for inputs {expected_outputs}'.format( + expected_outputs=expected_outputs, + ) + ) continue input_values = {} @@ -603,6 +638,7 @@ def create_compute_node_graph(execution_info): output_handle = topo_solid.output_handle(output_def.name) cn_output_node_map[output_handle] = subgraph.terminal_cn_output_handle + return _create_compute_node_graph(compute_nodes) @@ -611,7 +647,7 @@ def _create_compute_node_graph(compute_nodes): for cn in compute_nodes: cn_dict[cn.guid] = cn - deps = defaultdict(set) + deps = defaultdict() for cn in compute_nodes: deps[cn.guid] = set() diff --git a/python_modules/dagster/dagster/core/core_tests/test_decorators.py b/python_modules/dagster/dagster/core/core_tests/test_decorators.py index 29b5b98414ae5..75105c9ad9ad2 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_decorators.py +++ b/python_modules/dagster/dagster/core/core_tests/test_decorators.py @@ -197,7 +197,6 @@ def hello_world(foo_to_foo): pipeline, environment=config.Environment(), ) - print([p.solid.name for p in pipeline_result.result_list]) result = pipeline_result.result_for_solid('hello_world') diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py index 99c0e6e15ef58..8c9177d458e34 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py @@ -107,7 +107,6 @@ def transform_fn(_context, args): ) result_list = pipeline_result.result_list - print([r.solid.name for r in result_list]) assert result_list[0].success assert result_list[1].success diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index 0dd12d1df5b83..bc2c7c8ad8988 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -236,6 +236,14 @@ def _execute_graph_iterator(context, execution_graph, environment): cn_nodes = list(cn_graph.topological_nodes()) + check.invariant(len(cn_nodes) > 0, 'No compute nodes!') + + context.debug( + 'About to execute the compute node graph in the following order {order}'.format( + order=[cn.friendly_name for cn in cn_nodes] + ) + ) + check.invariant(len(cn_nodes[0].node_inputs) == 0) solid = None From 9ff2b739ed2b266e1d5b502e26b34173112aaae1 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Thu, 6 Sep 2018 16:55:40 -0700 Subject: [PATCH 059/103] PipelineContextDefinition --- python_modules/dagster/dagster/__init__.py | 1 + .../dagster/dagster/core/definitions.py | 31 +++++++++++-------- 2 files changed, 19 insertions(+), 13 deletions(-) diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index 05b55b09b7547..836edba15dc3f 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -41,5 +41,6 @@ 'Field', 'InputDefinition', 'OutputDefinition', + 'PipelineContextDefinition', 'PipelineDefinition', ] diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index c562f23544ebc..b1bd3864b0f1d 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -74,24 +74,26 @@ class PipelineContextDefinition(object): a dictionary key'ed by its name so the name is not present in this object. Attributes: - config_def (ConfigurationDefinition): + config_def (ConfigurationDefinition): The configuration for the pipeline context. + context_fn (callable): - A callable that either returns *or* yields an ExecutionContext. Type sig: - (pipeline: PipelineDefinition, config: Any) : ExecutionContext + Signature is (pipeline: PipelineDefintion, config_value: Any) => ExecutionContext + + A callable that either returns *or* yields an ExecutionContext. + description (str): A description of what this context represents ''' @staticmethod def passthrough_context_definition(context): - '''Create a context definition from a pre-existing context. - This can be useful in testing contexts where you may want to - create a context manually and then pass it into a one-off - PipelineDefinition + '''Create a context definition from a pre-existing context. This can be useful + in testing contexts where you may want to create a context manually and then + pass it into a one-off PipelineDefinition Args: - context: ExecutionContext + context (ExecutionContext): The context that will provided to the pipeline. Returns: - PipelineContextDefinition + PipelineContextDefinition: The passthrough context definition. ''' check.inst_param(context, 'context', ExecutionContext) @@ -101,10 +103,10 @@ def passthrough_context_definition(context): def __init__(self, context_fn, config_def=None, description=None): ''' Args: - config_def: ConfigDefinition - Define the configuration for the context + context_fn (callable): + Signature of context_fn: + (pipeline: PipelineDefintion, config_value: Any) => ExecutionContext - context_fn: callable (pipeline: PipelineDefinition, args: dict str => Any Returns *or* yields an ExecutionContext. If it yields a context, the code after the yield executes after pipeline @@ -116,7 +118,10 @@ def __init__(self, context_fn, config_def=None, description=None): resources object be a namedtuple, where each property is an object that manages a particular resource, e.g. aws, a local filesystem manager, etc. - description: str (optional) + config_def (ConfigDefinition): + Define the configuration for the context + + description (str): Description of the context definition. ''' self.config_def = check.opt_inst_param( config_def, From 4531634b6fe58c32132cc5e8dca29d556995692d Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Thu, 6 Sep 2018 16:57:28 -0700 Subject: [PATCH 060/103] Result --- python_modules/dagster/dagster/__init__.py | 1 + python_modules/dagster/dagster/core/definitions.py | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index 836edba15dc3f..90a9a599369a9 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -43,4 +43,5 @@ 'OutputDefinition', 'PipelineContextDefinition', 'PipelineDefinition', + 'Result', ] diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index b1bd3864b0f1d..715e90b4aeb88 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -720,8 +720,9 @@ class Result(namedtuple('_Result', 'value output_name')): yields objects of this type. Attributes: - value: Any - output_name: str (optional) defaults to "result"''' + value (Any): Value returned by the transform. + output_name (str): Name of the output returns. defaults to "result" +''' def __new__(cls, value, output_name=DEFAULT_OUTPUT): return super(Result, cls).__new__( From 388650f080b54e86544fb687ea66b71bb4c3a5bb Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Thu, 6 Sep 2018 17:08:09 -0700 Subject: [PATCH 061/103] RepositoryDefinition --- python_modules/dagster/dagster/__init__.py | 1 + .../dagster/dagster/core/definitions.py | 43 ++++++++++++++++++- 2 files changed, 42 insertions(+), 2 deletions(-) diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index 90a9a599369a9..49784c47f48cc 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -44,4 +44,5 @@ 'PipelineContextDefinition', 'PipelineDefinition', 'Result', + 'RepositoryDefinition', ] diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index 715e90b4aeb88..caa2ec83a4643 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -1087,7 +1087,28 @@ def _create_subgraph(execution_graph, from_solids, through_solids): class RepositoryDefinition(object): + '''Define a repository that contains a collection of pipelines. + + Attributes: + name (str): The name of the pipeline. + pipeline_dict (Dict[str, callable]): + An dictionary of pipelines. The value of the dictionary is a function that takes + no parameters and returns a PipelineDefiniton. + + We pass callables instead of the PipelineDefinitions itself so that they can be + created on demand when accessed by name. + + As the pipelines are retrieved it ensures that the keys of the dictionary and the + name of the pipeline are the same. + + ''' + def __init__(self, name, pipeline_dict): + ''' + Args: + name (str): Name of pipeline. + pipeline_dict (Dict[str, callable]): See top-level class documentation + ''' self.name = check.str_param(name, 'name') check.dict_param( @@ -1104,6 +1125,14 @@ def __init__(self, name, pipeline_dict): self._pipeline_cache = {} def get_pipeline(self, name): + '''Get a pipeline by name. Only constructs that pipeline and caches it. + + Args: + name (str): Name of the pipeline to retriever + + Returns: + PipelineDefinition: Instance of PipelineDefinition with that name. +''' if name in self._pipeline_cache: return self._pipeline_cache[name] @@ -1115,19 +1144,29 @@ def get_pipeline(self, name): ) ) - check.inst( + self._pipeline_cache[name] = check.inst( pipeline, PipelineDefinition, 'Function passed into pipeline_dict with key {key} must return a PipelineDefinition'. format(key=name), ) - self._pipeline_cache[name] = pipeline return pipeline def iterate_over_pipelines(self): + '''Yield all pipelines one at a time + + Returns: + Iterable[PipelineDefinition]: + ''' for name in self.pipeline_dict.keys(): yield self.get_pipeline(name) def get_all_pipelines(self): + '''Return all pipelines as a list + + Returns: + List[PipelineDefinition]: + + ''' return list(self.iterate_over_pipelines()) From 6c25b9f606a0599fa7bdd724454f2f787ed3ea19 Mon Sep 17 00:00:00 2001 From: Mikhail Novikov Date: Fri, 7 Sep 2018 11:49:41 +0300 Subject: [PATCH 062/103] Make apidocs render --- python_modules/dagster/docs/apidocs/index.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python_modules/dagster/docs/apidocs/index.rst b/python_modules/dagster/docs/apidocs/index.rst index e3cc3b4253ece..015405875e38b 100644 --- a/python_modules/dagster/docs/apidocs/index.rst +++ b/python_modules/dagster/docs/apidocs/index.rst @@ -6,4 +6,4 @@ Top-level Dagster API .. automodule:: dagster :members: - :special-members: __init__ + :undoc-members: From 3546d8dfb3f417aa329cc25346cf86e2716ccd5c Mon Sep 17 00:00:00 2001 From: Mikhail Novikov Date: Fri, 7 Sep 2018 15:57:21 +0300 Subject: [PATCH 063/103] Add more docs --- python_modules/dagit/bin/publish.sh | 1 + python_modules/dagster/bin/publish.sh | 1 + python_modules/dagster/dagster/__init__.py | 31 +++++- .../dagster/dagster/core/decorators.py | 97 ++++++++++++++++++- .../dagster/dagster/core/definitions.py | 18 +++- .../dagster/dagster/core/execution.py | 44 ++++++++- python_modules/dagster/dagster/core/types.py | 60 +++++++++++- .../dagster/docs/apidocs/decorators.rst | 13 +++ .../dagster/docs/apidocs/definitions.rst | 42 ++++++++ .../dagster/docs/apidocs/errors.rst | 16 +++ .../dagster/docs/apidocs/execution.rst | 19 ++++ python_modules/dagster/docs/apidocs/index.rst | 16 ++- python_modules/dagster/docs/apidocs/types.rst | 40 ++++++++ python_modules/dagster/docs/conf.py | 2 + python_modules/dagster/docs/contributing.rst | 68 +++++++++++++ .../dagster/docs/developing/index.rst | 36 ------- python_modules/dagster/docs/index.rst | 31 +++--- python_modules/dagster/docs/principles.rst | 6 +- 18 files changed, 467 insertions(+), 74 deletions(-) create mode 100644 python_modules/dagster/docs/apidocs/decorators.rst create mode 100644 python_modules/dagster/docs/apidocs/definitions.rst create mode 100644 python_modules/dagster/docs/apidocs/errors.rst create mode 100644 python_modules/dagster/docs/apidocs/execution.rst create mode 100644 python_modules/dagster/docs/apidocs/types.rst create mode 100644 python_modules/dagster/docs/contributing.rst delete mode 100644 python_modules/dagster/docs/developing/index.rst diff --git a/python_modules/dagit/bin/publish.sh b/python_modules/dagit/bin/publish.sh index 10abbd212262e..8edc870a96111 100755 --- a/python_modules/dagit/bin/publish.sh +++ b/python_modules/dagit/bin/publish.sh @@ -1,5 +1,6 @@ #!/bin/bash +rm -r dist/ cd ./dagit/webapp yarn install yarn build diff --git a/python_modules/dagster/bin/publish.sh b/python_modules/dagster/bin/publish.sh index ee425a4a99e28..7f48d10ac8639 100755 --- a/python_modules/dagster/bin/publish.sh +++ b/python_modules/dagster/bin/publish.sh @@ -1,4 +1,5 @@ #!/bin/bash +rm -r dist/ python3 setup.py sdist bdist_wheel twine upload dist/* diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index 49784c47f48cc..a24357e00d4ae 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -1,7 +1,7 @@ from __future__ import (absolute_import, division, print_function, unicode_literals) from builtins import * # pylint: disable=W0622,W0401 -from dagster.core.execution import execute_pipeline +from dagster.core.execution import execute_pipeline_iterator, execute_pipeline, PipelineExecutionResult, SolidExecutionResult from dagster.core.execution_context import ExecutionContext from dagster.core.definitions import ( @@ -29,20 +29,45 @@ DagsterInvalidDefinitionError, DagsterInvariantViolationError, DagsterTypeError, + DagsterUserCodeExecutionError, + DagsterExpectationFailedError, ) import dagster.core.types as types __all__ = [ + # Definition 'ConfigDefinition', 'DependencyDefinition', 'ExpectationDefinition', 'ExpectationResult', - 'Field', 'InputDefinition', 'OutputDefinition', 'PipelineContextDefinition', 'PipelineDefinition', - 'Result', 'RepositoryDefinition', + 'SolidDefinition', + 'Result', + + # Decorators + 'solid', + 'with_context', + 'MultipleResults', + + # Execution + 'execute_pipeline_iterator', + 'execute_pipeline', + 'ExecutionContext', + 'PipelineExecutionResult', + 'SolidExecutionResult', + + # Errors + 'DagsterInvalidDefinitionError', + 'DagsterInvariantViolationError', + 'DagsterTypeError', + 'DagsterUserCodeExecutionError', + 'DagsterExpectationFailedError', + + # types + 'types', ] diff --git a/python_modules/dagster/dagster/core/decorators.py b/python_modules/dagster/dagster/core/decorators.py index 5f7da5a258ccc..5e1f253cdce79 100644 --- a/python_modules/dagster/dagster/core/decorators.py +++ b/python_modules/dagster/dagster/core/decorators.py @@ -23,6 +23,37 @@ class MultipleResults(namedtuple('_MultipleResults', 'results')): + '''A shortcut to output multiple results. + + Attributes: + results (list[Result]): list of :py:class:`Result` + + Example: + + .. code-block:: python + + @solid(outputs=[ + OutputDefinition(name='foo'), + OutputDefinition(name='bar'), + ]) + def my_solid(): + return MultipleResults([ + Result('Barb', 'foo'), + Result('Glarb', 'bar'), + ]) + + + @solid(outputs=[ + OutputDefinition(name='foo'), + OutputDefinition(name='bar'), + ]) + def my_solid_from_dict(): + return MultipleResults.from_dict({ + 'foo': 'Barb', + 'bar': 'Glarb', + }) + ''' + def __new__(cls, *results): return super(MultipleResults, cls).__new__( cls, @@ -32,6 +63,7 @@ def __new__(cls, *results): @staticmethod def from_dict(result_dict): + '''Create MultipleResults object from a dictionary. Keys become result names''' check.dict_param(result_dict, 'result_dict', key_type=str) results = [] for name, value in result_dict.items(): @@ -40,8 +72,17 @@ def from_dict(result_dict): def with_context(fn): - """Pass context as a first argument to a transform. - """ + '''(decorator) Pass context as a first argument to a transform. + + Example: + + .. code-block:: python + + @solid() + @with_context + def my_solid(context): + pass + ''' return _WithContext(fn) @@ -96,6 +137,58 @@ def __call__(self, fn): def solid(name=None, inputs=None, output=None, outputs=None, description=None): + '''(decorator) Create a solid with specified parameters. + + This shortcut simplifies core solid API by exploding arguments into kwargs of the transform function and omitting additional parameters when they are not needed. Parameters are otherwise as per :py:class:`SolidDefinition`. By using :py:function:`with_context` one can request context object to be passed too. + + Decorated function becomes transform. Instead of having to yield result objects, transform support multiple simpler output types. + + 1. Return a value. This is returned as a :py:class:`Result` for a single output solid. + 2. Return a :py:class:`Result`. Works like yielding result. + 3. Return a :py:class:`MultipleResults`. Works like yielding several results for multiple outputs. Useful for solids that have multiple outputs. + 4. Yield :py:class:`Result`. Same as default transform behaviour. + + Examples: + + .. code-block:: python + + @solid(outputs=[OutputDefinition()]) + def hello_world(): + return {'foo': 'bar'} + + @solid(output=OutputDefinition()) + def hello_world(): + return Result(value={'foo': 'bar'}) + + @solid(output=OutputDefinition()) + def hello_world(): + yield Result(value={'foo': 'bar'}) + + @solid(outputs=[ + OutputDefinition(name="left"), + OutputDefinition(name="right"), + ]) + def hello_world(): + return MultipleResults.from_dict({ + 'left': {'foo': 'left'}, + 'right': {'foo': 'right'}, + }) + + @solid( + inputs=[InputDefinition(name="foo_to_foo")], + outputs=[OutputDefinition()] + ) + def hello_world(foo_to_foo): + return foo_to_foo + + @solid( + inputs=[InputDefinition(name="foo_to_foo")], + outputs=[OutputDefinition()] + ) + @with_context + def hello_world(context, foo_to_foo): + return foo_to_foo + ''' return _Solid(name=name, inputs=inputs, output=output, outputs=outputs, description=description) diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index caa2ec83a4643..8404e94ddd150 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -91,7 +91,7 @@ def passthrough_context_definition(context): pass it into a one-off PipelineDefinition Args: - context (ExecutionContext): The context that will provided to the pipeline. + context (ExecutionContext): The context that will provided to the pipeline. Returns: PipelineContextDefinition: The passthrough context definition. ''' @@ -746,6 +746,22 @@ class ConfigDefinition(object): @staticmethod def config_dict(field_dict): + '''Shorcut to create a dictionary based config definition. + + + Args: + field_dict (dict): dictionary of `Field` objects keyed by their names. + + Example: + + .. code-block:: python + + ConfigDefinition.config_dict({ + 'int_field': Field(types.Int), + 'string_field': Field(types.String), + }) + + ''' return ConfigDefinition(types.ConfigDictionary(field_dict)) def __init__(self, config_type=types.Any): diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index bc2c7c8ad8988..60b102403e770 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -49,6 +49,13 @@ class PipelineExecutionResult(object): + '''Result of execution of the whole pipeline. Returned eg by :py:function:`execute_pipeline`. + + Attributes: + context (ExecutionContext): ExecutionContext of that particular Pipeline run. + result_list (list[SolidExecutionResult]): List of results for each pipeline solid. + ''' + def __init__( self, context, @@ -61,9 +68,15 @@ def __init__( @property def success(self): + '''Whether the pipeline execution was successful at all steps''' return all([result.success for result in self.result_list]) def result_for_solid(self, name): + '''Get a :py:class:`SolidExecutionResult` for a given solid name. + + Returns: + SolidExecutionResult + ''' check.str_param(name, 'name') for result in self.result_list: if result.solid.name == name: @@ -74,6 +87,13 @@ def result_for_solid(self, name): class SolidExecutionResult(object): + '''Execution result for one solid of the pipeline. + + Attributes: + context (ExecutionContext): ExecutionContext of that particular Pipeline run. + solid (SolidDefinition): Solid for which this result is + ''' + def __init__(self, context, solid, input_expectations, transforms, output_expectations): self.context = check.inst_param(context, 'context', ExecutionContext) self.solid = check.inst_param(solid, 'solid', SolidDefinition) @@ -113,6 +133,7 @@ def from_results(context, results): @property def success(self): + '''Whether the solid execution was successful''' return all( [ result.success for result in @@ -122,6 +143,7 @@ def success(self): @property def transformed_values(self): + '''Return dictionary of transformed results, with keys being output names. Returns None if execution result isn't a success.''' if self.success and self.transforms: return { result.success_data.output_name: result.success_data.value @@ -131,6 +153,7 @@ def transformed_values(self): return None def transformed_value(self, output_name=DEFAULT_OUTPUT): + '''Returns transformed value either for DEFAULT_OUTPUT or for the output given as outputname. Returns None if execution result isn't a success''' check.str_param(output_name, 'output_name') if self.success: for result in self.transforms: @@ -153,6 +176,7 @@ def reraise_user_error(self): @property def dagster_user_exception(self): + '''Returns exception that happened during this solid's execution, if any''' for result in itertools.chain( self.input_expectations, self.output_expectations, self.transforms ): @@ -213,6 +237,12 @@ def yield_context(pipeline, environment): def execute_pipeline_iterator(pipeline, environment): + '''Returns iterator that yields :py:class:`SolidExecutionResult` for each solid executed in the pipeline + + Parameters: + pipeline (PipelineDefinition): pipeline to run + execution (ExecutionContext): execution context of the run + ''' check.inst_param(pipeline, 'pipeline', PipelineDefinition) check.inst_param(environment, 'enviroment', config.Environment) @@ -274,12 +304,18 @@ def execute_pipeline( throw_on_error=True, ): ''' - "Synchronous" version of execute_pipeline_iteator. - - throw_on_error makes the function throw when an error is encoutered rather than returning - the LegacySolidExecutionResult in an error-state. + "Synchronous" version of `execute_pipeline_iterator`. Note: throw_on_error is very useful in testing contexts when not testing for error conditions + + Parameters: + pipeline (PipelineDefinition): pipeline to run + execution (ExecutionContext): execution context of the run + throw_on_error (bool): throw_on_error makes the function throw when an error is encoutered rather than returning the py:class:`SolidExecutionResult` in an error-state. + + + Returns: + PipelineExecutionResult ''' check.inst_param(pipeline, 'pipeline', PipelineDefinition) diff --git a/python_modules/dagster/dagster/core/types.py b/python_modules/dagster/dagster/core/types.py index 66118d12f9f6a..89def0739db92 100644 --- a/python_modules/dagster/dagster/core/types.py +++ b/python_modules/dagster/dagster/core/types.py @@ -5,25 +5,60 @@ class DagsterType(object): + '''Base class for Dagster Type system. Should be inherited by a subclass. Subclass must implement `evaluate_value` + + Attributes: + name (str): Name of the type + + description (str): Description of the type + ''' + def __init__(self, name, description=None): self.name = check.str_param(name, 'name') self.description = check.opt_str_param(description, 'description') + self.__doc__ = description def __repr__(self): return 'DagsterType({name})'.format(name=self.name) def evaluate_value(self, _value): + '''Subclasses must implement this method. Check if the value is a valid one and output :py:class:`IncomingValueResult`. + + Args: + value: The value to check + + Returns: + IncomingValueResult + ''' check.not_implemented('Must implement in subclass') class DagsterScalarType(DagsterType): + '''Base class for dagster types that are scalar python values. + + Attributes: + name (str): Name of the type + + description (str): Description of the type + ''' + def __init__(self, *args, **kwargs): super(DagsterScalarType, self).__init__(*args, **kwargs) def process_value(self, value): + '''Modify the value before it's evaluated. Subclasses may override. + + Returns: + any: New value + ''' return value def is_python_valid_value(self, _value): + '''Subclasses must implement this method. Check if the value and output a boolean. + + Returns: + bool: Whether the value is valid. + ''' raise Exception('must implement') def evaluate_value(self, value): @@ -57,6 +92,8 @@ def nullable_isinstance(value, typez): class PythonObjectType(DagsterType): + '''Dagster Type that checks if the value is an instance of some `python_type`''' + def __init__( self, name, @@ -161,6 +198,9 @@ def __setitem__(self, _key, _value): class DagsterCompositeType(DagsterType): + '''Dagster type representing a type with a list of named :py:class:`Field` objects. + ''' + def __init__(self, name, fields, ctor, description=None): self.field_dict = FieldDefinitionDictionary(fields) self.ctor = check.callable_param(ctor, 'ctor') @@ -173,17 +213,31 @@ def evaluate_value(self, value): class ConfigDictionary(DagsterCompositeType): + '''Configuration dictionary. + + Typed-checked but then passed to implementations as a python dict + + Arguments: + fields (dict): dictonary of :py:class:`Field` objects keyed by name''' + def __init__(self, fields): super(ConfigDictionary, self).__init__( 'ConfigDictionary', fields, lambda val: val, - '''Configuration dictionary. - Typed-checked but then passed to implementations as a python dict''', + self.__doc__, ) class IncomingValueResult(namedtuple('_IncomingValueResult', 'success value error_msg')): + '''Result of a dagster typecheck. + + Attributes: + success (bool): whether value is a valid one. + value (any): the actual value + error_msg (str): error message + ''' + def __new__(cls, success, value, error_msg): return super(IncomingValueResult, cls).__new__( cls, @@ -194,10 +248,12 @@ def __new__(cls, success, value, error_msg): @staticmethod def create_success(value): + '''Create a succesful IncomingValueResult out of a value''' return IncomingValueResult(success=True, value=value, error_msg=None) @staticmethod def create_failure(error_msg): + '''Create a failing IncomingValueResult with a error_msg''' return IncomingValueResult(success=False, value=None, error_msg=error_msg) diff --git a/python_modules/dagster/docs/apidocs/decorators.rst b/python_modules/dagster/docs/apidocs/decorators.rst new file mode 100644 index 0000000000000..a9a76f45447fc --- /dev/null +++ b/python_modules/dagster/docs/apidocs/decorators.rst @@ -0,0 +1,13 @@ +Decorators +=========================== + +.. currentmodule:: dagster + +A more concise way to define solids. + +.. autofunction:: solid + +.. autofunction:: with_context + +.. autoclass:: MultipleResults + :members: diff --git a/python_modules/dagster/docs/apidocs/definitions.rst b/python_modules/dagster/docs/apidocs/definitions.rst new file mode 100644 index 0000000000000..8d9f1728d60db --- /dev/null +++ b/python_modules/dagster/docs/apidocs/definitions.rst @@ -0,0 +1,42 @@ +Definitions +=========================== + +.. currentmodule:: dagster + +Core API to define Solids and Pipelines. + +.. autoclass:: ConfigDefinition + :members: + +.. autoclass:: DependencyDefinition + :members: + +.. autoclass:: ExpectationDefinition + :members: + +.. autoclass:: ExpectationResult + :members: + +.. autoclass:: Field + :members: + +.. autoclass:: InputDefinition + :members: + +.. autoclass:: OutputDefinition + :members: + +.. autoclass:: PipelineContextDefinition + :members: + +.. autoclass:: PipelineDefinition + :members: + +.. autoclass:: RepositoryDefinition + :members: + +.. autoclass:: SolidDefinition + :members: + +.. autoclass:: Result + :members: diff --git a/python_modules/dagster/docs/apidocs/errors.rst b/python_modules/dagster/docs/apidocs/errors.rst new file mode 100644 index 0000000000000..4aa3d27ad7cab --- /dev/null +++ b/python_modules/dagster/docs/apidocs/errors.rst @@ -0,0 +1,16 @@ +Errors +========= + +.. currentmodule:: dagster + +Core dagster error classes. + +.. autoexception:: DagsterInvalidDefinitionError + +.. autoexception:: DagsterInvariantViolationError + +.. autoexception:: DagsterTypeError + +.. autoexception:: DagsterUserCodeExecutionError + +.. autoexception:: DagsterExpectationFailedError diff --git a/python_modules/dagster/docs/apidocs/execution.rst b/python_modules/dagster/docs/apidocs/execution.rst new file mode 100644 index 0000000000000..3c6f9e6330c44 --- /dev/null +++ b/python_modules/dagster/docs/apidocs/execution.rst @@ -0,0 +1,19 @@ +Execution +========= + +.. currentmodule:: dagster + +Executing pipelines and solids. + +.. autofunction:: execute_pipeline_iterator + +.. autofunction:: execute_pipeline + +.. autoclass:: ExecutionContext + :members: + +.. autoclass:: PipelineExecutionResult + :members: + +.. autoclass:: SolidExecutionResult + :members: diff --git a/python_modules/dagster/docs/apidocs/index.rst b/python_modules/dagster/docs/apidocs/index.rst index 015405875e38b..59235d31c8e8a 100644 --- a/python_modules/dagster/docs/apidocs/index.rst +++ b/python_modules/dagster/docs/apidocs/index.rst @@ -1,9 +1,15 @@ API Reference ============= -Top-level Dagster API ---------------------- +.. module:: dagster -.. automodule:: dagster - :members: - :undoc-members: +Top level dagster definitions + +.. toctree:: + :maxdepth: 1 + + ./definitions + ./decorators + ./execution + ./errors + ./types diff --git a/python_modules/dagster/docs/apidocs/types.rst b/python_modules/dagster/docs/apidocs/types.rst new file mode 100644 index 0000000000000..571975f94a0aa --- /dev/null +++ b/python_modules/dagster/docs/apidocs/types.rst @@ -0,0 +1,40 @@ +Types +========= + +.. module:: dagster.core.types + +Dagster type system. + +Type definitions +----------------- + +.. autoclass:: DagsterType + :members: + +.. autoclass:: DagsterScalarType + :members: process_value, is_python_valid_value + +.. autoclass:: DagsterCompositeType + +.. autoclass:: ConfigDictionary + +.. autoclass:: PythonObjectType + +.. autodata:: Any + +.. autodata:: String + +.. autodata:: Path + +.. autodata:: Int + +.. autodata:: Bool + +Utilities +------------ + +.. autoclass:: Field + :members: + +.. autoclass:: IncomingValueResult + :members: diff --git a/python_modules/dagster/docs/conf.py b/python_modules/dagster/docs/conf.py index 318956764edcb..428573eda2c42 100644 --- a/python_modules/dagster/docs/conf.py +++ b/python_modules/dagster/docs/conf.py @@ -156,3 +156,5 @@ # -- Extension configuration ------------------------------------------------- extensions = ['sphinx.ext.napoleon'] + +html_sidebars = {'**': ['globaltoc.html', 'relations.html', 'sourcelink.html', 'searchbox.html']} diff --git a/python_modules/dagster/docs/contributing.rst b/python_modules/dagster/docs/contributing.rst new file mode 100644 index 0000000000000..8f1bb2cef2b0c --- /dev/null +++ b/python_modules/dagster/docs/contributing.rst @@ -0,0 +1,68 @@ +Contributing +======================= + +Local development setup +~~~~~~~~~~~~~~~~~~~~~~~~~~ + +1. Create and activate virtualenv + +:: + + python3 -m venv dagsterenv + source dagsterenv/bin/activate + +2. Install dagster locally and install dev tools + +:: + + cd dagster/python_modules + pip install -e ./dagit + pip install -e ./dagster + pip install -r ./dagster/dev-requirements.txt + +3. Install dagit webapp dependencies + +:: + + cd python_modules/dagit/dagit/webapp + yarn + +4. Run tests + +:: + + cd python_modules/dagster + tox + cd python_modules/dagit + tox + cd python_modules/dagit/dagit/webapp + yarn test + +Running dagit webapp in development +------------------------------------- + +Run dagit on different port + +:: + + dagit -p 3333 + +Run local development version of webapp + +:: + + cd python_modules/dagit/dagit/webapp + REACT_APP_GRAPHQL_URI="http://localhost:3333/graphql" yarn start + +Releasing +----------- + +Dagster and dagit both have `./bin/publish.sh` scripts. + +Developing docs +--------------- + +:: + + cd python_modules/dagster/docs + make livehtml diff --git a/python_modules/dagster/docs/developing/index.rst b/python_modules/dagster/docs/developing/index.rst deleted file mode 100644 index e0baefaf1d2e3..0000000000000 --- a/python_modules/dagster/docs/developing/index.rst +++ /dev/null @@ -1,36 +0,0 @@ -Local development setup -======================= - -1. Create and activate virtualenv - -:: - - python3 -m venv dagsterenv - source dagsterenv/bin/activate - -2. Install dagster locally and install dev tools - -:: - - pip install -e ./dagster - pip install -r ./dagster/dev-requirements.txt - -3. Install pre-commit hooks - -:: - - pre-commit install - -4. Run tests - -:: - - tox - -Developing docs ---------------- - -:: - - cd docs - make livehtml diff --git a/python_modules/dagster/docs/index.rst b/python_modules/dagster/docs/index.rst index 31c65d6568add..ed73e7447b879 100644 --- a/python_modules/dagster/docs/index.rst +++ b/python_modules/dagster/docs/index.rst @@ -4,35 +4,30 @@ Welcome to Dagster, an opinionated programming model for data pipelines. -Documentation Structure: - .. toctree:: - :maxdepth: 3 + :maxdepth: 1 + :caption: Documentation + :name: Documentation motivations principles tutorials topic_guides - apidocs/index - -Other docs -========== - -.. toctree:: - :maxdepth: 3 - :caption: Contents: - - developing/index + contributing API Reference ================ .. toctree:: - :maxdepth: 2 - :caption: Contents: - - apidocs/index - + :maxdepth: 1 + :name: API Reference + :caption: API Reference + + apidocs/definitions + apidocs/decorators + apidocs/execution + apidocs/errors + apidocs/types Indices and tables ================== diff --git a/python_modules/dagster/docs/principles.rst b/python_modules/dagster/docs/principles.rst index ef13964d64a0a..3aea14b525517 100644 --- a/python_modules/dagster/docs/principles.rst +++ b/python_modules/dagster/docs/principles.rst @@ -32,10 +32,10 @@ about what libraries and engines use to do actual compute. Dagster pipelines can any python computation, which could be Pandas, Spark, or it could in turn invoke SQL or any other DSL or library deemed appropriate to the task. -Testable +Testable ^^^^^^^^ Testing data pipelines is notoriously difficult. Because it is so difficult it is often never -done, or done poorly. Dagster pipelines are designed to be tested. They have explicit support +done, or done poorly. Dagster pipelines are designed to be tested. They have explicit support for pipeline authors to manage and maintain multiple operating environments -- for example, unit testing, integration testing, and production environments, among others. In addition dagster can execute arbitrary subsets and nodes of the pipeline which is critical for testability. (This @@ -60,4 +60,4 @@ Gradual, optional typing Dagster contains a type system to describe the values flowing through the pipeline and the configuration of the pipeline. This allows nodes in a pipeline know if they are properly arranged and configuration prior to execution before execution, and serves as valuable -documentation and runtime error checking. +documentation and runtime error checking. From 80811b99d6055f279dc292631d9d34d53e66a5dd Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Fri, 7 Sep 2018 06:44:43 -0700 Subject: [PATCH 064/103] SolidDefinition docs improvement --- .../dagster/dagster/core/definitions.py | 54 +++++++++++++++---- 1 file changed, 44 insertions(+), 10 deletions(-) diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index 8404e94ddd150..816305ec723d2 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -773,18 +773,52 @@ def __init__(self, config_type=types.Any): class SolidDefinition(object): - '''A solid is a node of computation within a pipeline. + '''A solid (a name extracted from the acronym of "software-structured data" (SSD)) represents + a unit of computation within a data pipeline. + + As its core, a solid is a function. It accepts inputs (which are values produced from + other solids) and configuration, and produces outputs. These solids are composed as a + directed, acyclic graph (DAG) within a pipeline to form a computation that produces + data assets. + + Solids should be implemented as idempotent, parameterizable, non-destructive functions. + Data computations with these properties are much easier to test, reason about, and operate. + + The inputs and outputs are gradually, optionally typed by the dagster type system. Types + can be user-defined and can represent entites as varied as scalars, dataframe, database + tables, and so forth. They can represent pure in-memory objects, or handles to assets + on disk or in external resources. + + A solid is a generalized abstraction that could take many forms. + + Example: + + .. code-block:: python + + def _read_csv(context, inputs, config_dict): + yield Result(pandas.read_csv(config_dict['path'])) + + SolidDefinition( + name='read_csv', + inputs=[], + config_def=ConfigDefinition(types.ConfigDictionary({'path' => types.Path})), + outputs=[OutputDefinition()] # default name ('result') and any typed + transform_fn + ) Attributes: - name: str - inputs: list of InputDefinitions - transform_fn: callable with sig ( - context: ExecutionContext, - inputs: str => Any, - config_value: Any) : Iterable - outputs: list of OutputDefinitions - config_def: ConfigDefinition - description: str + name (str): Name of the solid. + inputs (List[InputDefiniton]): Inputs of the solid. + transform_fn (callable): + Callable with the signature + ( + context: ExecutionContext, + inputs: Dict[str, Any], + conf: Any + ) : Iterable + outputs (List[OutputDefinition]): Outputs of the solid. + config_def (ConfigDefinition): How the solid configured. + description (str): Description of the solid. ''' def __init__(self, name, inputs, transform_fn, outputs, config_def=None, description=None): From 224e381b0ab7af959b950a027a7f12d1b00949b6 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Fri, 7 Sep 2018 06:55:27 -0700 Subject: [PATCH 065/103] Document single_output_transform --- .../dagster/dagster/core/definitions.py | 27 ++++++++++++++++++- 1 file changed, 26 insertions(+), 1 deletion(-) diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index 816305ec723d2..ae9f8bbfc64ad 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -852,7 +852,32 @@ def __init__(self, name, inputs, transform_fn, outputs, config_def=None, descrip def single_output_transform(name, inputs, transform_fn, output, description=None): '''It is commmon to want a Solid that has only inputs, a single output (with the default name), and no config. So this is a helper function to do that. This transform function - must return the naked return value (as opposed to a Result object)''' + must return the naked return value (as opposed to a Result object). + + Args: + name (str): Name of the solid. + inputs (List[InputDefinition]): Inputs of solid. + transform_fn (callable): + Callable with the signature + (context: ExecutionContext, inputs: Dict[str, Any]) : Any + output (OutputDefinition): Output of the solid. + description (str): Descripion of the solid. + + Returns: + SolidDefinition: + + Example: + + .. code-block:: python + + SolidDefinition.single_output_transform( + 'add_one', + inputs=InputDefinition('num', types.Int), + output=OutputDefinition(types.Int), + transform_fn=lambda context, inputs: inputs['num'] + 1 + ) + + ''' def _new_transform_fn(context, inputs, _config_dict): value = transform_fn(context, inputs) From 1f68d8d244375a7cf11f5c1e64958eb9c60410f1 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Fri, 7 Sep 2018 07:04:31 -0700 Subject: [PATCH 066/103] Clean up decorator docs --- .../dagster/dagster/core/decorators.py | 22 ++++++++++++------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/python_modules/dagster/dagster/core/decorators.py b/python_modules/dagster/dagster/core/decorators.py index 5e1f253cdce79..5c75f2a5c72c4 100644 --- a/python_modules/dagster/dagster/core/decorators.py +++ b/python_modules/dagster/dagster/core/decorators.py @@ -1,10 +1,6 @@ -import inspect -if hasattr(inspect, 'signature'): - funcsigs = inspect -else: - import funcsigs from collections import namedtuple from functools import wraps +import inspect from .definitions import ( ConfigDefinition, @@ -18,6 +14,11 @@ from .types import Any +if hasattr(inspect, 'signature'): + funcsigs = inspect +else: + import funcsigs + # Error messages are long # pylint: disable=C0301 @@ -139,13 +140,18 @@ def __call__(self, fn): def solid(name=None, inputs=None, output=None, outputs=None, description=None): '''(decorator) Create a solid with specified parameters. - This shortcut simplifies core solid API by exploding arguments into kwargs of the transform function and omitting additional parameters when they are not needed. Parameters are otherwise as per :py:class:`SolidDefinition`. By using :py:function:`with_context` one can request context object to be passed too. + This shortcut simplifies core solid API by exploding arguments into kwargs of the + transform function and omitting additional parameters when they are not needed. + Parameters are otherwise as per :py:class:`SolidDefinition`. By using + :py:function:`with_context` one can request context object to be passed too. - Decorated function becomes transform. Instead of having to yield result objects, transform support multiple simpler output types. + Decorated function is the transform function itself. Instead of having to yield + result objects, transform support multiple simpler output types. 1. Return a value. This is returned as a :py:class:`Result` for a single output solid. 2. Return a :py:class:`Result`. Works like yielding result. - 3. Return a :py:class:`MultipleResults`. Works like yielding several results for multiple outputs. Useful for solids that have multiple outputs. + 3. Return a :py:class:`MultipleResults`. Works like yielding several results for + multiple outputs. Useful for solids that have multiple outputs. 4. Yield :py:class:`Result`. Same as default transform behaviour. Examples: From 9e1baf3ccbdb3dbd595789dfc54615580fd9877e Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Fri, 7 Sep 2018 09:28:16 -0700 Subject: [PATCH 067/103] Change @solid to unconditionally accept context and conf Eliminate with_context. Enforce that @solid accepts both a context and a conf. If we need to reintroduce information hiding we will. --- .../dagster_ge_tests/test_pandas_ge.py | 6 +- python_modules/dagster/dagster/__init__.py | 10 +- python_modules/dagster/dagster/config.py | 19 ++- .../dagster/dagster/core/compute_nodes.py | 10 +- .../core/core_tests/test_compute_nodes.py | 2 +- .../core/core_tests/test_custom_context.py | 24 ++-- .../core/core_tests/test_decorators.py | 117 +++++++++++------- .../core/core_tests/test_naming_collisions.py | 4 +- .../core/core_tests/test_solid_with_config.py | 16 +-- .../dagster/dagster/core/decorators.py | 105 +++++++--------- .../dagster/dagster/core/definitions.py | 1 + .../dagster/dagster/core/execution.py | 6 +- .../dagster/dagster/core/test_utils.py | 2 + .../test_sql_project_pipeline.py | 3 +- .../pandas_hello_world/env.yml | 2 +- .../pandas_hello_world/pipeline.py | 6 +- .../test_pandas_hello_world_library_slide.py | 2 +- .../pandas/pandas_tests/test_pandas_solids.py | 6 +- .../pandas_tests/test_pandas_user_error.py | 4 +- .../test_isolated_templated_sql_tests.py | 7 +- .../dagster/dagster_tests/test_config.py | 2 +- .../dagster/docs/apidocs/decorators.rst | 2 - 22 files changed, 181 insertions(+), 175 deletions(-) diff --git a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py index 2ede5fc2f0172..52db733479761 100644 --- a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py +++ b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py @@ -37,7 +37,7 @@ def col_exists(name, col_name): ], output=OutputDefinition(dagster_type=dagster_pd.DataFrame) ) -def sum_solid(num_df): +def sum_solid(_context, _conf, num_df): return _sum_solid_impl(num_df) @@ -51,7 +51,7 @@ def sum_solid(num_df): ], output=OutputDefinition(dagster_type=dagster_pd.DataFrame) ) -def sum_solid_fails_input_expectation(num_df): +def sum_solid_fails_input_expectation(_context, _conf, num_df): return _sum_solid_impl(num_df) @@ -69,7 +69,7 @@ def sum_solid_fails_input_expectation(num_df): ], output=OutputDefinition(dagster_type=dagster_pd.DataFrame) ) -def sum_solid_expectations_config(num_df): +def sum_solid_expectations_config(_context, _conf, num_df): return _sum_solid_impl(num_df) diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index a24357e00d4ae..61eedc589bcd8 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -1,7 +1,13 @@ from __future__ import (absolute_import, division, print_function, unicode_literals) from builtins import * # pylint: disable=W0622,W0401 -from dagster.core.execution import execute_pipeline_iterator, execute_pipeline, PipelineExecutionResult, SolidExecutionResult +from dagster.core.execution import ( + PipelineExecutionResult, + SolidExecutionResult, + execute_pipeline, + execute_pipeline_iterator, +) + from dagster.core.execution_context import ExecutionContext from dagster.core.definitions import ( @@ -22,7 +28,6 @@ from dagster.core.decorators import ( MultipleResults, solid, - with_context, ) from dagster.core.errors import ( @@ -51,7 +56,6 @@ # Decorators 'solid', - 'with_context', 'MultipleResults', # Execution diff --git a/python_modules/dagster/dagster/config.py b/python_modules/dagster/dagster/config.py index 2fd59b968cb82..4ceda5b8ec267 100644 --- a/python_modules/dagster/dagster/config.py +++ b/python_modules/dagster/dagster/config.py @@ -4,21 +4,18 @@ # lifted from https://bit.ly/2HcQAuv -class Context(namedtuple('ContextData', 'name args')): - def __new__(cls, name, args): +class Context(namedtuple('ContextData', 'name config')): + def __new__(cls, name, config=None): return super(Context, cls).__new__( cls, check.str_param(name, 'name'), - check.dict_param(args, 'args', key_type=str), + config, ) -class Solid(namedtuple('Solid', 'config_dict')): - def __new__(cls, config_dict): - return super(Solid, cls).__new__( - cls, - check.dict_param(config_dict, 'config_dict', key_type=str), - ) +class Solid(namedtuple('Solid', 'config')): + def __new__(cls, config): + return super(Solid, cls).__new__(cls, config) class Environment(namedtuple('EnvironmentData', 'context solids expectations')): @@ -26,7 +23,7 @@ def __new__(cls, solids=None, context=None, expectations=None): check.opt_inst_param(context, 'context', Context) if context is None: - context = Context(name='default', args={}) + context = Context(name='default', config={}) if expectations is None: expectations = Expectations(evaluate=True) @@ -54,7 +51,7 @@ def __new__(cls, evaluate): def _construct_context(yml_config_object): context_obj = check.opt_dict_elem(yml_config_object, 'context') if context_obj: - return Context(check.str_elem(context_obj, 'name'), check.dict_elem(context_obj, 'args')) + return Context(check.str_elem(context_obj, 'name'), context_obj['config']) else: return None diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index e774dad46f719..c8e3c20744741 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -206,7 +206,7 @@ def _yield_transform_results(context, compute_node, values_dict, config_dict): ) yield result -def _execute_core_transform(context, compute_node, values_dict, config_dict): +def _execute_core_transform(context, compute_node, values_dict, conf): ''' Execute the user-specified transform for the solid. Wrap in an error boundary and do all relevant logging and metrics tracking @@ -214,7 +214,6 @@ def _execute_core_transform(context, compute_node, values_dict, config_dict): check.inst_param(context, 'context', ExecutionContext) check.inst_param(compute_node, 'compute_node', ComputeNode) check.dict_param(values_dict, 'values_dict', key_type=str) - check.dict_param(config_dict, 'config_dict', key_type=str) error_str = 'Error occured during core transform' @@ -228,7 +227,7 @@ def _execute_core_transform(context, compute_node, values_dict, config_dict): time_execution_scope() as timer_result, \ context.value('solid', compute_node.solid.name): - for result in _yield_transform_results(context, compute_node, values_dict, config_dict): + for result in _yield_transform_results(context, compute_node, values_dict, conf): yield result context.metric('core_transform_time_ms', timer_result.millis) @@ -559,9 +558,9 @@ def validate_config_dict(execution_info, solid): name = solid.name solid_configs = execution_info.environment.solids - config_dict = solid_configs[name].config_dict if name in solid_configs else {} + conf = solid_configs[name].config if name in solid_configs else {} - evaluation_result = solid.config_def.config_type.evaluate_value(config_dict) + evaluation_result = solid.config_def.config_type.evaluate_value(conf) if evaluation_result.success: return evaluation_result.value else: @@ -762,7 +761,6 @@ def _do_expectation(context, _compute_node, inputs): def create_compute_node_from_solid_transform(solid, node_inputs, config_args): check.inst_param(solid, 'solid', SolidDefinition) check.list_param(node_inputs, 'node_inputs', of_type=ComputeNodeInput) - check.dict_param(config_args, 'config_args', key_type=str) return ComputeNode( friendly_name='{solid.name}.transform'.format(solid=solid), diff --git a/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py b/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py index a83aa265cb5f4..5a29e71ed2a22 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py +++ b/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py @@ -26,7 +26,7 @@ def silencing_default_context(): @solid(name='noop', inputs=[], outputs=[OutputDefinition()]) -def noop_solid(): +def noop_solid(_context, _conf): return 'foo' diff --git a/python_modules/dagster/dagster/core/core_tests/test_custom_context.py b/python_modules/dagster/dagster/core/core_tests/test_custom_context.py index 16f9f96c5babb..2b4b6a421b37d 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_custom_context.py +++ b/python_modules/dagster/dagster/core/core_tests/test_custom_context.py @@ -11,7 +11,6 @@ execute_pipeline, solid, types, - with_context, ) from dagster.core.errors import (DagsterTypeError, DagsterInvariantViolationError) from dagster.utils.logging import (INFO, ERROR) @@ -25,8 +24,7 @@ def test_default_context(): inputs=[], outputs=[OutputDefinition()], ) - @with_context - def default_context_transform(context): + def default_context_transform(context, _conf): for logger in context._logger.loggers: assert logger.level == ERROR @@ -41,8 +39,7 @@ def test_default_context_with_log_level(): inputs=[], outputs=[OutputDefinition()], ) - @with_context - def default_context_transform(context): + def default_context_transform(context, _conf): for logger in context._logger.loggers: assert logger.level == INFO @@ -65,8 +62,7 @@ def _get_config_test_solid(config_key, config_value): inputs=[], outputs=[OutputDefinition()], ) - @with_context - def config_test(context): + def config_test(context, _conf): assert context.resources == {config_key: config_value} return config_test @@ -101,8 +97,7 @@ def test_custom_contexts(): inputs=[], outputs=[OutputDefinition()], ) - @with_context - def custom_context_transform(context): + def custom_context_transform(context, _conf): assert context.resources == {'field_one': 'value_two'} pipeline = PipelineDefinition( @@ -149,8 +144,7 @@ def test_yield_context(): inputs=[], outputs=[OutputDefinition()], ) - @with_context - def custom_context_transform(context): + def custom_context_transform(context, _conf): assert context.resources == {'field_one': 'value_two'} assert context._context_dict['foo'] == 'bar' # pylint: disable=W0212 events.append('during') @@ -223,11 +217,9 @@ def never_transform(): context_definitions={ 'default': PipelineContextDefinition( - config_def=ConfigDefinition.config_dict( - { - 'string_field': Field(types.String) - } - ), + config_def=ConfigDefinition.config_dict({ + 'string_field': Field(types.String) + }), context_fn=lambda _pipeline, _config_value: _config_value ) } diff --git a/python_modules/dagster/dagster/core/core_tests/test_decorators.py b/python_modules/dagster/dagster/core/core_tests/test_decorators.py index 75105c9ad9ad2..4d9e940528e24 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_decorators.py +++ b/python_modules/dagster/dagster/core/core_tests/test_decorators.py @@ -1,6 +1,7 @@ import pytest import dagster from dagster import ( + ConfigDefinition, DagsterInvalidDefinitionError, DependencyDefinition, ExecutionContext, @@ -12,7 +13,6 @@ config, execute_pipeline, solid, - with_context, ) from dagster.core.test_utils import execute_single_solid @@ -32,9 +32,41 @@ def create_empty_test_env(): return config.Environment() +def test_no_parens_solid(): + called = {} + + @solid + def hello_world(_context, _conf): + called['yup'] = True + + result = execute_single_solid( + create_test_context(), + hello_world, + environment=create_empty_test_env(), + ) + + assert called['yup'] + + +def test_empty_solid(): + called = {} + + @solid() + def hello_world(_context, _conf): + called['yup'] = True + + result = execute_single_solid( + create_test_context(), + hello_world, + environment=create_empty_test_env(), + ) + + assert called['yup'] + + def test_solid(): @solid(outputs=[OutputDefinition()]) - def hello_world(): + def hello_world(_context, _conf): return {'foo': 'bar'} result = execute_single_solid( @@ -50,7 +82,7 @@ def hello_world(): def test_solid_one_output(): @solid(output=OutputDefinition()) - def hello_world(): + def hello_world(_context, _conf): return {'foo': 'bar'} result = execute_single_solid( @@ -66,7 +98,7 @@ def hello_world(): def test_solid_yield(): @solid(output=OutputDefinition()) - def hello_world(): + def hello_world(_context, _conf): yield Result(value={'foo': 'bar'}) result = execute_single_solid( @@ -82,7 +114,7 @@ def hello_world(): def test_solid_result_return(): @solid(output=OutputDefinition()) - def hello_world(): + def hello_world(_context, _conf): return Result(value={'foo': 'bar'}) result = execute_single_solid( @@ -101,7 +133,7 @@ def test_solid_multiple_outputs(): OutputDefinition(name="left"), OutputDefinition(name="right"), ]) - def hello_world(): + def hello_world(_context, _conf): return MultipleResults( Result(value={'foo': 'left'}, output_name='left'), Result(value={'foo': 'right'}, output_name='right') @@ -125,7 +157,7 @@ def test_dict_multiple_outputs(): OutputDefinition(name="left"), OutputDefinition(name="right"), ]) - def hello_world(): + def hello_world(_context, _conf): return MultipleResults.from_dict({ 'left': { 'foo': 'left' @@ -150,24 +182,7 @@ def hello_world(): def test_solid_with_name(): @solid(name="foobar", outputs=[OutputDefinition()]) - def hello_world(): - return {'foo': 'bar'} - - result = execute_single_solid( - create_test_context(), - hello_world, - environment=create_empty_test_env(), - ) - - assert result.success - assert len(result.result_list) == 1 - assert result.result_list[0].transformed_value()['foo'] == 'bar' - - -def test_solid_with_context(): - @solid(name="foobar", outputs=[OutputDefinition()]) - @with_context - def hello_world(_context): + def hello_world(_context, _conf): return {'foo': 'bar'} result = execute_single_solid( @@ -183,7 +198,7 @@ def hello_world(_context): def test_solid_with_input(): @solid(inputs=[InputDefinition(name="foo_to_foo")], outputs=[OutputDefinition()]) - def hello_world(foo_to_foo): + def hello_world(_context, _conf, foo_to_foo): return foo_to_foo pipeline = PipelineDefinition( @@ -208,14 +223,13 @@ def test_solid_definition_errors(): with pytest.raises(DagsterInvalidDefinitionError): @solid(inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()]) - @with_context - def vargs(_context, foo, *args): + def vargs(_context, _conf, foo, *args): pass with pytest.raises(DagsterInvalidDefinitionError): @solid(inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()]) - def wrong_name(bar): + def wrong_name(_context, _conf, bar): pass with pytest.raises(DagsterInvalidDefinitionError): @@ -225,42 +239,39 @@ def wrong_name(bar): InputDefinition(name="bar")], outputs=[OutputDefinition()] ) - def wrong_name_2(foo): + def wrong_name_2(_context, _conf, foo): pass with pytest.raises(DagsterInvalidDefinitionError): @solid(inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()]) - @with_context def no_context(foo): pass with pytest.raises(DagsterInvalidDefinitionError): @solid(inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()]) - def yes_context(_context, foo): + def no_conf(_context, foo): pass with pytest.raises(DagsterInvalidDefinitionError): @solid(inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()]) - def extras(foo, bar): + def yes_context(_context, foo): pass - @solid( - inputs=[InputDefinition(name="foo"), - InputDefinition(name="bar")], - outputs=[OutputDefinition()] - ) - def valid_kwargs(**kwargs): - pass + with pytest.raises(DagsterInvalidDefinitionError): + + @solid(inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()]) + def extras(_context, _conf, foo, bar): + pass @solid( inputs=[InputDefinition(name="foo"), InputDefinition(name="bar")], outputs=[OutputDefinition()] ) - def valid(foo, bar): + def valid_kwargs(_context, _conf, **kwargs): pass @solid( @@ -268,8 +279,7 @@ def valid(foo, bar): InputDefinition(name="bar")], outputs=[OutputDefinition()] ) - @with_context - def valid_rontext(context, foo, bar): + def valid(_context, _conf, foo, bar): pass @solid( @@ -277,8 +287,7 @@ def valid_rontext(context, foo, bar): InputDefinition(name="bar")], outputs=[OutputDefinition()] ) - @with_context - def valid_context_2(_context, foo, bar): + def valid_rontext(context, _conf, foo, bar): pass @@ -303,3 +312,21 @@ def solid_desc(): pass assert solid_desc.description == 'foo' + + +def test_any_config_definition(): + called = {} + conf_value = 234 + + @solid(config_def=ConfigDefinition()) + def hello_world(_context, conf): + assert conf == conf_value + called['yup'] = True + + result = execute_single_solid( + create_test_context(), + hello_world, + environment=config.Environment(solids={'hello_world': config.Solid(conf_value)}) + ) + + assert called['yup'] diff --git a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py index 8cbd72d70e558..372b228363207 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py +++ b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py @@ -52,7 +52,9 @@ def test_execute_solid_with_input_same_name(): result = execute_pipeline( pipeline, - config.Environment(solids={'pass_value': config.Solid(config_dict={'value': 'foo'})}), + config.Environment(solids={'pass_value': config.Solid({ + 'value': 'foo' + })}), ) assert result.result_for_solid('a_thing').transformed_value() == 'foofoo' diff --git a/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py b/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py index a2f743bfa74f4..299ef39cf7e18 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py +++ b/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py @@ -20,7 +20,7 @@ def _t_fn(_context, _inputs, config_dict): did_get['yep'] = config_dict solid = SolidDefinition( - name='with_context', + name='solid_with_context', inputs=[], outputs=[], config_def=ConfigDefinition.config_dict({ @@ -33,9 +33,9 @@ def _t_fn(_context, _inputs, config_dict): execute_pipeline( pipeline, - config.Environment( - solids={'with_context': config.Solid(config_dict={'some_config': 'foo'})} - ), + config.Environment(solids={'solid_with_context': config.Solid({ + 'some_config': 'foo' + })}), ) assert 'yep' in did_get @@ -47,7 +47,7 @@ def _t_fn(_context, _inputs, _config_dict): raise Exception('should not reach') solid = SolidDefinition( - name='with_context', + name='solid_with_context', inputs=[], outputs=[], config_def=ConfigDefinition.config_dict({ @@ -61,7 +61,7 @@ def _t_fn(_context, _inputs, _config_dict): with pytest.raises(DagsterTypeError): execute_pipeline( pipeline, - config.Environment( - solids={'with_context': config.Solid(config_dict={'some_config': 1})} - ), + config.Environment(solids={'solid_with_context': config.Solid({ + 'some_config': 1 + })}), ) diff --git a/python_modules/dagster/dagster/core/decorators.py b/python_modules/dagster/dagster/core/decorators.py index 5c75f2a5c72c4..16a2454e907d2 100644 --- a/python_modules/dagster/dagster/core/decorators.py +++ b/python_modules/dagster/dagster/core/decorators.py @@ -72,30 +72,6 @@ def from_dict(result_dict): return MultipleResults(*results) -def with_context(fn): - '''(decorator) Pass context as a first argument to a transform. - - Example: - - .. code-block:: python - - @solid() - @with_context - def my_solid(context): - pass - ''' - return _WithContext(fn) - - -class _WithContext(object): - def __init__(self, fn): - self.fn = fn - - @property - def has_context(self): - return True - - class _Solid(object): def __init__( self, @@ -115,18 +91,16 @@ def __init__( self.outputs = check.opt_list_param(outputs, 'outputs', OutputDefinition) self.description = check.opt_str_param(description, 'description') - self.config_def = check.opt_inst_param(config_def, 'config_def', ConfigDefinition(Any)) + self.config_def = check.opt_inst_param(config_def, 'config_def', ConfigDefinition) def __call__(self, fn): - expect_context = getattr(fn, 'has_context', False) - if expect_context: - fn = fn.fn + check.callable_param(fn, 'fn') if not self.name: self.name = fn.__name__ - _validate_transform_fn(self.name, fn, self.input_defs, expect_context) - transform_fn = _create_transform_wrapper(fn, self.input_defs, self.outputs, expect_context) + _validate_transform_fn(self.name, fn, self.input_defs) + transform_fn = _create_transform_wrapper(fn, self.input_defs, self.outputs) return SolidDefinition( name=self.name, inputs=self.input_defs, @@ -137,13 +111,12 @@ def __call__(self, fn): ) -def solid(name=None, inputs=None, output=None, outputs=None, description=None): +def solid(name=None, inputs=None, output=None, outputs=None, config_def=None, description=None): '''(decorator) Create a solid with specified parameters. This shortcut simplifies core solid API by exploding arguments into kwargs of the transform function and omitting additional parameters when they are not needed. - Parameters are otherwise as per :py:class:`SolidDefinition`. By using - :py:function:`with_context` one can request context object to be passed too. + Parameters are otherwise as per :py:class:`SolidDefinition`. Decorated function is the transform function itself. Instead of having to yield result objects, transform support multiple simpler output types. @@ -158,23 +131,31 @@ def solid(name=None, inputs=None, output=None, outputs=None, description=None): .. code-block:: python + @solid + def hello_world(context, conf): + print('hello') + + @solid() + def hello_world(context, conf): + print('hello') + @solid(outputs=[OutputDefinition()]) - def hello_world(): + def hello_world(context, conf): return {'foo': 'bar'} @solid(output=OutputDefinition()) - def hello_world(): + def hello_world(context, conf): return Result(value={'foo': 'bar'}) @solid(output=OutputDefinition()) - def hello_world(): + def hello_world(context, conf): yield Result(value={'foo': 'bar'}) @solid(outputs=[ OutputDefinition(name="left"), OutputDefinition(name="right"), ]) - def hello_world(): + def hello_world(context, conf): return MultipleResults.from_dict({ 'left': {'foo': 'left'}, 'right': {'foo': 'right'}, @@ -184,33 +165,41 @@ def hello_world(): inputs=[InputDefinition(name="foo_to_foo")], outputs=[OutputDefinition()] ) - def hello_world(foo_to_foo): + def hello_world(context, conf, foo_to_foo): return foo_to_foo - @solid( - inputs=[InputDefinition(name="foo_to_foo")], - outputs=[OutputDefinition()] - ) - @with_context - def hello_world(context, foo_to_foo): - return foo_to_foo ''' - return _Solid(name=name, inputs=inputs, output=output, outputs=outputs, description=description) - - -def _create_transform_wrapper(fn, inputs, outputs, include_context=False): + if callable(name): + check.invariant(inputs is None) + check.invariant(output is None) + check.invariant(outputs is None) + check.invariant(description is None) + check.invariant(config_def is None) + return _Solid()(name) + + return _Solid( + name=name, + inputs=inputs, + output=output, + outputs=outputs, + config_def=config_def, + description=description, + ) + + +# TODO change names and check types +def _create_transform_wrapper(fn, inputs, outputs): + check.callable_param(fn, 'fn') input_names = [input.name for input in inputs] @wraps(fn) - def transform(context, args, _config): + def transform(context, args, conf): kwargs = {} for input_name in input_names: kwargs[input_name] = args[input_name] - if include_context: - result = fn(context, **kwargs) - else: - result = fn(**kwargs) + result = fn(context, conf, **kwargs) + if inspect.isgenerator(result): for item in result: yield item @@ -244,12 +233,10 @@ def __init__(self, error_type, param=None, missing_names=None, **kwargs): self.missing_names = missing_names -def _validate_transform_fn(solid_name, transform_fn, inputs, expect_context=False): +def _validate_transform_fn(solid_name, transform_fn, inputs): names = set(inp.name for inp in inputs) - if expect_context: - expected_positionals = ('context', ) - else: - expected_positionals = () + # Currently being super strict about naming. Might be a good idea to relax. Starting strict. + expected_positionals = ('context', 'conf') try: _validate_decorated_fn(transform_fn, names, expected_positionals) except FunctionValidationError as e: diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index ae9f8bbfc64ad..83fcda8c75947 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -30,6 +30,7 @@ DISALLOWED_NAMES = set( [ 'context', + 'conf', 'meta', 'arg_dict', 'dict', diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index 60b102403e770..e02e5888ff217 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -227,17 +227,19 @@ def yield_context(pipeline, environment): context_definition = pipeline.context_definitions[context_name] config_type = context_definition.config_def.config_type - evaluation_result = config_type.evaluate_value(environment.context.args) + evaluation_result = config_type.evaluate_value(environment.context.config) if not evaluation_result.success: raise DagsterTypeError( 'Invalid config value: {error_msg}'.format(error_msg=evaluation_result.error_msg) ) + thing = context_definition.context_fn(pipeline, evaluation_result.value) return _wrap_in_yield(thing) def execute_pipeline_iterator(pipeline, environment): - '''Returns iterator that yields :py:class:`SolidExecutionResult` for each solid executed in the pipeline + '''Returns iterator that yields :py:class:`SolidExecutionResult` for each + solid executed in the pipeline. Parameters: pipeline (PipelineDefinition): pipeline to run diff --git a/python_modules/dagster/dagster/core/test_utils.py b/python_modules/dagster/dagster/core/test_utils.py index 3d51ad6d1cad3..5752596b0e92f 100644 --- a/python_modules/dagster/dagster/core/test_utils.py +++ b/python_modules/dagster/dagster/core/test_utils.py @@ -18,6 +18,8 @@ def execute_single_solid(context, solid, environment, throw_on_error=True): single_solid_environment = config.Environment( expectations=environment.expectations, context=environment.context, + solids={solid.name: environment.solids[solid.name]} + if solid.name in environment.solids else None ) pipeline_result = execute_pipeline( diff --git a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py index 000a3dcd79782..c299a2e571dbe 100644 --- a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py +++ b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py @@ -149,7 +149,8 @@ def test_full_in_memory_pipeline(): def test_full_persisted_pipeline(): pipeline = create_full_pipeline() pipeline_result = execute_pipeline( - pipeline, environment=config.Environment(context=config.Context(name='persisted', args={})) + pipeline, + environment=config.Environment(context=config.Context(name='persisted')), ) assert pipeline_result.success diff --git a/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/env.yml b/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/env.yml index b5cdc38072abf..5644f5042f754 100644 --- a/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/env.yml +++ b/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/env.yml @@ -1,6 +1,6 @@ context: name: default - args: + config: log_level: ERROR solids: diff --git a/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py b/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py index 854ad62ac2092..f13bac131bf7c 100644 --- a/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py +++ b/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py @@ -13,7 +13,7 @@ inputs=[InputDefinition('num', dagster_pd.DataFrame)], outputs=[OutputDefinition(dagster_type=dagster_pd.DataFrame)] ) -def sum_solid(num): +def sum_solid(_context, _conf, num): sum_df = num.copy() sum_df['sum'] = sum_df['num1'] + sum_df['num2'] return sum_df @@ -23,7 +23,7 @@ def sum_solid(num): inputs=[InputDefinition('sum_df', dagster_pd.DataFrame)], outputs=[OutputDefinition(dagster_type=dagster_pd.DataFrame)] ) -def sum_sq_solid(sum_df): +def sum_sq_solid(_context, _conf, sum_df): sum_sq_df = sum_df.copy() sum_sq_df['sum_sq'] = sum_df['sum']**2 return sum_sq_df @@ -33,7 +33,7 @@ def sum_sq_solid(sum_df): inputs=[InputDefinition('sum_sq_solid', dagster_pd.DataFrame)], outputs=[OutputDefinition(dagster_type=dagster_pd.DataFrame)] ) -def always_fails_solid(**_kwargs): +def always_fails_solid(_context, _conf, **_kwargs): raise Exception('I am a programmer and I make error') diff --git a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_library_slide.py b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_library_slide.py index dccd24ab14f9e..941baf11c0719 100644 --- a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_library_slide.py +++ b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_library_slide.py @@ -126,7 +126,7 @@ def create_decorator_based_solid(): inputs=[InputDefinition('num_csv', dagster_pd.DataFrame)], outputs=[OutputDefinition(dagster_type=dagster_pd.DataFrame)], ) - def hello_world(num_csv): + def hello_world(_context, _conf, num_csv): num_csv['sum'] = num_csv['num1'] + num_csv['num2'] return num_csv diff --git a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py index 1f8c6771f6cea..96a1923bc1b8a 100644 --- a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py +++ b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py @@ -180,7 +180,7 @@ def transform(_context, args): inputs=[InputDefinition('num_csv', dagster_pd.DataFrame)], outputs=[OutputDefinition(dagster_type=dagster_pd.DataFrame)], ) -def sum_table(num_csv): +def sum_table(_context, _conf, num_csv): check.inst_param(num_csv, 'num_csv', pd.DataFrame) num_csv['sum'] = num_csv['num1'] + num_csv['num2'] return num_csv @@ -190,7 +190,7 @@ def sum_table(num_csv): inputs=[InputDefinition('sum_df', dagster_pd.DataFrame)], outputs=[OutputDefinition(dagster_type=dagster_pd.DataFrame)], ) -def sum_sq_table(sum_df): +def sum_sq_table(_context, _conf, sum_df): sum_df['sum_squared'] = sum_df['sum'] * sum_df['sum'] return sum_df @@ -199,7 +199,7 @@ def sum_sq_table(sum_df): inputs=[InputDefinition('sum_table_renamed', dagster_pd.DataFrame)], outputs=[OutputDefinition(dagster_type=dagster_pd.DataFrame)], ) -def sum_sq_table_renamed_input(sum_table_renamed): +def sum_sq_table_renamed_input(_context, _conf, sum_table_renamed): sum_table_renamed['sum_squared'] = sum_table_renamed['sum'] * sum_table_renamed['sum'] return sum_table_renamed diff --git a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_user_error.py b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_user_error.py index afd1cee9b8639..4783d49b7697f 100644 --- a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_user_error.py +++ b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_user_error.py @@ -36,7 +36,7 @@ def test_wrong_output_value(): inputs=[csv_input], outputs=[OutputDefinition(dagster_type=dagster_pd.DataFrame)] ) - def df_solid(num_csv): + def df_solid(_context, _conf, num_csv): return 'not a dataframe' pass_solid = define_stub_solid('pass_solid', pd.DataFrame()) @@ -61,7 +61,7 @@ def test_wrong_input_value(): inputs=[InputDefinition('foo', dagster_pd.DataFrame)], outputs=[OutputDefinition()], ) - def df_solid(foo): + def df_solid(_context, _conf, foo): return foo pass_solid = define_stub_solid('pass_solid', 'not a dataframe') diff --git a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_templated_sql_tests.py b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_templated_sql_tests.py index cfec8ecb52326..2d649f769b7f0 100644 --- a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_templated_sql_tests.py +++ b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_templated_sql_tests.py @@ -23,11 +23,6 @@ def _load_table(context, table_name): ).fetchall() -def table_name_source(table_name): - check.str_param(table_name, 'table_name') - return config.Source('TABLENAME', args={'name': table_name}) - - def pipeline_test_def(solids, context, dependencies=None): return PipelineDefinition( solids=solids, @@ -390,6 +385,6 @@ def create_multi_input_pipeline(): def test_jinja(): templated_sql = '''SELECT * FROM {{some_table.name}}''' - sql = _render_template_string(templated_sql, config_dict={'some_table': {'name': 'fill_me_in'}}) + sql = _render_template_string(templated_sql, {'some_table': {'name': 'fill_me_in'}}) assert sql == '''SELECT * FROM fill_me_in''' diff --git a/python_modules/dagster/dagster_tests/test_config.py b/python_modules/dagster/dagster_tests/test_config.py index c0fe57d596ecf..07839412dc832 100644 --- a/python_modules/dagster/dagster_tests/test_config.py +++ b/python_modules/dagster/dagster_tests/test_config.py @@ -7,7 +7,7 @@ def test_construct_full_environment(): document = ''' context: name: default - args: + config: context_arg: context_value ''' diff --git a/python_modules/dagster/docs/apidocs/decorators.rst b/python_modules/dagster/docs/apidocs/decorators.rst index a9a76f45447fc..e64909fa71d68 100644 --- a/python_modules/dagster/docs/apidocs/decorators.rst +++ b/python_modules/dagster/docs/apidocs/decorators.rst @@ -7,7 +7,5 @@ A more concise way to define solids. .. autofunction:: solid -.. autofunction:: with_context - .. autoclass:: MultipleResults :members: From edc56e68dfd49092d49164b9c704ae897cee07dd Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Fri, 7 Sep 2018 19:12:35 -0700 Subject: [PATCH 068/103] lambda_solid and solid There are now two flavors of decorator. 1) lambda_solid. Takes inputs. No config or context. Produces a single output. Only return naked value out of the transform. 2) solids. Takes inputs, produces outputs. Accepts context and config. The transform can return a naked value, a Result, or a generator that yields a sequence of Results. This satisfies the need/desire for "simple" and introductory use cases for solids (in the form of a lambda_solid) relatively elegantly but then has a decorator-style solid that can cover nearly all use cases. --- .../dagster_ge_tests/test_pandas_ge.py | 14 +- python_modules/dagster/dagster/__init__.py | 2 + .../core/core_tests/test_compute_nodes.py | 10 +- .../core/core_tests/test_custom_context.py | 41 ++-- .../core/core_tests/test_decorators.py | 118 +++++++---- .../dagster/dagster/core/decorators.py | 190 +++++++++++++++--- .../dagster/dagster/core/definitions.py | 2 +- .../pandas_hello_world/pipeline.py | 20 +- .../test_pandas_hello_world_library_slide.py | 8 +- .../pandas/pandas_tests/test_pandas_solids.py | 20 +- .../pandas_tests/test_pandas_user_error.py | 13 +- .../dagster/docs/apidocs/decorators.rst | 2 + python_modules/dagster/setup.py | 2 +- 13 files changed, 295 insertions(+), 147 deletions(-) diff --git a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py index 52db733479761..3ac6035293bc1 100644 --- a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py +++ b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py @@ -9,7 +9,7 @@ PipelineDefinition, config, execute_pipeline, - solid, + lambda_solid, ) from dagster.core.errors import DagsterExpectationFailedError from dagster.core.utility_solids import define_stub_solid @@ -29,7 +29,7 @@ def col_exists(name, col_name): return dagster_ge.ge_expectation(name, lambda ge_df: ge_df.expect_column_to_exist(col_name)) -@solid( +@lambda_solid( inputs=[ InputDefinition( 'num_df', dagster_pd.DataFrame, expectations=[col_exists('num1_exists', 'num1')] @@ -37,11 +37,11 @@ def col_exists(name, col_name): ], output=OutputDefinition(dagster_type=dagster_pd.DataFrame) ) -def sum_solid(_context, _conf, num_df): +def sum_solid(num_df): return _sum_solid_impl(num_df) -@solid( +@lambda_solid( inputs=[ InputDefinition( 'num_df', @@ -51,11 +51,11 @@ def sum_solid(_context, _conf, num_df): ], output=OutputDefinition(dagster_type=dagster_pd.DataFrame) ) -def sum_solid_fails_input_expectation(_context, _conf, num_df): +def sum_solid_fails_input_expectation(num_df): return _sum_solid_impl(num_df) -@solid( +@lambda_solid( inputs=[ InputDefinition( 'num_df', @@ -69,7 +69,7 @@ def sum_solid_fails_input_expectation(_context, _conf, num_df): ], output=OutputDefinition(dagster_type=dagster_pd.DataFrame) ) -def sum_solid_expectations_config(_context, _conf, num_df): +def sum_solid_expectations_config(num_df): return _sum_solid_impl(num_df) diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index 61eedc589bcd8..e124a322cd2bf 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -27,6 +27,7 @@ from dagster.core.decorators import ( MultipleResults, + lambda_solid, solid, ) @@ -55,6 +56,7 @@ 'Result', # Decorators + 'lambda_solid', 'solid', 'MultipleResults', diff --git a/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py b/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py index 5a29e71ed2a22..ad1640b7c682a 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py +++ b/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py @@ -1,11 +1,9 @@ from dagster import ( - ConfigDefinition, ExecutionContext, - OutputDefinition, PipelineContextDefinition, PipelineDefinition, config, - solid, + lambda_solid, ) from dagster.core.execution import ( @@ -25,8 +23,8 @@ def silencing_default_context(): } -@solid(name='noop', inputs=[], outputs=[OutputDefinition()]) -def noop_solid(_context, _conf): +@lambda_solid +def noop(): return 'foo' @@ -36,7 +34,7 @@ def silencing_pipeline(solids): def test_compute_noop_node(): pipeline = silencing_pipeline(solids=[ - noop_solid, + noop, ]) environment = config.Environment() diff --git a/python_modules/dagster/dagster/core/core_tests/test_custom_context.py b/python_modules/dagster/dagster/core/core_tests/test_custom_context.py index 2b4b6a421b37d..f25bafc2158ea 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_custom_context.py +++ b/python_modules/dagster/dagster/core/core_tests/test_custom_context.py @@ -9,6 +9,7 @@ PipelineContextDefinition, config, execute_pipeline, + lambda_solid, solid, types, ) @@ -20,11 +21,8 @@ def test_default_context(): - @solid( - inputs=[], - outputs=[OutputDefinition()], - ) - def default_context_transform(context, _conf): + @solid(inputs=[], outputs=[OutputDefinition()]) + def default_context_transform(context, _): for logger in context._logger.loggers: assert logger.level == ERROR @@ -35,11 +33,8 @@ def default_context_transform(context, _conf): def test_default_context_with_log_level(): - @solid( - inputs=[], - outputs=[OutputDefinition()], - ) - def default_context_transform(context, _conf): + @solid(inputs=[], outputs=[OutputDefinition()]) + def default_context_transform(context, _): for logger in context._logger.loggers: assert logger.level == INFO @@ -58,11 +53,8 @@ def default_context_transform(context, _conf): def test_default_value(): def _get_config_test_solid(config_key, config_value): - @solid( - inputs=[], - outputs=[OutputDefinition()], - ) - def config_test(context, _conf): + @solid(inputs=[], outputs=[OutputDefinition()]) + def config_test(context, _): assert context.resources == {config_key: config_value} return config_test @@ -93,11 +85,8 @@ def config_test(context, _conf): def test_custom_contexts(): - @solid( - inputs=[], - outputs=[OutputDefinition()], - ) - def custom_context_transform(context, _conf): + @solid(inputs=[], outputs=[OutputDefinition()]) + def custom_context_transform(context, _): assert context.resources == {'field_one': 'value_two'} pipeline = PipelineDefinition( @@ -140,11 +129,8 @@ def custom_context_transform(context, _conf): def test_yield_context(): events = [] - @solid( - inputs=[], - outputs=[OutputDefinition()], - ) - def custom_context_transform(context, _conf): + @solid(inputs=[], outputs=[OutputDefinition()]) + def custom_context_transform(context, _): assert context.resources == {'field_one': 'value_two'} assert context._context_dict['foo'] == 'bar' # pylint: disable=W0212 events.append('during') @@ -183,10 +169,7 @@ def _yield_context(_pipeline, config_value): # TODO: reenable pending the ability to specific optional arguments # https://github.com/dagster-io/dagster/issues/56 def test_invalid_context(): - @solid( - inputs=[], - outputs=[OutputDefinition()], - ) + @lambda_solid def never_transform(): raise Exception('should never execute') diff --git a/python_modules/dagster/dagster/core/core_tests/test_decorators.py b/python_modules/dagster/dagster/core/core_tests/test_decorators.py index 4d9e940528e24..4fcb47fa7c579 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_decorators.py +++ b/python_modules/dagster/dagster/core/core_tests/test_decorators.py @@ -1,5 +1,4 @@ import pytest -import dagster from dagster import ( ConfigDefinition, DagsterInvalidDefinitionError, @@ -12,6 +11,7 @@ Result, config, execute_pipeline, + lambda_solid, solid, ) @@ -32,11 +32,29 @@ def create_empty_test_env(): return config.Environment() +def test_multiple_single_result(): + mr = MultipleResults(Result('value', 'output_one')) + assert mr.results == [Result('value', 'output_one')] + + +def test_multiple_double_result(): + mr = MultipleResults(Result('value_one', 'output_one'), Result('value_two', 'output_two')) + assert mr.results == [Result('value_one', 'output_one'), Result('value_two', 'output_two')] + + +def test_multiple_dict(): + mr = MultipleResults.from_dict({'output_one': 'value_one', 'output_two': 'value_two'}) + assert set(mr.results) == set( + [Result('value_one', 'output_one'), + Result('value_two', 'output_two')] + ) + + def test_no_parens_solid(): called = {} - @solid - def hello_world(_context, _conf): + @lambda_solid + def hello_world(): called['yup'] = True result = execute_single_solid( @@ -51,8 +69,8 @@ def hello_world(_context, _conf): def test_empty_solid(): called = {} - @solid() - def hello_world(_context, _conf): + @lambda_solid() + def hello_world(): called['yup'] = True result = execute_single_solid( @@ -81,8 +99,8 @@ def hello_world(_context, _conf): def test_solid_one_output(): - @solid(output=OutputDefinition()) - def hello_world(_context, _conf): + @lambda_solid + def hello_world(): return {'foo': 'bar'} result = execute_single_solid( @@ -97,8 +115,8 @@ def hello_world(_context, _conf): def test_solid_yield(): - @solid(output=OutputDefinition()) - def hello_world(_context, _conf): + @solid(outputs=[OutputDefinition()]) + def hello_world(_context, _): yield Result(value={'foo': 'bar'}) result = execute_single_solid( @@ -113,7 +131,7 @@ def hello_world(_context, _conf): def test_solid_result_return(): - @solid(output=OutputDefinition()) + @solid(outputs=[OutputDefinition()]) def hello_world(_context, _conf): return Result(value={'foo': 'bar'}) @@ -129,10 +147,7 @@ def hello_world(_context, _conf): def test_solid_multiple_outputs(): - @solid(outputs=[ - OutputDefinition(name="left"), - OutputDefinition(name="right"), - ]) + @solid(outputs=[OutputDefinition(name="left"), OutputDefinition(name="right")]) def hello_world(_context, _conf): return MultipleResults( Result(value={'foo': 'left'}, output_name='left'), @@ -153,10 +168,7 @@ def hello_world(_context, _conf): def test_dict_multiple_outputs(): - @solid(outputs=[ - OutputDefinition(name="left"), - OutputDefinition(name="right"), - ]) + @solid(outputs=[OutputDefinition(name="left"), OutputDefinition(name="right")]) def hello_world(_context, _conf): return MultipleResults.from_dict({ 'left': { @@ -180,6 +192,22 @@ def hello_world(_context, _conf): assert solid_result.transformed_value('right')['foo'] == 'right' +def test_lambda_solid_with_name(): + @lambda_solid(name="foobar") + def hello_world(): + return {'foo': 'bar'} + + result = execute_single_solid( + create_test_context(), + hello_world, + environment=create_empty_test_env(), + ) + + assert result.success + assert len(result.result_list) == 1 + assert result.result_list[0].transformed_value()['foo'] == 'bar' + + def test_solid_with_name(): @solid(name="foobar", outputs=[OutputDefinition()]) def hello_world(_context, _conf): @@ -197,8 +225,8 @@ def hello_world(_context, _conf): def test_solid_with_input(): - @solid(inputs=[InputDefinition(name="foo_to_foo")], outputs=[OutputDefinition()]) - def hello_world(_context, _conf, foo_to_foo): + @lambda_solid(inputs=[InputDefinition(name="foo_to_foo")]) + def hello_world(foo_to_foo): return foo_to_foo pipeline = PipelineDefinition( @@ -219,39 +247,55 @@ def hello_world(_context, _conf, foo_to_foo): assert result.transformed_value()['foo'] == 'bar' +def test_lambda_solid_definition_errors(): + with pytest.raises(DagsterInvalidDefinitionError, match='positional vararg'): + + @lambda_solid(inputs=[InputDefinition(name="foo")]) + def vargs(foo, *args): + pass + + def test_solid_definition_errors(): - with pytest.raises(DagsterInvalidDefinitionError): + with pytest.raises(DagsterInvalidDefinitionError, match='positional vararg'): @solid(inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()]) - def vargs(_context, _conf, foo, *args): + def vargs(context, conf, foo, *args): pass with pytest.raises(DagsterInvalidDefinitionError): @solid(inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()]) - def wrong_name(_context, _conf, bar): + def wrong_name(context, conf, bar): pass with pytest.raises(DagsterInvalidDefinitionError): @solid( - inputs=[InputDefinition(name="foo"), - InputDefinition(name="bar")], + inputs=[ + InputDefinition(name="foo"), + InputDefinition(name="bar"), + ], outputs=[OutputDefinition()] ) - def wrong_name_2(_context, _conf, foo): + def wrong_name_2(context, conf, foo): pass with pytest.raises(DagsterInvalidDefinitionError): @solid(inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()]) - def no_context(foo): + def no_context(conf, foo): pass with pytest.raises(DagsterInvalidDefinitionError): @solid(inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()]) - def no_conf(_context, foo): + def no_context_with_both(conf, foo): + pass + + with pytest.raises(DagsterInvalidDefinitionError): + + @solid(inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()]) + def no_conf(context, foo): pass with pytest.raises(DagsterInvalidDefinitionError): @@ -271,15 +315,7 @@ def extras(_context, _conf, foo, bar): InputDefinition(name="bar")], outputs=[OutputDefinition()] ) - def valid_kwargs(_context, _conf, **kwargs): - pass - - @solid( - inputs=[InputDefinition(name="foo"), - InputDefinition(name="bar")], - outputs=[OutputDefinition()] - ) - def valid(_context, _conf, foo, bar): + def valid_kwargs(context, conf, **kwargs): pass @solid( @@ -287,7 +323,7 @@ def valid(_context, _conf, foo, bar): InputDefinition(name="bar")], outputs=[OutputDefinition()] ) - def valid_rontext(context, _conf, foo, bar): + def valid(context, conf, foo, bar): pass @@ -298,12 +334,12 @@ def non_solid_func(): with pytest.raises( DagsterInvalidDefinitionError, match='You have passed a lambda or function non_solid_func' ): - dagster.PipelineDefinition(solids=[non_solid_func]) + PipelineDefinition(solids=[non_solid_func]) with pytest.raises( DagsterInvalidDefinitionError, match='You have passed a lambda or function ' ): - dagster.PipelineDefinition(solids=[lambda x: x]) + PipelineDefinition(solids=[lambda x: x]) def test_descriptions(): @@ -319,7 +355,7 @@ def test_any_config_definition(): conf_value = 234 @solid(config_def=ConfigDefinition()) - def hello_world(_context, conf): + def hello_world(context, conf): assert conf == conf_value called['yup'] = True diff --git a/python_modules/dagster/dagster/core/decorators.py b/python_modules/dagster/dagster/core/decorators.py index 16a2454e907d2..c61c72d89d68d 100644 --- a/python_modules/dagster/dagster/core/decorators.py +++ b/python_modules/dagster/dagster/core/decorators.py @@ -12,8 +12,6 @@ check, ) -from .types import Any - if hasattr(inspect, 'signature'): funcsigs = inspect else: @@ -38,10 +36,10 @@ class MultipleResults(namedtuple('_MultipleResults', 'results')): OutputDefinition(name='bar'), ]) def my_solid(): - return MultipleResults([ + return MultipleResults( Result('Barb', 'foo'), Result('Glarb', 'bar'), - ]) + ) @solid(outputs=[ @@ -72,24 +70,48 @@ def from_dict(result_dict): return MultipleResults(*results) -class _Solid(object): +class _LambdaSolid(object): def __init__( self, name=None, inputs=None, - outputs=None, output=None, description=None, - config_def=None, ): self.name = check.opt_str_param(name, 'name') self.input_defs = check.opt_list_param(inputs, 'inputs', InputDefinition) + self.output_def = check.inst_param(output, 'output', OutputDefinition) + self.description = check.opt_str_param(description, 'description') - if output is not None and outputs is None: - self.outputs = [check.opt_inst_param(output, 'output', OutputDefinition)] - else: - self.outputs = check.opt_list_param(outputs, 'outputs', OutputDefinition) + def __call__(self, fn): + check.callable_param(fn, 'fn') + + if not self.name: + self.name = fn.__name__ + + _validate_transform_fn(self.name, fn, self.input_defs) + transform_fn = _create_lambda_solid_transform_wrapper(fn, self.input_defs, self.output_def) + return SolidDefinition( + name=self.name, + inputs=self.input_defs, + outputs=[self.output_def], + transform_fn=transform_fn, + description=self.description, + ) + +class _Solid(object): + def __init__( + self, + name=None, + inputs=None, + outputs=None, + description=None, + config_def=None, + ): + self.name = check.opt_str_param(name, 'name') + self.input_defs = check.opt_list_param(inputs, 'inputs', InputDefinition) + self.outputs = check.opt_list_param(outputs, 'outputs', OutputDefinition) self.description = check.opt_str_param(description, 'description') self.config_def = check.opt_inst_param(config_def, 'config_def', ConfigDefinition) @@ -99,8 +121,8 @@ def __call__(self, fn): if not self.name: self.name = fn.__name__ - _validate_transform_fn(self.name, fn, self.input_defs) - transform_fn = _create_transform_wrapper(fn, self.input_defs, self.outputs) + _validate_transform_fn(self.name, fn, self.input_defs, ['context', 'conf']) + transform_fn = _create_solid_transform_wrapper(fn, self.input_defs, self.outputs) return SolidDefinition( name=self.name, inputs=self.input_defs, @@ -111,12 +133,64 @@ def __call__(self, fn): ) -def solid(name=None, inputs=None, output=None, outputs=None, config_def=None, description=None): +def lambda_solid( + name=None, + inputs=None, + output=None, + description=None, +): + '''(decorator) Create a simple solid + + This shortcut allows the creation of simple solids that do not require + configuration and whose implementations do not require a context. + + Lambda solids take inputs an and produce an output. The body of the function + should return a single value. + + Args: + name (str): Name of solid + inputs (List[InputDefinition]): List of inputs + output (OutputDefinition): The output of the solid. Defaults to OutputDefinition() + description (str): Solid description + + .. code-block:: python + + @lambda_solid + def hello_world(): + return 'hello' + + @lambda_solid(inputs=[InputDefinition(name="foo")]) + def hello_world(foo): + return foo + + ''' + output = output or OutputDefinition() + + if callable(name): + check.invariant(inputs is None) + check.invariant(description is None) + return _LambdaSolid(output=output)(name) + + return _LambdaSolid( + name=name, + inputs=inputs, + output=output, + description=description, + ) + + +def solid( + name=None, + inputs=None, + outputs=None, + config_def=None, + description=None, +): '''(decorator) Create a solid with specified parameters. This shortcut simplifies core solid API by exploding arguments into kwargs of the transform function and omitting additional parameters when they are not needed. - Parameters are otherwise as per :py:class:`SolidDefinition`. + Parameters are otherwise as per :py:class:`SolidDefinition`. Decorated function is the transform function itself. Instead of having to yield result objects, transform support multiple simpler output types. @@ -127,6 +201,14 @@ def solid(name=None, inputs=None, output=None, outputs=None, config_def=None, de multiple outputs. Useful for solids that have multiple outputs. 4. Yield :py:class:`Result`. Same as default transform behaviour. + Args: + name (str): Name of solid + inputs (List[InputDefinition]): List of inputs + outputs (List[OutputDefinition]): List of outputs + config_def (ConfigDefinition): + The configuration for this solid. + description (str): Description of this solid. + Examples: .. code-block:: python @@ -143,11 +225,11 @@ def hello_world(context, conf): def hello_world(context, conf): return {'foo': 'bar'} - @solid(output=OutputDefinition()) + @solid(outputs=[OutputDefinition()]) def hello_world(context, conf): return Result(value={'foo': 'bar'}) - @solid(output=OutputDefinition()) + @solid(outputs=[OutputDefinition()]) def hello_world(context, conf): yield Result(value={'foo': 'bar'}) @@ -162,16 +244,32 @@ def hello_world(context, conf): }) @solid( - inputs=[InputDefinition(name="foo_to_foo")], + inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()] ) - def hello_world(context, conf, foo_to_foo): - return foo_to_foo + def hello_world(context, conf, foo): + return foo + + @solid( + inputs=[InputDefinition(name="foo")], + outputs=[OutputDefinition()], + ) + def hello_world(context, conf, foo): + context.info('log something') + return foo + + @solid( + inputs=[InputDefinition(name="foo")], + outputs=[OutputDefinition()], + config_def=ConfigDefinition(types.ConfigDictionary({'str_value' : Field(types.String)})), + ) + def hello_world(context, conf, foo): + # conf is a dictionary with 'str_value' key + return foo ''' if callable(name): check.invariant(inputs is None) - check.invariant(output is None) check.invariant(outputs is None) check.invariant(description is None) check.invariant(config_def is None) @@ -180,17 +278,37 @@ def hello_world(context, conf, foo_to_foo): return _Solid( name=name, inputs=inputs, - output=output, outputs=outputs, config_def=config_def, description=description, ) -# TODO change names and check types -def _create_transform_wrapper(fn, inputs, outputs): +def _create_lambda_solid_transform_wrapper(fn, input_defs, output_def): check.callable_param(fn, 'fn') - input_names = [input.name for input in inputs] + check.list_param(input_defs, 'input_defs', of_type=InputDefinition) + check.inst_param(output_def, 'output_def', OutputDefinition) + + input_names = [input_def.name for input_def in input_defs] + + @wraps(fn) + def transform(_context, inputs, _conf): + kwargs = {} + for input_name in input_names: + kwargs[input_name] = inputs[input_name] + + result = fn(**kwargs) + yield Result(value=result, output_name=output_def.name) + + return transform + + +def _create_solid_transform_wrapper(fn, input_defs, output_defs): + check.callable_param(fn, 'fn') + check.list_param(input_defs, 'input_defs', of_type=InputDefinition) + check.list_param(output_defs, 'output_defs', of_type=OutputDefinition) + + input_names = [input_def.name for input_def in input_defs] @wraps(fn) def transform(context, args, conf): @@ -209,8 +327,8 @@ def transform(context, args, conf): elif isinstance(result, MultipleResults): for item in result.results: yield item - elif len(outputs) == 1: - yield Result(value=result, output_name=outputs[0].name) + elif len(output_defs) == 1: + yield Result(value=result, output_name=output_defs[0].name) elif result is not None: # XXX(freiksenet) raise Exception('Output for a solid without an output.') @@ -233,10 +351,18 @@ def __init__(self, error_type, param=None, missing_names=None, **kwargs): self.missing_names = missing_names -def _validate_transform_fn(solid_name, transform_fn, inputs): +def _validate_transform_fn(solid_name, transform_fn, inputs, expected_positionals=None): + check.str_param(solid_name, 'solid_name') + check.callable_param(transform_fn, 'transform_fn') + check.list_param(inputs, 'inputs', of_type=InputDefinition) + expected_positionals = check.opt_list_param( + expected_positionals, + 'expected_positionals', + of_type=str, + ) + names = set(inp.name for inp in inputs) # Currently being super strict about naming. Might be a good idea to relax. Starting strict. - expected_positionals = ('context', 'conf') try: _validate_decorated_fn(transform_fn, names, expected_positionals) except FunctionValidationError as e: @@ -277,8 +403,10 @@ def _validate_decorated_fn(fn, names, expected_positionals): for expected, actual in zip(expected_positionals, expected_positional_params): possible_names = [ - expected, '_{expected}'.format(expected=expected), - '{expected}_'.format(expected=expected) + '_', + expected, + '_{expected}'.format(expected=expected), + '{expected}_'.format(expected=expected), ] if ( actual.kind not in [ diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index 83fcda8c75947..9f434316d1252 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -344,7 +344,7 @@ def __init__( raise DagsterInvalidDefinitionError( '''You have passed a lambda or function {func} into a pipeline that is not a solid. You have likely forgetten to annotate this function - with an @solid decorator located in dagster.core.decorators + with an @solid or @lambda_solid decorator located in dagster.core.decorators '''.format(func=solid.__name__) ) diff --git a/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py b/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py index f13bac131bf7c..81821ccf4dd28 100644 --- a/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py +++ b/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py @@ -4,36 +4,36 @@ InputDefinition, OutputDefinition, PipelineDefinition, - solid, + lambda_solid, ) import dagster.pandas as dagster_pd -@solid( +@lambda_solid( inputs=[InputDefinition('num', dagster_pd.DataFrame)], - outputs=[OutputDefinition(dagster_type=dagster_pd.DataFrame)] + output=OutputDefinition(dagster_type=dagster_pd.DataFrame), ) -def sum_solid(_context, _conf, num): +def sum_solid(num): sum_df = num.copy() sum_df['sum'] = sum_df['num1'] + sum_df['num2'] return sum_df -@solid( +@lambda_solid( inputs=[InputDefinition('sum_df', dagster_pd.DataFrame)], - outputs=[OutputDefinition(dagster_type=dagster_pd.DataFrame)] + output=OutputDefinition(dagster_type=dagster_pd.DataFrame), ) -def sum_sq_solid(_context, _conf, sum_df): +def sum_sq_solid(sum_df): sum_sq_df = sum_df.copy() sum_sq_df['sum_sq'] = sum_df['sum']**2 return sum_sq_df -@solid( +@lambda_solid( inputs=[InputDefinition('sum_sq_solid', dagster_pd.DataFrame)], - outputs=[OutputDefinition(dagster_type=dagster_pd.DataFrame)] + output=OutputDefinition(dagster_type=dagster_pd.DataFrame), ) -def always_fails_solid(_context, _conf, **_kwargs): +def always_fails_solid(**_kwargs): raise Exception('I am a programmer and I make error') diff --git a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_library_slide.py b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_library_slide.py index 941baf11c0719..6927646b40a39 100644 --- a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_library_slide.py +++ b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_library_slide.py @@ -8,9 +8,9 @@ SolidDefinition, config, execute_pipeline, + lambda_solid, ) -from dagster.core.decorators import solid from dagster.utils import script_relative_path from dagster.utils.test import get_temp_file_name @@ -122,11 +122,11 @@ def transform_fn(_context, args): def create_decorator_based_solid(): - @solid( + @lambda_solid( inputs=[InputDefinition('num_csv', dagster_pd.DataFrame)], - outputs=[OutputDefinition(dagster_type=dagster_pd.DataFrame)], + output=OutputDefinition(dagster_type=dagster_pd.DataFrame), ) - def hello_world(_context, _conf, num_csv): + def hello_world(num_csv): num_csv['sum'] = num_csv['num1'] + num_csv['num2'] return num_csv diff --git a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py index 96a1923bc1b8a..98994f3640c02 100644 --- a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py +++ b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py @@ -11,7 +11,7 @@ SolidDefinition, check, config, - solid, + lambda_solid, ) from dagster.core.execution import ( @@ -176,30 +176,30 @@ def transform(_context, args): ) -@solid( +@lambda_solid( inputs=[InputDefinition('num_csv', dagster_pd.DataFrame)], - outputs=[OutputDefinition(dagster_type=dagster_pd.DataFrame)], + output=OutputDefinition(dagster_type=dagster_pd.DataFrame), ) -def sum_table(_context, _conf, num_csv): +def sum_table(num_csv): check.inst_param(num_csv, 'num_csv', pd.DataFrame) num_csv['sum'] = num_csv['num1'] + num_csv['num2'] return num_csv -@solid( +@lambda_solid( inputs=[InputDefinition('sum_df', dagster_pd.DataFrame)], - outputs=[OutputDefinition(dagster_type=dagster_pd.DataFrame)], + output=OutputDefinition(dagster_type=dagster_pd.DataFrame), ) -def sum_sq_table(_context, _conf, sum_df): +def sum_sq_table(sum_df): sum_df['sum_squared'] = sum_df['sum'] * sum_df['sum'] return sum_df -@solid( +@lambda_solid( inputs=[InputDefinition('sum_table_renamed', dagster_pd.DataFrame)], - outputs=[OutputDefinition(dagster_type=dagster_pd.DataFrame)], + output=OutputDefinition(dagster_type=dagster_pd.DataFrame), ) -def sum_sq_table_renamed_input(_context, _conf, sum_table_renamed): +def sum_sq_table_renamed_input(sum_table_renamed): sum_table_renamed['sum_squared'] = sum_table_renamed['sum'] * sum_table_renamed['sum'] return sum_table_renamed diff --git a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_user_error.py b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_user_error.py index 4783d49b7697f..59dd361c9d8c5 100644 --- a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_user_error.py +++ b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_user_error.py @@ -13,7 +13,7 @@ SolidDefinition, config, execute_pipeline, - solid, + lambda_solid, ) from dagster.core.errors import DagsterInvariantViolationError from dagster.core.utility_solids import define_stub_solid @@ -31,12 +31,12 @@ def _dataframe_solid(name, inputs, transform_fn): def test_wrong_output_value(): csv_input = InputDefinition('num_csv', dagster_pd.DataFrame) - @solid( + @lambda_solid( name="test_wrong_output", inputs=[csv_input], - outputs=[OutputDefinition(dagster_type=dagster_pd.DataFrame)] + output=OutputDefinition(dagster_type=dagster_pd.DataFrame), ) - def df_solid(_context, _conf, num_csv): + def df_solid(num_csv): return 'not a dataframe' pass_solid = define_stub_solid('pass_solid', pd.DataFrame()) @@ -56,12 +56,11 @@ def df_solid(_context, _conf, num_csv): def test_wrong_input_value(): - @solid( + @lambda_solid( name="test_wrong_input", inputs=[InputDefinition('foo', dagster_pd.DataFrame)], - outputs=[OutputDefinition()], ) - def df_solid(_context, _conf, foo): + def df_solid(foo): return foo pass_solid = define_stub_solid('pass_solid', 'not a dataframe') diff --git a/python_modules/dagster/docs/apidocs/decorators.rst b/python_modules/dagster/docs/apidocs/decorators.rst index e64909fa71d68..7bbdd8688e528 100644 --- a/python_modules/dagster/docs/apidocs/decorators.rst +++ b/python_modules/dagster/docs/apidocs/decorators.rst @@ -5,6 +5,8 @@ Decorators A more concise way to define solids. +.. autofunction:: lambda_solid + .. autofunction:: solid .. autoclass:: MultipleResults diff --git a/python_modules/dagster/setup.py b/python_modules/dagster/setup.py index cbbb52692b33c..4aaecc40fc9f6 100644 --- a/python_modules/dagster/setup.py +++ b/python_modules/dagster/setup.py @@ -18,7 +18,7 @@ def long_description(): setup( name='dagster', - version='0.2.0.dev5', + version='0.2.0.dev8', author='Elementl', author_email='schrockn@elementl.com', license='Apache-2.0', From 944727747dbede4624a5448be21393032ae29895 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Sat, 8 Sep 2018 05:15:33 -0700 Subject: [PATCH 069/103] Change order of arguments to transform function in SolidDefintion To be consistent with decorators (and logic generally), now: context, conf, inputs --- .../dagster/dagster/core/compute_nodes.py | 28 ++++++------- .../core/core_tests/test_definition_errors.py | 4 +- .../core/core_tests/test_multiple_outputs.py | 8 ++-- .../core/core_tests/test_naming_collisions.py | 14 +++---- .../core/core_tests/test_pipeline_errors.py | 4 +- .../core_tests/test_pipeline_execution.py | 4 +- .../core/core_tests/test_solid_with_config.py | 6 +-- .../dagster/dagster/core/decorators.py | 6 +-- .../dagster/dagster/core/definitions.py | 2 +- .../dagster/dagster/core/utility_solids.py | 2 +- .../dagster/dagster/pandas/__init__.py | 12 +++--- .../test_pandas_hello_world_library_slide.py | 4 +- ...est_pandas_hello_world_no_library_slide.py | 24 +++++------ .../pandas/pandas_tests/test_pandas_solids.py | 40 +++++++++---------- .../subquery_builder_experimental.py | 8 ++-- .../dagster/dagster/sqlalchemy/templated.py | 6 +-- python_modules/dagster/setup.py | 2 +- 17 files changed, 85 insertions(+), 89 deletions(-) diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index c8e3c20744741..f59369bfb71d1 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -178,8 +178,8 @@ class ComputeNodeTag(Enum): EXPECTATION_INPUT = 'expectation_input' -def _yield_transform_results(context, compute_node, values_dict, config_dict): - gen = compute_node.solid.transform_fn(context, values_dict, config_dict) +def _yield_transform_results(context, compute_node, conf, inputs): + gen = compute_node.solid.transform_fn(context, conf, inputs) if isinstance(gen, Result): raise DagsterInvariantViolationError( @@ -206,14 +206,14 @@ def _yield_transform_results(context, compute_node, values_dict, config_dict): ) yield result -def _execute_core_transform(context, compute_node, values_dict, conf): +def _execute_core_transform(context, compute_node, conf, inputs): ''' Execute the user-specified transform for the solid. Wrap in an error boundary and do all relevant logging and metrics tracking ''' check.inst_param(context, 'context', ExecutionContext) check.inst_param(compute_node, 'compute_node', ComputeNode) - check.dict_param(values_dict, 'values_dict', key_type=str) + check.dict_param(inputs, 'inputs', key_type=str) error_str = 'Error occured during core transform' @@ -227,7 +227,7 @@ def _execute_core_transform(context, compute_node, values_dict, conf): time_execution_scope() as timer_result, \ context.value('solid', compute_node.solid.name): - for result in _yield_transform_results(context, compute_node, values_dict, conf): + for result in _yield_transform_results(context, compute_node, conf, inputs): yield result context.metric('core_transform_time_ms', timer_result.millis) @@ -552,15 +552,15 @@ def __setitem__(self, key, val): return dict.__setitem__(self, key, val) -def validate_config_dict(execution_info, solid): +def create_conf_value(execution_info, solid): check.inst_param(execution_info, 'execution_info', ComputeNodeExecutionInfo) check.inst_param(solid, 'solid', SolidDefinition) name = solid.name solid_configs = execution_info.environment.solids - conf = solid_configs[name].config if name in solid_configs else {} + config_input = solid_configs[name].config if name in solid_configs else {} - evaluation_result = solid.config_def.config_type.evaluate_value(conf) + evaluation_result = solid.config_def.config_type.evaluate_value(config_input) if evaluation_result.success: return evaluation_result.value else: @@ -615,13 +615,9 @@ def create_compute_node_graph(execution_info): ) ) - validated_config_args = validate_config_dict(execution_info, topo_solid) + conf = create_conf_value(execution_info, topo_solid) - solid_transform_cn = create_compute_node_from_solid_transform( - topo_solid, - cn_inputs, - validated_config_args, - ) + solid_transform_cn = create_compute_node_from_solid_transform(topo_solid, cn_inputs, conf) compute_nodes.append(solid_transform_cn) @@ -758,7 +754,7 @@ def _do_expectation(context, _compute_node, inputs): return _do_expectation -def create_compute_node_from_solid_transform(solid, node_inputs, config_args): +def create_compute_node_from_solid_transform(solid, node_inputs, conf): check.inst_param(solid, 'solid', SolidDefinition) check.list_param(node_inputs, 'node_inputs', of_type=ComputeNodeInput) @@ -773,8 +769,8 @@ def create_compute_node_from_solid_transform(solid, node_inputs, config_args): compute_fn=lambda context, compute_node, inputs: _execute_core_transform( context, compute_node, + conf, inputs, - config_args, ), tag=ComputeNodeTag.TRANSFORM, solid=solid, diff --git a/python_modules/dagster/dagster/core/core_tests/test_definition_errors.py b/python_modules/dagster/dagster/core/core_tests/test_definition_errors.py index 55a9286fbca71..d7e39268655d1 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_definition_errors.py +++ b/python_modules/dagster/dagster/core/core_tests/test_definition_errors.py @@ -16,13 +16,13 @@ def solid_a_b_list(): name='A', inputs=[], outputs=[OutputDefinition()], - transform_fn=lambda context, inputs, config: None, + transform_fn=lambda context, conf, inputs: None, ), SolidDefinition( name='B', inputs=[InputDefinition('b_input')], outputs=[], - transform_fn=lambda context, inputs, config: None, + transform_fn=lambda context, conf, inputs: None, ) ] diff --git a/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py b/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py index 2271893b2c988..afb616b61bfc6 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py +++ b/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py @@ -16,7 +16,7 @@ def test_multiple_outputs(): - def _t_fn(_context, _inputs, _config_dict): + def _t_fn(*_args): yield Result(output_name='output_one', value='foo') yield Result(output_name='output_two', value='bar') @@ -87,7 +87,7 @@ def _transform_fn(*_args, **_kwargs): def test_wrong_multiple_output(): - def _t_fn(_context, _inputs, _config_dict): + def _t_fn(*args): yield Result(output_name='mismatch', value='foo') solid = SolidDefinition( @@ -108,7 +108,7 @@ def _t_fn(_context, _inputs, _config_dict): def test_multiple_outputs_of_same_name_disallowed(): # make this illegal until it is supported - def _t_fn(_context, _inputs, _config_dict): + def _t_fn(*args): yield Result(output_name='output_one', value='foo') yield Result(output_name='output_one', value='foo') @@ -128,7 +128,7 @@ def _t_fn(_context, _inputs, _config_dict): def test_multiple_outputs_only_emit_one(): - def _t_fn(_context, _inputs, _config_dict): + def _t_fn(*args): yield Result(output_name='output_one', value='foo') solid = SolidDefinition( diff --git a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py index 372b228363207..356fc87e8fce0 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py +++ b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py @@ -20,8 +20,8 @@ def define_pass_value_solid(name, description=None): check.str_param(name, 'name') check.opt_str_param(description, 'description') - def _value_t_fn(_context, _inputs, config_dict): - yield Result(config_dict['value']) + def _value_t_fn(_context, conf, _inputs): + yield Result(conf['value']) return SolidDefinition( name=name, @@ -39,7 +39,7 @@ def test_execute_solid_with_input_same_name(): a_thing_solid = SolidDefinition.single_output_transform( 'a_thing', inputs=[InputDefinition(name='a_thing')], - transform_fn=lambda context, args: args['a_thing'] + args['a_thing'], + transform_fn=lambda context, inputs: inputs['a_thing'] + inputs['a_thing'], output=dagster.OutputDefinition(), ) @@ -66,14 +66,14 @@ def test_execute_two_solids_with_same_input_name(): solid_one = SolidDefinition.single_output_transform( 'solid_one', inputs=[input_def], - transform_fn=lambda context, args: args['a_thing'] + args['a_thing'], + transform_fn=lambda context, inputs: inputs['a_thing'] + inputs['a_thing'], output=dagster.OutputDefinition(), ) solid_two = SolidDefinition.single_output_transform( 'solid_two', inputs=[input_def], - transform_fn=lambda context, args: args['a_thing'] + args['a_thing'], + transform_fn=lambda context, inputs: inputs['a_thing'] + inputs['a_thing'], output=dagster.OutputDefinition(), ) @@ -120,7 +120,7 @@ def test_execute_dep_solid_different_input_name(): first_solid = SolidDefinition.single_output_transform( 'first_solid', inputs=[InputDefinition(name='a_thing')], - transform_fn=lambda context, args: args['a_thing'] + args['a_thing'], + transform_fn=lambda context, inputs: inputs['a_thing'] + inputs['a_thing'], output=dagster.OutputDefinition(), ) @@ -129,7 +129,7 @@ def test_execute_dep_solid_different_input_name(): inputs=[ InputDefinition(name='an_input'), ], - transform_fn=lambda context, args: args['an_input'] + args['an_input'], + transform_fn=lambda context, inputs: inputs['an_input'] + inputs['an_input'], output=dagster.OutputDefinition(), ) diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py index 8c9177d458e34..91e580d77a109 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py @@ -80,9 +80,9 @@ def test_failure_midstream(): solid_a = create_root_success_solid('A') solid_b = create_root_success_solid('B') - def transform_fn(_context, args): + def transform_fn(_context, inputs): check.failed('user error') - return [args['A'], args['B'], {'C': 'transform_called'}] + return [inputs['A'], inputs['B'], {'C': 'transform_called'}] solid_c = SolidDefinition.single_output_transform( name='C', diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py index cc2a32fb51015..f0f82cb3fce21 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py @@ -35,10 +35,10 @@ def create_dep_input_fn(name): def make_transform(name): - def transform(_context, args): + def transform(_context, inputs): passed_rows = [] seen = set() - for row in args.values(): + for row in inputs.values(): for item in row: key = list(item.keys())[0] if key not in seen: diff --git a/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py b/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py index 299ef39cf7e18..c089245bfd72e 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py +++ b/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py @@ -16,8 +16,8 @@ def test_basic_solid_with_config(): did_get = {} - def _t_fn(_context, _inputs, config_dict): - did_get['yep'] = config_dict + def _t_fn(_context, conf, _inputs): + did_get['yep'] = conf solid = SolidDefinition( name='solid_with_context', @@ -43,7 +43,7 @@ def _t_fn(_context, _inputs, config_dict): def test_config_arg_mismatch(): - def _t_fn(_context, _inputs, _config_dict): + def _t_fn(*args): raise Exception('should not reach') solid = SolidDefinition( diff --git a/python_modules/dagster/dagster/core/decorators.py b/python_modules/dagster/dagster/core/decorators.py index c61c72d89d68d..4c404aeff73eb 100644 --- a/python_modules/dagster/dagster/core/decorators.py +++ b/python_modules/dagster/dagster/core/decorators.py @@ -292,7 +292,7 @@ def _create_lambda_solid_transform_wrapper(fn, input_defs, output_def): input_names = [input_def.name for input_def in input_defs] @wraps(fn) - def transform(_context, inputs, _conf): + def transform(_context, _conf, inputs): kwargs = {} for input_name in input_names: kwargs[input_name] = inputs[input_name] @@ -311,10 +311,10 @@ def _create_solid_transform_wrapper(fn, input_defs, output_defs): input_names = [input_def.name for input_def in input_defs] @wraps(fn) - def transform(context, args, conf): + def transform(context, conf, inputs): kwargs = {} for input_name in input_names: - kwargs[input_name] = args[input_name] + kwargs[input_name] = inputs[input_name] result = fn(context, conf, **kwargs) diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index 9f434316d1252..bba03ee33bc97 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -880,7 +880,7 @@ def single_output_transform(name, inputs, transform_fn, output, description=None ''' - def _new_transform_fn(context, inputs, _config_dict): + def _new_transform_fn(context, _conf, inputs): value = transform_fn(context, inputs) if isinstance(value, Result): raise DagsterInvariantViolationError( diff --git a/python_modules/dagster/dagster/core/utility_solids.py b/python_modules/dagster/dagster/core/utility_solids.py index de652b4fb178b..b6638316afd42 100644 --- a/python_modules/dagster/dagster/core/utility_solids.py +++ b/python_modules/dagster/dagster/core/utility_solids.py @@ -9,7 +9,7 @@ def define_stub_solid(name, value): check.str_param(name, 'name') - def _value_t_fn(_context, _inputs, _config_dict): + def _value_t_fn(*_args): yield Result(value) return SolidDefinition( diff --git a/python_modules/dagster/dagster/pandas/__init__.py b/python_modules/dagster/dagster/pandas/__init__.py index cd4c3504018b6..aacebfc29d5ce 100644 --- a/python_modules/dagster/dagster/pandas/__init__.py +++ b/python_modules/dagster/dagster/pandas/__init__.py @@ -31,8 +31,8 @@ def _create_dataframe_type(): def load_csv_solid(name): check.str_param(name, 'name') - def _t_fn(_context, _inputs, config_dict): - yield Result(pd.read_csv(config_dict['path'])) + def _t_fn(_context, conf, _inputs): + yield Result(pd.read_csv(conf['path'])) return SolidDefinition( name=name, @@ -46,8 +46,8 @@ def _t_fn(_context, _inputs, config_dict): def to_csv_solid(name): - def _t_fn(_context, inputs, config_dict): - inputs['df'].to_csv(config_dict['path'], index=False) + def _t_fn(_context, conf, inputs): + inputs['df'].to_csv(conf['path'], index=False) return SolidDefinition( name=name, @@ -61,8 +61,8 @@ def _t_fn(_context, inputs, config_dict): def to_parquet_solid(name): - def _t_fn(_context, inputs, config_dict): - inputs['df'].to_parquet(config_dict['path']) + def _t_fn(_context, conf, inputs): + inputs['df'].to_parquet(conf['path']) return SolidDefinition( name=name, diff --git a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_library_slide.py b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_library_slide.py index 6927646b40a39..b7f9f2793993e 100644 --- a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_library_slide.py +++ b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_library_slide.py @@ -106,8 +106,8 @@ def run_hello_world(hello_world): def create_definition_based_solid(): table_input = InputDefinition('num_csv', dagster_pd.DataFrame) - def transform_fn(_context, args): - num_csv = args['num_csv'] + def transform_fn(_context, inputs): + num_csv = inputs['num_csv'] num_csv['sum'] = num_csv['num1'] + num_csv['num2'] return num_csv diff --git a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_no_library_slide.py b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_no_library_slide.py index 9db3abd9ede6e..e656f3df9601a 100644 --- a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_no_library_slide.py +++ b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_no_library_slide.py @@ -18,8 +18,8 @@ def define_read_csv_solid(name): - def _t_fn(_context, _inputs, config_dict): - yield Result(pd.read_csv(config_dict['path'])) + def _t_fn(_context, conf, _inputs): + yield Result(pd.read_csv(conf['path'])) return SolidDefinition( name=name, @@ -33,8 +33,8 @@ def _t_fn(_context, _inputs, config_dict): def define_to_csv_solid(name): - def _t_fn(_context, inputs, config_dict): - inputs['df'].to_csv(config_dict['path'], index=False) + def _t_fn(_context, conf, inputs): + inputs['df'].to_csv(conf['path'], index=False) return SolidDefinition( name=name, @@ -48,8 +48,8 @@ def _t_fn(_context, inputs, config_dict): def test_hello_world_pipeline_no_api(): - def hello_world_transform_fn(_context, args): - num_df = args['num_df'] + def hello_world_transform_fn(_context, inputs): + num_df = inputs['num_df'] num_df['sum'] = num_df['num1'] + num_df['num2'] return num_df @@ -94,8 +94,8 @@ def hello_world_transform_fn(_context, args): def create_hello_world_solid_composed_pipeline(): - def transform_fn(_context, args): - num_df = args['num_df'] + def transform_fn(_context, inputs): + num_df = inputs['num_df'] num_df['sum'] = num_df['num1'] + num_df['num2'] return num_df @@ -142,8 +142,8 @@ def test_hello_world_composed(): def test_pandas_hello_no_library(): - def solid_one_transform(_context, args): - num_df = args['num_df'] + def solid_one_transform(_context, inputs): + num_df = inputs['num_df'] num_df['sum'] = num_df['num1'] + num_df['num2'] return num_df @@ -154,8 +154,8 @@ def solid_one_transform(_context, args): output=OutputDefinition(), ) - def solid_two_transform(_context, args): - sum_df = args['sum_df'] + def solid_two_transform(_context, inputs): + sum_df = inputs['sum_df'] sum_df['sum_sq'] = sum_df['sum'] * sum_df['sum'] return sum_df diff --git a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py index 98994f3640c02..6f685233fa823 100644 --- a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py +++ b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py @@ -70,8 +70,8 @@ def create_test_context(): def test_basic_pandas_solid(): csv_input = InputDefinition('num_csv', dagster_pd.DataFrame) - def transform(_context, args): - num_csv = args['num_csv'] + def transform(_context, inputs): + num_csv = inputs['num_csv'] num_csv['sum'] = num_csv['num1'] + num_csv['num2'] return num_csv @@ -107,9 +107,9 @@ def test_pandas_csv_to_csv(): csv_input = InputDefinition('num_csv', dagster_pd.DataFrame) # just adding a second context arg to test that - def transform(context, args): + def transform(context, inputs): check.inst_param(context, 'context', ExecutionContext) - num_csv = args['num_csv'] + num_csv = inputs['num_csv'] num_csv['sum'] = num_csv['num1'] + num_csv['num2'] return num_csv @@ -163,8 +163,8 @@ def execute_transform_in_temp_csv_files(solid_inst): def create_sum_table(): - def transform(_context, args): - num_csv = args['num_csv'] + def transform(_context, inputs): + num_csv = inputs['num_csv'] check.inst_param(num_csv, 'num_csv', pd.DataFrame) num_csv['sum'] = num_csv['num1'] + num_csv['num2'] return num_csv @@ -232,9 +232,9 @@ def _sum_only_pipeline(): def test_two_input_solid(): - def transform(_context, args): - num_csv1 = args['num_csv1'] - num_csv2 = args['num_csv2'] + def transform(_context, inputs): + num_csv1 = inputs['num_csv1'] + num_csv2 = inputs['num_csv2'] check.inst_param(num_csv1, 'num_csv1', pd.DataFrame) check.inst_param(num_csv2, 'num_csv2', pd.DataFrame) num_csv1['sum'] = num_csv1['num1'] + num_csv2['num2'] @@ -287,7 +287,7 @@ def test_no_transform_solid(): num_table = _dataframe_solid( name='num_table', inputs=[InputDefinition('num_csv', dagster_pd.DataFrame)], - transform_fn=lambda _context, args: args['num_csv'], + transform_fn=lambda _context, inputs: inputs['num_csv'], ) context = create_test_context() df = get_solid_transformed_value( @@ -329,11 +329,11 @@ def create_diamond_dag(): num_table_solid = _dataframe_solid( name='num_table', inputs=[InputDefinition('num_csv', dagster_pd.DataFrame)], - transform_fn=lambda _context, args: args['num_csv'], + transform_fn=lambda _context, inputs: inputs['num_csv'], ) - def sum_transform(_context, args): - num_df = args['num_table'] + def sum_transform(_context, inputs): + num_df = inputs['num_table'] sum_df = num_df.copy() sum_df['sum'] = num_df['num1'] + num_df['num2'] return sum_df @@ -344,8 +344,8 @@ def sum_transform(_context, args): transform_fn=sum_transform, ) - def mult_transform(_context, args): - num_table = args['num_table'] + def mult_transform(_context, inputs): + num_table = inputs['num_table'] mult_table = num_table.copy() mult_table['mult'] = num_table['num1'] * num_table['num2'] return mult_table @@ -356,9 +356,9 @@ def mult_transform(_context, args): transform_fn=mult_transform, ) - def sum_mult_transform(_context, args): - sum_df = args['sum_table'] - mult_df = args['mult_table'] + def sum_mult_transform(_context, inputs): + sum_df = inputs['sum_table'] + mult_df = inputs['mult_table'] sum_mult_table = sum_df.copy() sum_mult_table['mult'] = mult_df['mult'] sum_mult_table['sum_mult'] = sum_df['sum'] * mult_df['mult'] @@ -597,8 +597,8 @@ def test_pandas_multiple_inputs(): }, ) - def transform_fn(_context, args): - return args['num_csv1'] + args['num_csv2'] + def transform_fn(_context, inputs): + return inputs['num_csv1'] + inputs['num_csv2'] double_sum = _dataframe_solid( name='double_sum', diff --git a/python_modules/dagster/dagster/sqlalchemy/subquery_builder_experimental.py b/python_modules/dagster/dagster/sqlalchemy/subquery_builder_experimental.py index 25490e12811b7..144e6abe34f1e 100644 --- a/python_modules/dagster/dagster/sqlalchemy/subquery_builder_experimental.py +++ b/python_modules/dagster/dagster/sqlalchemy/subquery_builder_experimental.py @@ -48,10 +48,10 @@ def from_target(self): def define_create_table_solid(name): - def _materialization_fn(context, inputs, config_dict): + def _materialization_fn(context, conf, inputs): sql_expr = inputs['expr'] check.inst(sql_expr, DagsterSqlExpression) - output_table_name = check.str_elem(config_dict, 'table_name') + output_table_name = check.str_elem(conf, 'table_name') total_sql = '''CREATE TABLE {output_table_name} AS {query_text}'''.format( output_table_name=output_table_name, query_text=sql_expr.query_text ) @@ -77,9 +77,9 @@ def _table_name_read_fn(_context, arg_dict): def create_sql_transform(sql_text): - def transform_fn(_context, args): + def transform_fn(_context, inputs): sql_texts = {} - for name, sql_expr in args.items(): + for name, sql_expr in inputs.items(): sql_texts[name] = sql_expr.from_target return DagsterSqlQueryExpression(sql_text.format(**sql_texts)) diff --git a/python_modules/dagster/dagster/sqlalchemy/templated.py b/python_modules/dagster/dagster/sqlalchemy/templated.py index b9e9e09efbca5..5635823aa45c4 100644 --- a/python_modules/dagster/dagster/sqlalchemy/templated.py +++ b/python_modules/dagster/dagster/sqlalchemy/templated.py @@ -42,9 +42,9 @@ def _render_template_string(template_text, config_dict): def _create_templated_sql_transform_with_output(sql): - def do_transform(context, _inputs, config_dict): - rendered_sql = _render_template_string(sql, config_dict) + def do_transform(context, conf, _inputs): + rendered_sql = _render_template_string(sql, conf) execute_sql_text_on_context(context, rendered_sql) - yield Result(config_dict) + yield Result(conf) return do_transform diff --git a/python_modules/dagster/setup.py b/python_modules/dagster/setup.py index 4aaecc40fc9f6..0d9c72392779b 100644 --- a/python_modules/dagster/setup.py +++ b/python_modules/dagster/setup.py @@ -18,7 +18,7 @@ def long_description(): setup( name='dagster', - version='0.2.0.dev8', + version='0.2.0.dev9', author='Elementl', author_email='schrockn@elementl.com', license='Apache-2.0', From 98bb42a8583a0a929caaf602185116d8b200ab55 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Sat, 8 Sep 2018 07:12:50 -0700 Subject: [PATCH 070/103] Intro Tutorial parts 1-4 It begins --- python_modules/dagster/dagster/__init__.py | 5 +- .../core/core_tests/test_multiple_outputs.py | 6 +- .../core/core_tests/test_solid_with_config.py | 2 +- .../dagster/dagster/core/decorators.py | 3 +- .../dagster/dagster/core/execution.py | 13 ++- .../test_intro_tutorial_part_four.py | 25 ++++++ .../tutorials/test_intro_tutorial_part_one.py | 22 ++++++ .../test_intro_tutorial_part_three.py | 55 +++++++++++++ .../tutorials/test_intro_tutorial_part_two.py | 32 ++++++++ python_modules/dagster/docs/conf.py | 4 +- python_modules/dagster/docs/index.rst | 14 +++- .../dagster/docs/intro_tutorial/part_four.rst | 59 ++++++++++++++ .../dagster/docs/intro_tutorial/part_one.rst | 41 ++++++++++ .../docs/intro_tutorial/part_three.rst | 79 +++++++++++++++++++ .../dagster/docs/intro_tutorial/part_two.rst | 61 ++++++++++++++ python_modules/dagster/docs/tutorials.rst | 0 16 files changed, 407 insertions(+), 14 deletions(-) create mode 100644 python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_four.py create mode 100644 python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_one.py create mode 100644 python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_three.py create mode 100644 python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_two.py create mode 100644 python_modules/dagster/docs/intro_tutorial/part_four.rst create mode 100644 python_modules/dagster/docs/intro_tutorial/part_one.rst create mode 100644 python_modules/dagster/docs/intro_tutorial/part_three.rst create mode 100644 python_modules/dagster/docs/intro_tutorial/part_two.rst delete mode 100644 python_modules/dagster/docs/tutorials.rst diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index e124a322cd2bf..94995893364e6 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -1,6 +1,3 @@ -from __future__ import (absolute_import, division, print_function, unicode_literals) -from builtins import * # pylint: disable=W0622,W0401 - from dagster.core.execution import ( PipelineExecutionResult, SolidExecutionResult, @@ -39,6 +36,7 @@ DagsterExpectationFailedError, ) +import dagster.config as config import dagster.core.types as types __all__ = [ @@ -75,5 +73,6 @@ 'DagsterExpectationFailedError', # types + 'config', 'types', ] diff --git a/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py b/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py index afb616b61bfc6..e78cb21757988 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py +++ b/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py @@ -87,7 +87,7 @@ def _transform_fn(*_args, **_kwargs): def test_wrong_multiple_output(): - def _t_fn(*args): + def _t_fn(*_args): yield Result(output_name='mismatch', value='foo') solid = SolidDefinition( @@ -108,7 +108,7 @@ def _t_fn(*args): def test_multiple_outputs_of_same_name_disallowed(): # make this illegal until it is supported - def _t_fn(*args): + def _t_fn(*_args): yield Result(output_name='output_one', value='foo') yield Result(output_name='output_one', value='foo') @@ -128,7 +128,7 @@ def _t_fn(*args): def test_multiple_outputs_only_emit_one(): - def _t_fn(*args): + def _t_fn(*_args): yield Result(output_name='output_one', value='foo') solid = SolidDefinition( diff --git a/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py b/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py index c089245bfd72e..237deb07fb99d 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py +++ b/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py @@ -43,7 +43,7 @@ def _t_fn(_context, conf, _inputs): def test_config_arg_mismatch(): - def _t_fn(*args): + def _t_fn(*_args): raise Exception('should not reach') solid = SolidDefinition( diff --git a/python_modules/dagster/dagster/core/decorators.py b/python_modules/dagster/dagster/core/decorators.py index 4c404aeff73eb..8e7d57cba5f50 100644 --- a/python_modules/dagster/dagster/core/decorators.py +++ b/python_modules/dagster/dagster/core/decorators.py @@ -111,7 +111,8 @@ def __init__( ): self.name = check.opt_str_param(name, 'name') self.input_defs = check.opt_list_param(inputs, 'inputs', InputDefinition) - self.outputs = check.opt_list_param(outputs, 'outputs', OutputDefinition) + outputs = outputs or [OutputDefinition()] + self.outputs = check.list_param(outputs, 'outputs', OutputDefinition) self.description = check.opt_str_param(description, 'description') self.config_def = check.opt_inst_param(config_def, 'config_def', ConfigDefinition) diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index e02e5888ff217..01bd7ad05eb13 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -302,7 +302,7 @@ def _execute_graph_iterator(context, execution_graph, environment): def execute_pipeline( pipeline, - environment, + environment=None, throw_on_error=True, ): ''' @@ -313,7 +313,9 @@ def execute_pipeline( Parameters: pipeline (PipelineDefinition): pipeline to run execution (ExecutionContext): execution context of the run - throw_on_error (bool): throw_on_error makes the function throw when an error is encoutered rather than returning the py:class:`SolidExecutionResult` in an error-state. + throw_on_error (bool): + throw_on_error makes the function throw when an error is encoutered rather than returning + the py:class:`SolidExecutionResult` in an error-state. Returns: @@ -321,7 +323,12 @@ def execute_pipeline( ''' check.inst_param(pipeline, 'pipeline', PipelineDefinition) - check.inst_param(environment, 'environment', config.Environment) + environment = check.opt_inst_param( + environment, + 'environment', + config.Environment, + config.Environment(), + ) execution_graph = ExecutionGraph.from_pipeline(pipeline) return _execute_graph(execution_graph, environment, throw_on_error) diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_four.py b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_four.py new file mode 100644 index 0000000000000..4eb39e951b3b2 --- /dev/null +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_four.py @@ -0,0 +1,25 @@ +# pylint: disable=W0622,W0614,W0401 +from dagster import * + + +@solid +def hello_world(_context, conf): + print(conf) + return conf + + +def test_tutorial_part_four(): + + result = execute_pipeline( + PipelineDefinition(solids=[hello_world]), + config.Environment(solids={'hello_world': config.Solid('Hello, World!')}), + ) + + assert result.success + assert len(result.result_list) == 1 + assert result.result_for_solid('hello_world').transformed_value() is 'Hello, World!' + return result + + +if __name__ == '__main__': + test_tutorial_part_four() diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_one.py b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_one.py new file mode 100644 index 0000000000000..ba7631c162262 --- /dev/null +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_one.py @@ -0,0 +1,22 @@ +# pylint: disable=W0622,W0614,W0401 +from dagster import * + + +def test_tutorial_part_one(): + pipeline = PipelineDefinition(solids=[hello_world]) + + result = execute_pipeline(pipeline) + + assert result.success + assert len(result.result_list) == 1 + assert result.result_for_solid('hello_world').transformed_value() is None + return result + + +@lambda_solid +def hello_world(): + print('hello') + + +if __name__ == '__main__': + test_tutorial_part_one() diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_three.py b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_three.py new file mode 100644 index 0000000000000..dd145c61b8cd1 --- /dev/null +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_three.py @@ -0,0 +1,55 @@ +# pylint: disable=W0622,W0614,W0401 +from dagster import * + +# B +# / \ +# A D +# \ / +# C + + +@lambda_solid +def solid_a(): + print('a: 1') + return 1 + + +@lambda_solid(inputs=[InputDefinition('arg_a')]) +def solid_b(arg_a): + print('b: {b}'.format(b=arg_a * 2)) + return arg_a * 2 + + +@lambda_solid(inputs=[InputDefinition('arg_a')]) +def solid_c(arg_a): + print('c: {c}'.format(c=arg_a * 3)) + return arg_a * 3 + + +@lambda_solid(inputs=[InputDefinition('arg_b'), InputDefinition('arg_c')]) +def solid_d(arg_b, arg_c): + print('d: {d}'.format(d=arg_b * arg_c)) + + +def test_tutorial_part_three(): + execute_pipeline( + PipelineDefinition( + solids=[solid_d, solid_c, solid_b, solid_a], + dependencies={ + 'solid_b': { + 'arg_a': DependencyDefinition('solid_a'), + }, + 'solid_c': { + 'arg_a': DependencyDefinition('solid_a'), + }, + 'solid_d': { + 'arg_b': DependencyDefinition('solid_b'), + 'arg_c': DependencyDefinition('solid_c'), + } + } + ) + ) + + +if __name__ == '__main__': + test_tutorial_part_three() diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_two.py b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_two.py new file mode 100644 index 0000000000000..b4483e73760a3 --- /dev/null +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_two.py @@ -0,0 +1,32 @@ +# pylint: disable=W0622,W0614,W0401 +from dagster import * + + +@lambda_solid +def solid_one(): + return 'foo' + + +@lambda_solid(inputs=[InputDefinition('arg_one')]) +def solid_two(arg_one): + print(arg_one * 2) + + +def test_tutorial_part_two(): + pipeline_result = execute_pipeline( + PipelineDefinition( + solids=[solid_one, solid_two], + dependencies={ + 'solid_two': { + 'arg_one': DependencyDefinition('solid_one'), + }, + } + ) + ) + + assert pipeline_result.success + return pipeline_result + + +if __name__ == '__main__': + test_tutorial_part_two() diff --git a/python_modules/dagster/docs/conf.py b/python_modules/dagster/docs/conf.py index 428573eda2c42..d55a7ec0405b8 100644 --- a/python_modules/dagster/docs/conf.py +++ b/python_modules/dagster/docs/conf.py @@ -21,8 +21,8 @@ from recommonmark.parser import CommonMarkParser project = 'Dagster' -copyright = '2018, Nicholas Schrock' # pylint: disable=W0622 -author = 'Nicholas Schrock' +copyright = '2018, Elementl, Inc' # pylint: disable=W0622 +author = 'The Dagster Team' # The short X.Y version version = '' diff --git a/python_modules/dagster/docs/index.rst b/python_modules/dagster/docs/index.rst index ed73e7447b879..235260b03a0d5 100644 --- a/python_modules/dagster/docs/index.rst +++ b/python_modules/dagster/docs/index.rst @@ -11,10 +11,22 @@ Welcome to Dagster, an opinionated programming model for data pipelines. motivations principles - tutorials topic_guides contributing +Intro Tutorial +============== +.. toctree:: + :maxdepth: 1 + :name: Intro Tutorial + :caption: Intro Tutorial + + intro_tutorial/part_one + intro_tutorial/part_two + intro_tutorial/part_three + intro_tutorial/part_four + + API Reference ================ diff --git a/python_modules/dagster/docs/intro_tutorial/part_four.rst b/python_modules/dagster/docs/intro_tutorial/part_four.rst new file mode 100644 index 0000000000000..b5126a2fe6e8a --- /dev/null +++ b/python_modules/dagster/docs/intro_tutorial/part_four.rst @@ -0,0 +1,59 @@ +Configuration +------------- + +So far we have only demonstrated pipelines that produce hardcoded values +and then flow them through the pipeline. In order to be useful a pipeline +must also interact with its external environment, and in general, it should +use configuration to do so. + +For maximum flexiblity, testabilty, and reusability pipelines should be fully +parameterizable. Configuration is how we achieve that end in dagster. + +We return to our hello world example, but now we will be able to parameterize +the string printed via config. + +In order to accomplish this we need to change APIs, from ``lambda_solid`` to ``solid``. +``lambda_solid`` only exposes a subset of solid features in order to provide a more +minimal API. ``solid`` is more complicated, but has more capabilities: + +.. code-block:: python + + from dagster import * + from dagster import config + + @solid + def hello_world(_context, conf): + print(conf) + return conf + + + if __name__ == '__main__': + execute_pipeline( + PipelineDefinition(solids=[hello_world]), + config.Environment( + solids={'hello_world': config.Solid('Hello, World!')} + ), + ) + +You'll notice a new API, ``solid``. We will be exploring this API in much more detail as this +tutorial proceeds. For now, the only is that the function annotated by solid now +takes two parameters where before it took zero (if it accepted no inputs). Ignore +the first parameter, ``_context``, for now. We will cover that in a later section +of the tutorial. The other parameter is ``conf``, which is short for config (which +we cannot use because of the name collision with the config namespace of dagster). +It represents the configuration for that solid. + +We must provide that configuration. And thusly turn your attention to the second argument +of execute_pipeline, which must be an instance of ``config.Environment``. An ``Environment`` +encompasses *all* of the configuration to execute an entire pipeline. It has many +sections. One of these is configuration provided on a per-solid basis, which is what +we are using here. The ``solids`` property of ``Environment`` is a dictionary keyed by +solid name. It points to a ``config.Solid`` object which in this case takes the value +that will be passed directly to the solid in question, in this case a string to be printed. + +So save this example as step_four.py + +.. code-block:: sh + + $ python3 step_four.py + Hello, World! diff --git a/python_modules/dagster/docs/intro_tutorial/part_one.rst b/python_modules/dagster/docs/intro_tutorial/part_one.rst new file mode 100644 index 0000000000000..5123ecedb5a76 --- /dev/null +++ b/python_modules/dagster/docs/intro_tutorial/part_one.rst @@ -0,0 +1,41 @@ +Hello, World +------------ + +The first step is to get dagster up and running quickly. We will initially be using the +python API for executing pipelines and eventually move on to driving execution and +inspecting pipelines with tools. + + + .. code-block:: python + + # Do not recommend using wildcard imports in production code + from dagster import * + + @lambda_solid + def hello_world(): + print('hello') + + if __name__ == '__main__': + execute_pipeline(PipelineDefinition(solids=[hello_world])) + +This example introduces three concepts: + +1) A solid. ``@lambda_solid`` marks the the function ``hello_world`` as solid. A solid represents +a functional unit of computations in a data pipeline. A lambda is a minimal form of a solid. We +will explore more aspects of solids as this tutorial continues. + +2) A pipeline. The call to ``PipelineDefinition`` which a set of solids arranged to do a +computation that produces data assets. In this case we are creating a pipeline with a single +solid and nothing else. + +3) ``execute_pipeline``. This will execute the pipeline. The framework will eventually call back +into ``hello_world`` and print hello to the screen. + +Save this to a file called ``step_one.py`` and run + +.. code-block:: sh + + $ python3 step_one.py + hello + +Congrats! You have run your first pipeline. \ No newline at end of file diff --git a/python_modules/dagster/docs/intro_tutorial/part_three.rst b/python_modules/dagster/docs/intro_tutorial/part_three.rst new file mode 100644 index 0000000000000..b50860d474857 --- /dev/null +++ b/python_modules/dagster/docs/intro_tutorial/part_three.rst @@ -0,0 +1,79 @@ +An actual DAG +------------- + +Next we will build a slightly more complicated DAG that demonstrates how it +effects execution order. In this case will be build a diamond dag: + +.. code-block:: python + + # B + # / \ + # A D + # \ / + # C + + + @lambda_solid + def solid_a(): + print('a: 1') + return 1 + + + @lambda_solid(inputs=[InputDefinition('arg_a')]) + def solid_b(arg_a): + print('b: {b}'.format(b=arg_a * 2)) + return arg_a * 2 + + + @lambda_solid(inputs=[InputDefinition('arg_a')]) + def solid_c(arg_a): + print('c: {c}'.format(c=arg_a * 3)) + return arg_a * 3 + + + @lambda_solid(inputs=[ + InputDefinition('arg_b'), + InputDefinition('arg_c'), + ]) + def solid_d(arg_b, arg_c): + print('d: {d}'.format(d=arg_b * arg_c)) + + + if __name__ == '__main__': + execute_pipeline( + PipelineDefinition( + # Order of this solid list does not matter. + # The dependencies argument determines execution order. + # Solids will execute in topological order. + solids=[solid_d, solid_c, solid_b, solid_a], + dependencies={ + 'solid_b': { + 'arg_a': DependencyDefinition('solid_a'), + }, + 'solid_c': { + 'arg_a': DependencyDefinition('solid_a'), + }, + 'solid_d': { + 'arg_b': DependencyDefinition('solid_b'), + 'arg_c': DependencyDefinition('solid_c'), + } + } + ) + ) + +Again it is worth noting how we are connecting *inputs* and *outputs* rather than just *tasks*. +Point your attention to the ``solid_d`` entry in the dependencies dictionary. We are declaring +dependencies on a per-input basis. + +Save this to a file named ``step_three.py`` + +.. code-block:: sh + + $ python3 step_three.py + a: 1 + b: 2 + c: 3 + d: 6 + +In this case ``solid_b`` happens to execute before ``solid_c``. However ``solid_c`` executing +before ``solid_b`` would also be a valid execution order given this DAG. diff --git a/python_modules/dagster/docs/intro_tutorial/part_two.rst b/python_modules/dagster/docs/intro_tutorial/part_two.rst new file mode 100644 index 0000000000000..bfc807d9b326f --- /dev/null +++ b/python_modules/dagster/docs/intro_tutorial/part_two.rst @@ -0,0 +1,61 @@ +Hello, DAG +---------- + +One of the core capabitilies of dagster is the ability to arrange solids into directed, acyclic +graphs (DAG) within pipelines. We will demonstrate how to do that. + +We will be building a very simple pipeline where the first step returns a hardcoded string, then +passes that value to the next solid, which concatenates it to itself, and prints it. + + + .. code-block:: python + + from dagster import * + + @lambda_solid + def solid_one(): + return 'foo' + + + @lambda_solid(inputs=[InputDefinition('arg_one')]) + def solid_two(arg_one): + print(arg_one * 2) + + + if __name__ == '__main__': + pipeline = PipelineDefinition( + solids=[solid_one, solid_two], + dependencies={ + 'solid_two': { + 'arg_one': DependencyDefinition('solid_one'), + }, + } + ) + pipeline_result = execute_pipeline(pipeline) + +We have a couple new concepts here. + +1) ``InputDefinition``: Creating an instance declares that a solid has an input. Dagster is +now aware of this input and can perform tasks like manage it dependencies. + +2) ``DependencyDefinition``: You'll notice a new argument to ``PipelineDefinition`` called +``dependencies``. This defines the depenedency graph of the DAG resident within a pipeline. +The first layer of keys are the names of solids. The second layer of keys are the names of +the inputs to that particular solid. Each input in the DAG must be provided a +``DependencyDefinition``. In this case the dictionary encodes the fact that the input ``arg_one`` +of solid ``solid_two`` should be sourced from the output of ``solid_one``. + +One of the distinguishing features of dagster that separates it from many workflow engines is that +dependencies connect *inputs* and *outputs* rather than just *tasks*. An author of a dagster +pipeline is not just defining the flow of execution, but also the flow of *data* within that +execution. Understanding this is critical to understanding the programming model of dagster, where +each step in the pipeline -- the solid -- is a *functional* unit of computation. + +Save this file to ``step_two.py`` + +and run + +.. code-block:: sh + + $ python3 step_two.py + foofoo \ No newline at end of file diff --git a/python_modules/dagster/docs/tutorials.rst b/python_modules/dagster/docs/tutorials.rst deleted file mode 100644 index e69de29bb2d1d..0000000000000 From 549afe03ad1b4de8e1934dc7a65084c5cdb39125 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Sun, 9 Sep 2018 06:34:56 -0700 Subject: [PATCH 071/103] Context Tutorial + API improvements Add context tutorial section. Clean up up default context handling. --- .../dagster_ge_tests/test_pandas_ge.py | 6 +- python_modules/dagster/dagster/config.py | 17 ++- .../dagster/dagster/core/compute_nodes.py | 22 ++-- .../core/core_tests/test_custom_context.py | 19 ++-- .../core/core_tests/test_decorators.py | 5 +- .../core/core_tests/test_multiple_outputs.py | 10 +- .../core/core_tests/test_pipeline_errors.py | 11 +- .../core_tests/test_pipeline_execution.py | 2 +- .../test_transform_only_pipeline.py | 4 +- .../dagster/dagster/core/definitions.py | 8 +- .../dagster/dagster/core/test_utils.py | 9 +- python_modules/dagster/dagster/core/types.py | 5 +- .../test_sql_project_pipeline.py | 6 +- .../pandas_tests/test_pandas_user_error.py | 10 +- .../sqlalchemy_tests/test_basic_solid.py | 7 +- .../test_isolated_sql_tests.py | 8 +- .../dagster/dagster_tests/test_config.py | 14 +++ .../test_intro_tutorial_part_five.py | 44 ++++++++ python_modules/dagster/docs/index.rst | 1 + .../dagster/docs/intro_tutorial/part_five.rst | 100 ++++++++++++++++++ .../dagster/docs/intro_tutorial/part_four.rst | 1 - .../docs/intro_tutorial/part_three.rst | 2 + 22 files changed, 231 insertions(+), 80 deletions(-) create mode 100644 python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_five.py create mode 100644 python_modules/dagster/docs/intro_tutorial/part_five.rst diff --git a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py index 3ac6035293bc1..1427dc586b749 100644 --- a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py +++ b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py @@ -82,7 +82,7 @@ def test_single_node_passing_expectation(): }} ) - result = execute_pipeline(pipeline, config.Environment()) + result = execute_pipeline(pipeline) assert result.success assert len(result.result_list) == 2 assert result.result_list[1].success @@ -104,7 +104,7 @@ def test_single_node_passing_json_config_expectations(): } ) - result = execute_pipeline(pipeline, config.Environment()) + result = execute_pipeline(pipeline) assert result.success assert len(result.result_list) == 2 assert result.result_list[1].success @@ -129,5 +129,5 @@ def test_single_node_failing_expectation(): # NOTE: this is not what I want to API to be but at least it exercises # the code path for now with pytest.raises(DagsterExpectationFailedError): - result = execute_pipeline(pipeline, config.Environment()) + result = execute_pipeline(pipeline) assert not result.success diff --git a/python_modules/dagster/dagster/config.py b/python_modules/dagster/dagster/config.py index 4ceda5b8ec267..70d806327e61a 100644 --- a/python_modules/dagster/dagster/config.py +++ b/python_modules/dagster/dagster/config.py @@ -2,13 +2,15 @@ from dagster import check +DEFAULT_CONTEXT_NAME = 'default' + # lifted from https://bit.ly/2HcQAuv class Context(namedtuple('ContextData', 'name config')): - def __new__(cls, name, config=None): + def __new__(cls, name=None, config=None): return super(Context, cls).__new__( cls, - check.str_param(name, 'name'), + check.opt_str_param(name, 'name', DEFAULT_CONTEXT_NAME), config, ) @@ -21,9 +23,10 @@ def __new__(cls, config): class Environment(namedtuple('EnvironmentData', 'context solids expectations')): def __new__(cls, solids=None, context=None, expectations=None): check.opt_inst_param(context, 'context', Context) + check.opt_inst_param(expectations, 'expectations', Expectations) if context is None: - context = Context(name='default', config={}) + context = Context() if expectations is None: expectations = Expectations(evaluate=True) @@ -32,13 +35,9 @@ def __new__(cls, solids=None, context=None, expectations=None): cls, context=context, solids=check.opt_dict_param(solids, 'solids', key_type=str, value_type=Solid), - expectations=check.inst_param(expectations, 'expectations', Expectations) + expectations=expectations, ) - @staticmethod - def empty(): - return Environment() - class Expectations(namedtuple('ExpectationsData', 'evaluate')): def __new__(cls, evaluate): @@ -51,7 +50,7 @@ def __new__(cls, evaluate): def _construct_context(yml_config_object): context_obj = check.opt_dict_elem(yml_config_object, 'context') if context_obj: - return Context(check.str_elem(context_obj, 'name'), context_obj['config']) + return Context(check.opt_str_elem(context_obj, 'name'), context_obj['config']) else: return None diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index f59369bfb71d1..fd94011d2be79 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -217,20 +217,22 @@ def _execute_core_transform(context, compute_node, conf, inputs): error_str = 'Error occured during core transform' - context.debug( - 'Executing core transform for solid {solid}.'.format( - solid=compute_node.solid.name + + with context.value('solid', compute_node.solid.name): + context.debug( + 'Executing core transform for solid {solid}.'.format( + solid=compute_node.solid.name + ) ) - ) - with _user_code_error_boundary(context, error_str), \ - time_execution_scope() as timer_result, \ - context.value('solid', compute_node.solid.name): + with time_execution_scope() as timer_result, \ + _user_code_error_boundary(context, error_str): + + for result in _yield_transform_results(context, compute_node, conf, inputs): + yield result - for result in _yield_transform_results(context, compute_node, conf, inputs): - yield result - context.metric('core_transform_time_ms', timer_result.millis) + context.metric('core_transform_time_ms', timer_result.millis) class ComputeNodeInput(object): diff --git a/python_modules/dagster/dagster/core/core_tests/test_custom_context.py b/python_modules/dagster/dagster/core/core_tests/test_custom_context.py index f25bafc2158ea..929a2c749f819 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_custom_context.py +++ b/python_modules/dagster/dagster/core/core_tests/test_custom_context.py @@ -21,15 +21,18 @@ def test_default_context(): + called = {} + @solid(inputs=[], outputs=[OutputDefinition()]) def default_context_transform(context, _): + called['yes'] = True for logger in context._logger.loggers: assert logger.level == ERROR pipeline = PipelineDefinition(solids=[default_context_transform]) - execute_pipeline( - pipeline, environment=config.Environment(context=config.Context('default', {})) - ) + execute_pipeline(pipeline) + + assert called['yes'] def test_default_context_with_log_level(): @@ -41,13 +44,13 @@ def default_context_transform(context, _): pipeline = PipelineDefinition(solids=[default_context_transform]) execute_pipeline( pipeline, - environment=config.Environment(context=config.Context('default', {'log_level': 'INFO'})) + environment=config.Environment(context=config.Context(config={'log_level': 'INFO'})) ) with pytest.raises(DagsterTypeError, message='Argument mismatch in context default'): execute_pipeline( pipeline, - environment=config.Environment(context=config.Context('default', {'log_level': 2})) + environment=config.Environment(context=config.Context(config={'log_level': 2})) ) @@ -185,7 +188,7 @@ def never_transform(): ) environment_field_name_mismatch = config.Environment( - context=config.Context('default', {'unexpected': 'value'}) + context=config.Context(config={'unexpected': 'value'}) ) with pytest.raises(DagsterTypeError, message='Argument mismatch in context default'): @@ -208,7 +211,7 @@ def never_transform(): } ) - environment_no_config_error = config.Environment(context=config.Context('default', {})) + environment_no_config_error = config.Environment(context=config.Context(config={})) with pytest.raises(DagsterTypeError, message='Argument mismatch in context default'): execute_pipeline( @@ -218,7 +221,7 @@ def never_transform(): ) environment_type_mismatch_error = config.Environment( - context=config.Context('default', {'string_field': 1}) + context=config.Context(config={'string_field': 1}) ) with pytest.raises(DagsterTypeError, message='Argument mismatch in context default'): diff --git a/python_modules/dagster/dagster/core/core_tests/test_decorators.py b/python_modules/dagster/dagster/core/core_tests/test_decorators.py index 4fcb47fa7c579..b9514b6220d11 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_decorators.py +++ b/python_modules/dagster/dagster/core/core_tests/test_decorators.py @@ -236,10 +236,7 @@ def hello_world(foo_to_foo): }} ) - pipeline_result = execute_pipeline( - pipeline, - environment=config.Environment(), - ) + pipeline_result = execute_pipeline(pipeline) result = pipeline_result.result_for_solid('hello_world') diff --git a/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py b/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py index e78cb21757988..3cab6d7ee4c06 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py +++ b/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py @@ -32,7 +32,7 @@ def _t_fn(*_args): pipeline = PipelineDefinition(solids=[solid]) - result = execute_pipeline(pipeline, config.Environment()) + result = execute_pipeline(pipeline) solid_result = result.result_list[0] assert solid_result.solid.name == 'multiple_outputs' @@ -79,7 +79,7 @@ def _transform_fn(*_args, **_kwargs): pipeline = PipelineDefinition(solids=[solid]) - result = execute_pipeline(pipeline, config.Environment()) + result = execute_pipeline(pipeline) assert result.success assert called['expectation_one'] @@ -102,7 +102,7 @@ def _t_fn(*_args): pipeline = PipelineDefinition(solids=[solid]) with pytest.raises(DagsterInvariantViolationError): - execute_pipeline(pipeline, config.Environment()) + execute_pipeline(pipeline) def test_multiple_outputs_of_same_name_disallowed(): @@ -124,7 +124,7 @@ def _t_fn(*_args): pipeline = PipelineDefinition(solids=[solid]) with pytest.raises(DagsterInvariantViolationError): - execute_pipeline(pipeline, config.Environment()) + execute_pipeline(pipeline) def test_multiple_outputs_only_emit_one(): @@ -175,7 +175,7 @@ def _transform_fn_two(*_args, **_kwargs): }, ) - result = execute_pipeline(pipeline, config.Environment()) + result = execute_pipeline(pipeline) assert result.success assert called['one'] diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py index 91e580d77a109..029081566ed72 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py @@ -63,9 +63,7 @@ def failed_transform(**_kwargs): def test_transform_failure_pipeline(): pipeline = silencing_pipeline(solids=[create_root_transform_failure_solid('failing')]) - pipeline_result = execute_pipeline( - pipeline, environment=config.Environment(), throw_on_error=False - ) + pipeline_result = execute_pipeline(pipeline, throw_on_error=False) assert not pipeline_result.success @@ -102,7 +100,6 @@ def transform_fn(_context, inputs): ) pipeline_result = execute_pipeline( pipeline, - environment=config.Environment(), throw_on_error=False, ) @@ -126,7 +123,7 @@ def test_do_not_yield_result(): DagsterInvariantViolationError, message='Tranform for solid do_not_yield_result return a Result', ): - execute_single_solid(ExecutionContext(), solid_inst, config.Environment()) + execute_single_solid(ExecutionContext(), solid_inst) def test_yield_non_result(): @@ -144,7 +141,7 @@ def _tn(*_args, **_kwargs): DagsterInvariantViolationError, message="Tranform for solid yield_wrong_thing yielded 'foo'", ): - execute_single_solid(ExecutionContext(), solid_inst, config.Environment()) + execute_single_solid(ExecutionContext(), solid_inst) def test_single_transform_returning_result(): @@ -156,4 +153,4 @@ def test_single_transform_returning_result(): ) with pytest.raises(DagsterInvariantViolationError): - execute_single_solid(ExecutionContext(), solid_inst, config.Environment()) + execute_single_solid(ExecutionContext(), solid_inst) diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py index f0f82cb3fce21..444682140a3f0 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py @@ -258,7 +258,7 @@ def test_create_single_solid_pipeline(): }, ) - result = execute_pipeline(single_solid_pipeline, config.Environment()) + result = execute_pipeline(single_solid_pipeline) assert result.success diff --git a/python_modules/dagster/dagster/core/core_tests/test_transform_only_pipeline.py b/python_modules/dagster/dagster/core/core_tests/test_transform_only_pipeline.py index 12b246ba1d69b..fccd0a57b2d83 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_transform_only_pipeline.py +++ b/python_modules/dagster/dagster/core/core_tests/test_transform_only_pipeline.py @@ -42,7 +42,7 @@ def test_execute_solid_with_dep_only_inputs_no_api(): # from dagster.utils import logging - pipeline_result = execute_pipeline(pipeline, environment=config.Environment.empty()) + pipeline_result = execute_pipeline(pipeline) assert pipeline_result.success @@ -79,7 +79,7 @@ def test_execute_solid_with_dep_only_inputs_with_api(): }, ) - pipeline_result = execute_pipeline(pipeline, environment=config.Environment.empty()) + pipeline_result = execute_pipeline(pipeline) for result in pipeline_result.result_list: assert result.success diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index bba03ee33bc97..37128310c6927 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -13,6 +13,8 @@ define_colored_console_logger, ) +from dagster.config import DEFAULT_CONTEXT_NAME + from .errors import ( DagsterInvalidDefinitionError, DagsterInvariantViolationError, @@ -99,7 +101,7 @@ def passthrough_context_definition(context): check.inst_param(context, 'context', ExecutionContext) context_definition = PipelineContextDefinition(context_fn=lambda _pipeline, _args: context) - return {'default': context_definition} + return {DEFAULT_CONTEXT_NAME: context_definition} def __init__(self, context_fn, config_def=None, description=None): ''' @@ -154,7 +156,7 @@ def _default_context_fn(_pipeline, config_value): ), context_fn=_default_context_fn, ) - return {'default': default_context_def} + return {DEFAULT_CONTEXT_NAME: default_context_def} class DependencyDefinition(namedtuple('_DependencyDefinition', 'solid output description')): @@ -747,7 +749,7 @@ class ConfigDefinition(object): @staticmethod def config_dict(field_dict): - '''Shorcut to create a dictionary based config definition. + '''Shortcut to create a dictionary based config definition. Args: diff --git a/python_modules/dagster/dagster/core/test_utils.py b/python_modules/dagster/dagster/core/test_utils.py index 5752596b0e92f..f4941132214b2 100644 --- a/python_modules/dagster/dagster/core/test_utils.py +++ b/python_modules/dagster/dagster/core/test_utils.py @@ -9,10 +9,15 @@ ) -def execute_single_solid(context, solid, environment, throw_on_error=True): +def execute_single_solid(context, solid, environment=None, throw_on_error=True): check.inst_param(context, 'context', ExecutionContext) check.inst_param(solid, 'solid', SolidDefinition) - check.inst_param(environment, 'environment', config.Environment) + environment = check.opt_inst_param( + environment, + 'environment', + config.Environment, + config.Environment(), + ) check.bool_param(throw_on_error, 'throw_on_error') single_solid_environment = config.Environment( diff --git a/python_modules/dagster/dagster/core/types.py b/python_modules/dagster/dagster/core/types.py index 89def0739db92..08f3a2114851e 100644 --- a/python_modules/dagster/dagster/core/types.py +++ b/python_modules/dagster/dagster/core/types.py @@ -207,7 +207,7 @@ def __init__(self, name, fields, ctor, description=None): super(DagsterCompositeType, self).__init__(name, description) def evaluate_value(self, value): - if not isinstance(value, dict): + if value is not None and not isinstance(value, dict): return IncomingValueResult.create_failure('Incoming value for composite must be dict') return process_incoming_composite_value(self, value, self.ctor) @@ -259,8 +259,7 @@ def create_failure(error_msg): def process_incoming_composite_value(dagster_composite_type, incoming_value, ctor): check.inst_param(dagster_composite_type, 'dagster_composite_type', DagsterCompositeType) - check.dict_param(incoming_value, 'incoming_value', key_type=str) - # check.str_param(error_context_str, 'error_context_str') + incoming_value = check.opt_dict_param(incoming_value, 'incoming_value', key_type=str) check.callable_param(ctor, 'ctor') field_dict = dagster_composite_type.field_dict diff --git a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py index c299a2e571dbe..fc6eb5f961728 100644 --- a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py +++ b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py @@ -62,7 +62,7 @@ def test_sql_create_tables(): pipeline = create_mem_sql_pipeline_context_tuple(solids=[create_all_tables_solids]) - pipeline_result = execute_pipeline(pipeline, environment=config.Environment.empty()) + pipeline_result = execute_pipeline(pipeline) assert pipeline_result.success assert set(pipeline_engine(pipeline_result).table_names()) == set( @@ -86,7 +86,7 @@ def test_sql_populate_tables(): } ) - pipeline_result = execute_pipeline(pipeline, environment=config.Environment.empty()) + pipeline_result = execute_pipeline(pipeline) assert pipeline_result.success @@ -137,7 +137,7 @@ def create_full_pipeline(): def test_full_in_memory_pipeline(): pipeline = create_full_pipeline() - pipeline_result = execute_pipeline(pipeline, environment=config.Environment.empty()) + pipeline_result = execute_pipeline(pipeline) assert pipeline_result.success engine = pipeline_engine(pipeline_result) diff --git a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_user_error.py b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_user_error.py index 59dd361c9d8c5..bf765a3e51bcf 100644 --- a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_user_error.py +++ b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_user_error.py @@ -49,10 +49,7 @@ def df_solid(num_csv): ) with pytest.raises(DagsterInvariantViolationError): - execute_pipeline( - pipeline, - environment=config.Environment(), - ) + execute_pipeline(pipeline) def test_wrong_input_value(): @@ -73,7 +70,4 @@ def df_solid(foo): ) with pytest.raises(DagsterInvariantViolationError): - execute_pipeline( - pipeline, - environment=config.Environment(), - ) + execute_pipeline(pipeline) diff --git a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_basic_solid.py b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_basic_solid.py index 6086709fd7d48..54564616b6472 100644 --- a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_basic_solid.py +++ b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_basic_solid.py @@ -129,12 +129,7 @@ def create_sum_sq_pipeline(context, expr, extra_solids=None, extra_deps=None): def test_execute_sql_sum_sq_solid(): pipeline = create_sum_sq_pipeline(in_mem_context(), DagsterSqlTableExpression('num_table')) - environment = config.Environment() - - pipeline_result = execute_pipeline( - pipeline, - environment=environment, - ) + pipeline_result = execute_pipeline(pipeline) assert pipeline_result.success diff --git a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_sql_tests.py b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_sql_tests.py index 38bb1a89e3723..56c11d5804a41 100644 --- a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_sql_tests.py +++ b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_sql_tests.py @@ -34,9 +34,7 @@ def test_basic_isolated_sql_solid(): basic_isolated_sql_solid = create_sql_statement_solid('basic_isolated_sql_solid', sql_text) - result = execute_single_solid( - context, basic_isolated_sql_solid, environment=config.Environment.empty() - ) + result = execute_single_solid(context, basic_isolated_sql_solid) assert result.success @@ -69,7 +67,7 @@ def test_basic_pipeline(): }, ) - pipeline_result = execute_pipeline(pipeline, environment=config.Environment.empty()) + pipeline_result = execute_pipeline(pipeline) assert pipeline_result.success @@ -107,7 +105,7 @@ def test_pipeline_from_files(): }, ) - pipeline_result = execute_pipeline(pipeline, environment=config.Environment.empty()) + pipeline_result = execute_pipeline(pipeline) assert pipeline_result.success diff --git a/python_modules/dagster/dagster_tests/test_config.py b/python_modules/dagster/dagster_tests/test_config.py index 07839412dc832..804f2a0a4d9ac 100644 --- a/python_modules/dagster/dagster_tests/test_config.py +++ b/python_modules/dagster/dagster_tests/test_config.py @@ -16,3 +16,17 @@ def test_construct_full_environment(): assert environment == config.Environment( context=config.Context('default', {'context_arg': 'context_value'}), ) + + +def test_construct_full_environment_default_context_name(): + document = ''' +context: + config: + context_arg: context_value +''' + + environment = config.construct_environment(yaml.load(document)) + + assert environment == config.Environment( + context=config.Context('default', {'context_arg': 'context_value'}), + ) diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_five.py b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_five.py new file mode 100644 index 0000000000000..1084d04184b17 --- /dev/null +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_five.py @@ -0,0 +1,44 @@ +# pylint: disable=W0622,W0614,W0401 +from dagster import * + + +@solid +def solid_one(context, _conf): + context.info('Something you should know about occurred.') + + +@solid +def solid_two(context, _conf): + context.error('An error occurred.') + + +def test_tutorial_part_five_sample_one(): + pipeline_result = execute_pipeline(PipelineDefinition(solids=[solid_one, solid_two])) + + assert pipeline_result.success + return pipeline_result + + +def test_tutorial_part_five_sample_two(): + pipeline_result = execute_pipeline( + PipelineDefinition(name='part_five', solids=[solid_one, solid_two]) + ) + + assert pipeline_result.success + return pipeline_result + + +def test_tutorial_part_five_sample_three(): + pipeline_result = execute_pipeline( + PipelineDefinition(name='part_five', solids=[solid_one, solid_two]), + config.Environment(context=config.Context(config={'log_level': 'DEBUG'})) + ) + + assert pipeline_result.success + return pipeline_result + + +if __name__ == '__main__': + # test_tutorial_part_five_sample_one() + # test_tutorial_part_five_sample_two() + test_tutorial_part_five_sample_three() diff --git a/python_modules/dagster/docs/index.rst b/python_modules/dagster/docs/index.rst index 235260b03a0d5..9422237062841 100644 --- a/python_modules/dagster/docs/index.rst +++ b/python_modules/dagster/docs/index.rst @@ -25,6 +25,7 @@ Intro Tutorial intro_tutorial/part_two intro_tutorial/part_three intro_tutorial/part_four + intro_tutorial/part_five API Reference diff --git a/python_modules/dagster/docs/intro_tutorial/part_five.rst b/python_modules/dagster/docs/intro_tutorial/part_five.rst new file mode 100644 index 0000000000000..e7ea97f113fc2 --- /dev/null +++ b/python_modules/dagster/docs/intro_tutorial/part_five.rst @@ -0,0 +1,100 @@ +Execution Context +----------------- + +In addition to its configuration and its inputs, a solid also receives a ``context`` +argument. In this tutorial part, we'll go over this ``context`` variable which was +ignored in the previous step of the tutorial. + +The context is an object of type :py:class:`ExecutionContext`. For every execution +of a particular pipeline, one instance of this context is created, no matter how +many solids are involved. Runtime state or information that is particular to +to a run rather than particular to an individual solid should be associated with +the context. + +Let's use the context for one of its core capabilities: logging. + +.. code-block:: python + + from dagster import * + + @solid + def solid_one(context, _conf): + context.info('Something you should know about occurred.') + + + @solid + def solid_two(context, _conf): + context.error('An error occurred.') + + + if __name__ == '__main__': + execute_pipeline( + PipelineDefinition(solids=[solid_one, solid_two]) + ) + + +Save this as part_five.py and run + +.. code-block:: sh + + $ python3 part_five.py + 2018-09-09 07:14:19 - dagster - ERROR - message="An error occurred." pipeline=<> solid=solid_two + +Notice that even though the user only logged the message "An error occurred", by +routing logging through the context we are able to provide richer error information and then +log that in a semi-structured format. + +For example, let's change the example so that the pipeline has a name. (This is good practice anyways). + +.. code-block:: python + + execute_pipeline( + PipelineDefinition( + name='part_five', + solids=[solid_one, solid_two] + ) + ) + +And then run it: + +.. code-block:: sh + + $ python3 part_five.py + 2018-09-09 07:17:31 - dagster - ERROR - message="An error occurred." pipeline=part_five solid=solid_two + +You'll note that the metadata in the log message now has the pipeline name. + +But what about the info message? The default context provided by dagster logs error messages only at +``ERROR`` level to the console. The context must be configured inorder to do something that is +not default. Just like we used the configuration system to configure a particular solid, we also +use that same system to configure a context. + + +.. code-block:: python + + execute_pipeline( + PipelineDefinition( + name='part_five', + solids=[solid_one, solid_two] + ), + config.Environment( + context=config.Context(config={'log_level': 'DEBUG'}) + ), + ) + +If we re-run the pipeline, you'll see a lot more output. + +.. code-block:: sh + + $ python3 part_five.py + ... + 2018-09-09 07:49:51 - dagster - INFO - message="Something you should know about occurred." pipeline=part_five solid=solid_one + 2018-09-09 07:49:51 - dagster - INFO - metric:core_transform_time_ms=0.137 pipeline=part_five solid=solid_one + 2018-09-09 07:49:51 - dagster - DEBUG - message="Executing core transform for solid solid_two." pipeline=part_five solid=solid_two + 2018-09-09 07:49:51 - dagster - ERROR - message="An error occurred." pipeline=part_five solid=solid_two + ... + +This just touches on the capabilities of the execution context. The context will end up +being the system by which pipeline authors actually are able to make their pipelines +executable in different operating contexts (e.g. unit-testing, CI/CD, prod, etc) without +changing business logic. diff --git a/python_modules/dagster/docs/intro_tutorial/part_four.rst b/python_modules/dagster/docs/intro_tutorial/part_four.rst index b5126a2fe6e8a..51ea403d00d52 100644 --- a/python_modules/dagster/docs/intro_tutorial/part_four.rst +++ b/python_modules/dagster/docs/intro_tutorial/part_four.rst @@ -19,7 +19,6 @@ minimal API. ``solid`` is more complicated, but has more capabilities: .. code-block:: python from dagster import * - from dagster import config @solid def hello_world(_context, conf): diff --git a/python_modules/dagster/docs/intro_tutorial/part_three.rst b/python_modules/dagster/docs/intro_tutorial/part_three.rst index b50860d474857..8910e821465e0 100644 --- a/python_modules/dagster/docs/intro_tutorial/part_three.rst +++ b/python_modules/dagster/docs/intro_tutorial/part_three.rst @@ -6,6 +6,8 @@ effects execution order. In this case will be build a diamond dag: .. code-block:: python + from dagster import * + # B # / \ # A D From d7e147d9ca4f5b757bcb02b534ec217f48b15d6e Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Sun, 9 Sep 2018 10:33:18 -0700 Subject: [PATCH 072/103] Repositories and Tools Tutorial Also be able to load repositories from files, not just modules. --- .../dagster/dagster/cli/pipeline.py | 14 ++- .../dagster/dagster/cli/repository_config.py | 24 ++++- .../repository.yml | 3 + .../test_intro_tutorial_part_six.py | 23 +++++ python_modules/dagster/docs/index.rst | 1 + .../dagster/docs/intro_tutorial/part_six.rst | 88 +++++++++++++++++++ 6 files changed, 146 insertions(+), 7 deletions(-) create mode 100644 python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_six/repository.yml create mode 100644 python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_six/test_intro_tutorial_part_six.py create mode 100644 python_modules/dagster/docs/intro_tutorial/part_six.rst diff --git a/python_modules/dagster/dagster/cli/pipeline.py b/python_modules/dagster/dagster/cli/pipeline.py index 6ede7a5ac3e7f..2b40dc92f0bc4 100644 --- a/python_modules/dagster/dagster/cli/pipeline.py +++ b/python_modules/dagster/dagster/cli/pipeline.py @@ -37,9 +37,18 @@ def create_pipeline_cli(): @repository_config_argument def list_command(conf): repository = load_repository_from_file(conf).repository - click.echo('Repository {name}'.format(name=repository.name)) + title = 'Repository {name}'.format(name=repository.name) + click.echo(title) + click.echo('*' * len(title)) + first = True for pipeline in repository.get_all_pipelines(): - click.echo('Pipeline: {name}'.format(name=pipeline.name)) + pipeline_title = 'Pipeline: {name}'.format(name=pipeline.name) + + if not first: + click.echo('*' * len(pipeline_title)) + first = False + + click.echo(pipeline_title) if pipeline.description: click.echo('Description:') click.echo(format_description(pipeline.description, indent=' ' * 4)) @@ -47,7 +56,6 @@ def list_command(conf): solid_graph = ExecutionGraph(pipeline, pipeline.solids, pipeline.dependency_structure) for solid in solid_graph.topological_solids: click.echo(' ' + solid.name) - click.echo('*************') def format_description(desc, indent): diff --git a/python_modules/dagster/dagster/cli/repository_config.py b/python_modules/dagster/dagster/cli/repository_config.py index 91da9db5c201c..f4d3dd1240936 100644 --- a/python_modules/dagster/dagster/cli/repository_config.py +++ b/python_modules/dagster/dagster/cli/repository_config.py @@ -1,5 +1,7 @@ from collections import namedtuple +import imp import importlib +import os import click import yaml @@ -19,10 +21,14 @@ def load_repository_from_file(file_path): with open(file_path, 'r') as ff: config = yaml.load(ff) repository_config = check.dict_elem(config, 'repository') - module_name = check.str_elem(repository_config, 'module') + module_name = check.opt_str_elem(repository_config, 'module') + file_name = check.opt_str_elem(repository_config, 'file') fn_name = check.str_elem(repository_config, 'fn') - module, fn, repository = load_repository(module_name, fn_name) + if module_name: + module, fn, repository = load_repository_from_module_name(module_name, fn_name) + else: + module, fn, repository = load_repository_from_python_file(file_name, fn_name) return RepositoryInfo( repository=repository, @@ -33,14 +39,24 @@ def load_repository_from_file(file_path): ) -def load_repository(module_name, fn_name): - module = importlib.import_module(module_name) +def _create_repo_from_module(module, fn_name): fn = getattr(module, fn_name) check.is_callable(fn) repository = check.inst(fn(), RepositoryDefinition) return (module, fn, repository) +def load_repository_from_python_file(file_path, fn_name): + module_name = os.path.splitext(os.path.basename(file_path))[0] + module = imp.load_source(module_name, file_path) + return _create_repo_from_module(module, fn_name) + + +def load_repository_from_module_name(module_name, fn_name): + module = importlib.import_module(module_name) + return _create_repo_from_module(module, fn_name) + + def reload_repository_info(repository_info): check.inst_param(repository_info, 'repository_info', RepositoryInfo) diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_six/repository.yml b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_six/repository.yml new file mode 100644 index 0000000000000..9d9c25dfff85a --- /dev/null +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_six/repository.yml @@ -0,0 +1,3 @@ +repository: + file: test_intro_tutorial_part_six.py + fn: define_part_six_repo diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_six/test_intro_tutorial_part_six.py b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_six/test_intro_tutorial_part_six.py new file mode 100644 index 0000000000000..19ed45a9e2b5a --- /dev/null +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_six/test_intro_tutorial_part_six.py @@ -0,0 +1,23 @@ +# pylint: disable=W0622,W0614,W0401 +from dagster import * + + +@lambda_solid +def hello_world(): + pass + + +def define_part_six_pipeline(): + return PipelineDefinition(name='part_six', solids=[hello_world]) + + +def define_part_six_repo(): + return RepositoryDefinition( + name='part_six_repo', pipeline_dict={ + 'part_six': define_part_six_pipeline, + } + ) + + +def test_part_six_repo(): + assert define_part_six_repo().get_pipeline('part_six').name == 'part_six' diff --git a/python_modules/dagster/docs/index.rst b/python_modules/dagster/docs/index.rst index 9422237062841..1b843e2808417 100644 --- a/python_modules/dagster/docs/index.rst +++ b/python_modules/dagster/docs/index.rst @@ -26,6 +26,7 @@ Intro Tutorial intro_tutorial/part_three intro_tutorial/part_four intro_tutorial/part_five + intro_tutorial/part_six API Reference diff --git a/python_modules/dagster/docs/intro_tutorial/part_six.rst b/python_modules/dagster/docs/intro_tutorial/part_six.rst new file mode 100644 index 0000000000000..76c84a940c170 --- /dev/null +++ b/python_modules/dagster/docs/intro_tutorial/part_six.rst @@ -0,0 +1,88 @@ +Repositories and Tools +---------------------- + +Dagster is a not just a programming model for pipeliens, it is also a platform for +tool-building. Included within dagster is a CLI tool, the first tool you'll use +to interact with pipelines. + +In order for a tool to interact with a pipeline, it must be made aware of the pipelines. + +Dagster has a concept of a repositories which is used to organize a collection of pipelines +(and eventually other concepts). Dagster tools are pointed a repository or many repositories +into order to function. + +Repostories are declared like the following: + +.. code-block:: python + + from dagster import * + + @lambda_solid + def hello_world(): + pass + + + def define_part_six_pipeline(): + return PipelineDefinition(name='part_six', solids=[hello_world]) + + + def define_part_six_repo(): + return RepositoryDefinition( + name='part_six_repo', + pipeline_dict={ + 'part_six': define_part_six_pipeline, + }, + ) + +Save this file as "part_six.py" + +For tools to operate they must be able to access and create a repo. The current mechanism is to +create a yaml file (default name: 'repository.yml') where you state the module or file and the +function that creates the repository you want to operate on. + +.. code-block:: yaml + + repository: + file: part_six.py + fn: define_part_six_repo + +Save this file as "repository.yml" + +Now you should be able to list the pipelines in this repo: + +.. code-block:: sh + + $ dagster pipeline list + Repository part_six_repo + Pipeline: part_six + Solids: (Execution Order) + hello_world + + +For a given project with 1 or more pipelines, you should define a repository for that +project so it is accessible via tools. If you are writing code within a python module +as opposed to a standalone file you can (and should) load the repository via that module. + +To do that you can specify a module instead of a file in the repository.yml file. + +.. code-block:: yaml + + # this works if you this works in your python enviroment + # from some_module import define_some_repository + repository: + module: some_module + fn: define_some_repository + +Dagit +^^^^^ + +Dagit is an essential tool for managaging and visualizing dagster pipelines. Install dagit +via pypi (``pip install dagit``) and then run it on your repository in the same way as +the CLI tool. + +.. code-block:: sh + + $ dagit + Serving on http://localhost:3000 + +Now navigate to http:://localhost:3000 in your web browser and you can visualize your pipelines! From 915e69c08c631127e8fb6852e7a8194e85906248 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Sun, 9 Sep 2018 10:48:37 -0700 Subject: [PATCH 073/103] fix typo --- python_modules/dagster/docs/intro_tutorial/part_six.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python_modules/dagster/docs/intro_tutorial/part_six.rst b/python_modules/dagster/docs/intro_tutorial/part_six.rst index 76c84a940c170..64b34e39f8434 100644 --- a/python_modules/dagster/docs/intro_tutorial/part_six.rst +++ b/python_modules/dagster/docs/intro_tutorial/part_six.rst @@ -1,7 +1,7 @@ Repositories and Tools ---------------------- -Dagster is a not just a programming model for pipeliens, it is also a platform for +Dagster is a not just a programming model for pipelines, it is also a platform for tool-building. Included within dagster is a CLI tool, the first tool you'll use to interact with pipelines. From c9ad242d82d06d251ebefc07dd71b797edb5cf64 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Sun, 9 Sep 2018 11:34:02 -0700 Subject: [PATCH 074/103] Add CLI-driven execution tutorial part Also made metrics execution time collection a little more accurate --- .../dagster/dagster/cli/pipeline.py | 3 +- .../dagster/dagster/core/compute_nodes.py | 20 ++++- .../test_intro_tutorial_part_seven/env.yml | 8 ++ .../repository.yml | 3 + .../test_intro_tutorial_part_seven.py | 55 +++++++++++++ python_modules/dagster/docs/index.rst | 1 + .../docs/intro_tutorial/part_seven.rst | 79 +++++++++++++++++++ 7 files changed, 164 insertions(+), 5 deletions(-) create mode 100644 python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_seven/env.yml create mode 100644 python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_seven/repository.yml create mode 100644 python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_seven/test_intro_tutorial_part_seven.py create mode 100644 python_modules/dagster/docs/intro_tutorial/part_seven.rst diff --git a/python_modules/dagster/dagster/cli/pipeline.py b/python_modules/dagster/dagster/cli/pipeline.py index 2b40dc92f0bc4..b3552b779dde6 100644 --- a/python_modules/dagster/dagster/cli/pipeline.py +++ b/python_modules/dagster/dagster/cli/pipeline.py @@ -240,7 +240,8 @@ def process_results_for_console(pipeline_iter, printer): click_context.exit(1) results.append(result) - print_metrics_to_console(results, printer) + # Commenting this out for now + # print_metrics_to_console(results, printer) def print_metrics_to_console(results, printer): diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index fd94011d2be79..fc7eed884a0a7 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -206,6 +206,17 @@ def _yield_transform_results(context, compute_node, conf, inputs): ) yield result + +def _collect_result_list(context, compute_node, conf, inputs): + for result in _yield_transform_results(context, compute_node, conf, inputs): + context.debug('Solid {solid} emitted output "{output}" value {value}'.format( + solid=compute_node.solid.name, + output=result.output_name, + value=repr(result.value), + )) + yield result + + def _execute_core_transform(context, compute_node, conf, inputs): ''' Execute the user-specified transform for the solid. Wrap in an error boundary and do @@ -228,12 +239,13 @@ def _execute_core_transform(context, compute_node, conf, inputs): with time_execution_scope() as timer_result, \ _user_code_error_boundary(context, error_str): - for result in _yield_transform_results(context, compute_node, conf, inputs): - yield result - + all_results = list(_collect_result_list(context, compute_node, conf, inputs)) context.metric('core_transform_time_ms', timer_result.millis) + for result in all_results: + yield result + class ComputeNodeInput(object): def __init__(self, name, dagster_type, prev_output_handle): @@ -399,7 +411,7 @@ def execute_compute_nodes(context, compute_nodes): check.list_param(compute_nodes, 'compute_nodes', of_type=ComputeNode) intermediate_results = {} - context.debug('Entering exeute_compute_nodes loop. Order: {order}'.format( + context.debug('Entering execute_compute_nodes loop. Order: {order}'.format( order=[cn.friendly_name for cn in compute_nodes] )) diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_seven/env.yml b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_seven/env.yml new file mode 100644 index 0000000000000..e70f2f5755715 --- /dev/null +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_seven/env.yml @@ -0,0 +1,8 @@ +context: + config: + log_level: DEBUG + +solids: + double_the_word: + config: + word: bar diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_seven/repository.yml b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_seven/repository.yml new file mode 100644 index 0000000000000..e1cbfdc36e1e1 --- /dev/null +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_seven/repository.yml @@ -0,0 +1,3 @@ +repository: + file: test_intro_tutorial_part_seven.py + fn: define_part_seven_repo diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_seven/test_intro_tutorial_part_seven.py b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_seven/test_intro_tutorial_part_seven.py new file mode 100644 index 0000000000000..fa428d629c36c --- /dev/null +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_seven/test_intro_tutorial_part_seven.py @@ -0,0 +1,55 @@ +# pylint: disable=W0622,W0614,W0401 +from collections import defaultdict + +from dagster import * + + +@solid +def double_the_word(_context, conf): + return conf['word'] * 2 + + +@lambda_solid(inputs=[InputDefinition('word')]) +def count_letters(word): + counts = defaultdict(int) + for letter in word: + counts[letter] += 1 + return dict(counts) + + +def define_part_seven_pipeline(): + return PipelineDefinition( + name='part_seven', + solids=[double_the_word, count_letters], + dependencies={ + 'count_letters': { + 'word': DependencyDefinition('double_the_word'), + }, + }, + ) + + +def define_part_seven_repo(): + return RepositoryDefinition( + name='part_seven_repo', pipeline_dict={ + 'part_seven': define_part_seven_pipeline, + } + ) + + +def test_part_seven_repo(): + environment = config.Environment( + solids={ + 'double_the_word': config.Solid(config={'word': 'bar'}), + } + ) + + pipeline_result = execute_pipeline(define_part_seven_pipeline(), environment) + + assert pipeline_result.success + assert pipeline_result.result_for_solid('double_the_word').transformed_value() == 'barbar' + assert pipeline_result.result_for_solid('count_letters').transformed_value() == { + 'b': 2, + 'a': 2, + 'r': 2, + } diff --git a/python_modules/dagster/docs/index.rst b/python_modules/dagster/docs/index.rst index 1b843e2808417..d5f7080fda818 100644 --- a/python_modules/dagster/docs/index.rst +++ b/python_modules/dagster/docs/index.rst @@ -27,6 +27,7 @@ Intro Tutorial intro_tutorial/part_four intro_tutorial/part_five intro_tutorial/part_six + intro_tutorial/part_seven API Reference diff --git a/python_modules/dagster/docs/intro_tutorial/part_seven.rst b/python_modules/dagster/docs/intro_tutorial/part_seven.rst new file mode 100644 index 0000000000000..034fbcc369a1c --- /dev/null +++ b/python_modules/dagster/docs/intro_tutorial/part_seven.rst @@ -0,0 +1,79 @@ +Pipeline Execution +------------------ + +So far we've been driving execution of pipelines from python APIs. Now it is +time to instead drive these from the command line. + +Similar to the the part six tutorial, we are going to create a pipeline, a repository, +and a yaml file so that the CLI tool can know about the repository. + +.. code-block:: python + + from dagster import * + + @solid + def double_the_word(_context, conf): + return conf['config_key'] * 2 + + @lambda_solid(inputs=[InputDefinition('word')]) + def count_letters(word): + counts = defaultdict(int) + for letter in word: + counts[letter] += 1 + return dict(counts) + + def define_part_seven_pipeline(): + return PipelineDefinition( + name='part_seven', + solids=[double_the_word, count_letters], + dependencies={ + 'count_letters': { + 'word': DependencyDefinition('double_the_word'), + }, + }, + ) + + def define_part_seven_repo(): + return RepositoryDefinition( + name='part_seven_repo', + pipeline_dict={ + 'part_seven': define_part_seven_pipeline, + }, + ) +And now the repository file: + +.. code-block:: yaml + + repository: + file: part_seven.py + fn: define_part_seven_repo + +Now we want to execute it from the command line. In order to do that we need to create a yaml file +with all the elements we need to create an environment. The form of this file is very similar +to the in-memory ``config.Environment`` and related objects that were used in previous steps +in the tutorial. + +.. code-block:: yaml + + context: + config: + log_level: DEBUG + + solids: + double_the_word: + config: + word: bar + +With these elements in place we can now drive execution from the CLI + +.. code-block:: sh + + dagster pipeline execute part_seven -e env.yml + 2018-09-09 11:47:38 - dagster - DEBUG - message="About to execute the compute node graph in the following order ['double_the_word.transform', 'count_letters.transform']" + 2018-09-09 11:47:38 - dagster - DEBUG - message="Entering execute_compute_nodes loop. Order: ['double_the_word.transform', 'count_letters.transform']" + 2018-09-09 11:47:38 - dagster - DEBUG - message="Executing core transform for solid double_the_word." solid=double_the_word + 2018-09-09 11:47:38 - dagster - DEBUG - message="Solid double_the_word emitted output "result" value 'barbar'" solid=double_the_word + 2018-09-09 11:47:38 - dagster - INFO - metric:core_transform_time_ms=0.136 solid=double_the_word + 2018-09-09 11:47:38 - dagster - DEBUG - message="Executing core transform for solid count_letters." solid=count_letters + 2018-09-09 11:47:38 - dagster - DEBUG - message="Solid count_letters emitted output "result" value {'b': 2, 'a': 2, 'r': 2}" solid=count_letters + 2018-09-09 11:47:38 - dagster - INFO - metric:core_transform_time_ms=0.130 solid=count_letters \ No newline at end of file From 50d865161d89b4eac8318cf08edef24ddd8f0486 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Sun, 9 Sep 2018 14:34:35 -0700 Subject: [PATCH 075/103] Logging improvements 1) Make message and value escaping by using json.dumps to wrap all string with proper quotes when they are used to log things in a semi-structured key=value pattern. 2) Refactor core logging code to make relationship between automatic values (like log_message_id), context values, and message values more clear. --- .../core/core_tests/test_context_logging.py | 25 +++- .../dagster/dagster/core/execution_context.py | 141 ++++++++++++------ .../test_sql_project_pipeline.py | 17 ++- .../dagster/dagster/utils/logging.py | 2 +- python_modules/dagster/setup.py | 4 +- 5 files changed, 132 insertions(+), 57 deletions(-) diff --git a/python_modules/dagster/dagster/core/core_tests/test_context_logging.py b/python_modules/dagster/dagster/core/core_tests/test_context_logging.py index 8d45a5a291aa7..af6d624cbaa1c 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_context_logging.py +++ b/python_modules/dagster/dagster/core/core_tests/test_context_logging.py @@ -46,7 +46,7 @@ def test_test_logger(): def orig_message(message): - return message.extra['log_message'] + return message.extra['orig_message'] def test_context_logging(): @@ -80,7 +80,7 @@ def test_context_value(): context.info('some message') assert logger.messages[0].extra['some_key'] == 'some_value' - assert 'some_key=some_value' in logger.messages[0].msg + assert 'some_key="some_value"' in logger.messages[0].msg assert 'message="some message"' in logger.messages[0].msg @@ -104,11 +104,26 @@ def test_interleaved_context_value(): message_one = logger.messages[0] assert message_one.extra['key_one'] == 'value_one' assert 'key_two' not in message_one.extra - assert 'key_one=value_one' in message_one.msg + assert 'key_one="value_one"' in message_one.msg assert 'key_two' not in message_one.msg message_two = logger.messages[1] assert message_two.extra['key_one'] == 'value_one' assert message_two.extra['key_two'] == 'value_two' - assert 'key_one=value_one' in message_two.msg - assert 'key_two=value_two' in message_two.msg + assert 'key_one="value_one"' in message_two.msg + assert 'key_two="value_two"' in message_two.msg + + +def test_message_specific_logging(): + logger = LoggerForTest() + context = ExecutionContext(loggers=[logger]) + with context.value('key_one', 'value_one'): + context.info('message one', key_two='value_two') + + message_one = logger.messages[0] + assert message_one.extra['key_one'] == 'value_one' + assert message_one.extra['key_two'] == 'value_two' + + assert set(message_one.extra.keys()) == set( + ['key_one', 'key_two', 'log_message_id', 'orig_message'] + ) diff --git a/python_modules/dagster/dagster/core/execution_context.py b/python_modules/dagster/dagster/core/execution_context.py index 94dc1ec3a4a19..56a9fd0cf43e1 100644 --- a/python_modules/dagster/dagster/core/execution_context.py +++ b/python_modules/dagster/dagster/core/execution_context.py @@ -1,11 +1,13 @@ -import itertools import copy +import itertools +import json +import uuid + from collections import ( OrderedDict, namedtuple, ) from contextlib import contextmanager -import uuid from dagster import check from dagster.utils.logging import CompositeLogger @@ -13,6 +15,12 @@ Metric = namedtuple('Metric', 'context_dict metric_name value') +def _kv_message(all_items): + return ' '.join( + ['{key}={value}'.format(key=key, value=json.dumps(value)) for key, value in all_items] + ) + + class ExecutionContext(object): ''' A context object flowed through the entire scope of single execution of a @@ -25,9 +33,16 @@ class ExecutionContext(object): reporting. - resources is an arbitrary user-defined object that can be passed in - by a user and then access during pipeline execution. This exists so that - a user does not have to subclass ExecutionContext + Args: + loggers (List[logging.Logger]): + The list of loggers that will be invoked whenever the context logging + functions are called. + + resources(Any): + An arbitrary user-defined object that can be passed in by a user and + then access during pipeline execution. This exists so that a user can + inject their own objects into the context without having to subclass + ExecutionContext. ''' def __init__(self, loggers=None, resources=None): @@ -36,61 +51,85 @@ def __init__(self, loggers=None, resources=None): self._metrics = [] self.resources = resources - def _maybe_quote(self, val): - str_val = str(val) - if ' ' in str_val: - return '"{val}"'.format(val=str_val) - return str_val - - def _kv_message(self, extra=None): - extra = check.opt_dict_param(extra, 'extra') - return ' '.join( - [ - '{key}={value}'.format(key=key, value=self._maybe_quote(value)) - for key, value in itertools.chain(self._context_dict.items(), extra.items()) - ] + def _log(self, method, orig_message, message_props): + check.str_param(method, 'method') + check.str_param(orig_message, 'orig_message') + check.dict_param(message_props, 'message_props') + + check.invariant( + 'extra' not in message_props, + 'do not allow until explicit support is handled', + ) + check.invariant( + 'exc_info' not in message_props, + 'do not allow until explicit support is handled', ) - def _log(self, method, msg, **kwargs): - check.str_param(method, 'method') - check.str_param(msg, 'msg') + check.invariant('orig_message' not in message_props, 'orig_message reserved value') + check.invariant('message' not in message_props, 'message reserved value') + check.invariant('log_message_id' not in message_props, 'log_message_id reserved value') - check.invariant('extra' not in kwargs, 'do not allow until explicit support is handled') - check.invariant('exc_info' not in kwargs, 'do not allow until explicit support is handled') + log_message_id = str(uuid.uuid4()) - check.invariant('log_message' not in kwargs, 'log_message_id reserved value') - check.invariant('log_message_id' not in kwargs, 'log_message_id reserved value') + synth_props = {'orig_message': orig_message, 'log_message_id': log_message_id} - full_message = 'message="{message}" {kv_message}'.format( - message=msg, kv_message=self._kv_message(kwargs) + # We first generate all props for the purpose of producing the semi-structured + # log message via _kv_messsage + all_props = dict( + itertools.chain( + synth_props.items(), + self._context_dict.items(), + message_props.items(), + ) ) - log_props = copy.copy(self._context_dict) - log_props['log_message'] = msg + message_with_structured_props = _kv_message(all_props.items()) - log_props['log_message'] = msg - log_props['log_message_id'] = str(uuid.uuid4()) + getattr(self._logger, method)(message_with_structured_props, extra=all_props) - extra = {} - extra.update(log_props) - extra.update(kwargs) + def debug(self, msg, **kwargs): + ''' + Debug level logging directive. Ends up invoking loggers with DEBUG error level. - getattr(self._logger, method)(full_message, extra=extra) + The message will be automatically adorned with context information about the name + of the pipeline, the name of the solid, and so forth. The use can also add + context values during execution using the value() method of ExecutionContext. + Therefore it is generally unnecessary to include this type of information + (solid name, pipeline name, etc) in the log message unless it is critical + for the readability/fluency of the log message text itself. - def debug(self, msg, **kwargs): - return self._log('debug', msg, **kwargs) + You can optionally additional context key-value pairs to an individual log + message using the keyword args to this message + + Args: + msg (str): The core string + **kwargs (Dict[str, Any]): Additional context values for only this log message. + ''' + return self._log('debug', msg, kwargs) def info(self, msg, **kwargs): - return self._log('info', msg, **kwargs) + '''Log at INFO level + + See debug()''' + return self._log('info', msg, kwargs) def warning(self, msg, **kwargs): - return self._log('warning', msg, **kwargs) + '''Log at WARNING level + + See debug()''' + return self._log('warning', msg, kwargs) def error(self, msg, **kwargs): - return self._log('error', msg, **kwargs) + '''Log at ERROR level + + See debug()''' + return self._log('error', msg, kwargs) def critical(self, msg, **kwargs): - return self._log('critical', msg, **kwargs) + '''Log at CRITICAL level + + See debug()''' + return self._log('critical', msg, kwargs) # FIXME: Actually make this work # def exception(self, e): @@ -101,6 +140,21 @@ def critical(self, msg, **kwargs): @contextmanager def value(self, key, value): + ''' + Adds a context value to the Execution for a particular scope, using the + python contextmanager abstraction. This allows the user to add scoped metadata + just like the framework does (for things such as solid name). + + Examples: + + .. code-block:: python + + with context.value('some_key', 'some_value): + context.info('msg with some_key context value') + + context.info('msg without some_key context value') + + ''' check.str_param(key, 'key') check.not_none_param(value, 'value') @@ -122,10 +176,13 @@ def metric(self, metric_name, value): format_string = 'metric:{metric_name}={value:.3f} {kv_message}' else: format_string = 'metric:{metric_name}={value} {kv_message}' + value = json.dumps(value) self._logger.info( format_string.format( - metric_name=metric_name, value=value, kv_message=self._kv_message() + metric_name=metric_name, + value=value, + kv_message=_kv_message(self._context_dict.items()), ), extra=self._context_dict ) diff --git a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py index fc6eb5f961728..8374d9805308b 100644 --- a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py +++ b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py @@ -146,11 +146,12 @@ def test_full_in_memory_pipeline(): assert engine.execute('SELECT * FROM sum_sq_table').fetchall() == [(1, 2, 3, 9), (3, 4, 7, 49)] -def test_full_persisted_pipeline(): - pipeline = create_full_pipeline() - pipeline_result = execute_pipeline( - pipeline, - environment=config.Environment(context=config.Context(name='persisted')), - ) - - assert pipeline_result.success +# Commmenting out for now because it takes two seconds +# def test_full_persisted_pipeline(): +# pipeline = create_full_pipeline() +# pipeline_result = execute_pipeline( +# pipeline, +# environment=config.Environment(context=config.Context(name='persisted')), +# ) + +# assert pipeline_result.success diff --git a/python_modules/dagster/dagster/utils/logging.py b/python_modules/dagster/dagster/utils/logging.py index f9399b7913c02..a92c26bfd724b 100644 --- a/python_modules/dagster/dagster/utils/logging.py +++ b/python_modules/dagster/dagster/utils/logging.py @@ -110,5 +110,5 @@ def get_formatted_stack_trace(exception): tb = exception.__traceback__ else: import sys - exc_type, exc_value, tb = sys.exc_info() + _exc_type, _exc_value, tb = sys.exc_info() return ''.join(traceback.format_tb(tb)) diff --git a/python_modules/dagster/setup.py b/python_modules/dagster/setup.py index 0d9c72392779b..06bfcc1ca7d09 100644 --- a/python_modules/dagster/setup.py +++ b/python_modules/dagster/setup.py @@ -18,7 +18,7 @@ def long_description(): setup( name='dagster', - version='0.2.0.dev9', + version='0.2.0.dev10', author='Elementl', author_email='schrockn@elementl.com', license='Apache-2.0', @@ -27,6 +27,8 @@ def long_description(): long_description_content_type='text/markdown', url='https://github.com/dagster-io/dagster', classifiers=( + 'Programming Language :: Python :: 2.7', + 'Programming Language :: Python :: 3.5', 'Programming Language :: Python :: 3.6', 'License :: OSI Approved :: Apache Software License', 'Operating System :: OS Independent', From c4110a833761958174846db3d398469e09bb9665 Mon Sep 17 00:00:00 2001 From: Mikhail Novikov Date: Mon, 10 Sep 2018 12:43:40 +0300 Subject: [PATCH 076/103] Add single sourced versions --- python_modules/dagit/dagit/__init__.py | 1 + python_modules/dagit/dagit/version.py | 1 + python_modules/dagit/setup.py | 4 ++++ python_modules/dagster/dagster/__init__.py | 2 ++ python_modules/dagster/dagster/version.py | 1 + python_modules/dagster/setup.py | 6 +++++- 6 files changed, 14 insertions(+), 1 deletion(-) create mode 100644 python_modules/dagit/dagit/version.py create mode 100644 python_modules/dagster/dagster/version.py diff --git a/python_modules/dagit/dagit/__init__.py b/python_modules/dagit/dagit/__init__.py index e69de29bb2d1d..58f3ace6c03d0 100644 --- a/python_modules/dagit/dagit/__init__.py +++ b/python_modules/dagit/dagit/__init__.py @@ -0,0 +1 @@ +from .version import __version__ diff --git a/python_modules/dagit/dagit/version.py b/python_modules/dagit/dagit/version.py new file mode 100644 index 0000000000000..efbf9fb8ea7b5 --- /dev/null +++ b/python_modules/dagit/dagit/version.py @@ -0,0 +1 @@ +__version__ = '0.2.0.dev3' diff --git a/python_modules/dagit/setup.py b/python_modules/dagit/setup.py index 900e17f27d300..f6973a83e6654 100644 --- a/python_modules/dagit/setup.py +++ b/python_modules/dagit/setup.py @@ -16,6 +16,10 @@ def long_description(): return fh.read() +version = {} +with open("dagit/version.py") as fp: + exec(fp.read(), version) + setup( name='dagit', version='0.2.0.dev3', diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index 94995893364e6..62446272dc429 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -7,6 +7,8 @@ from dagster.core.execution_context import ExecutionContext +from .version import __version__ + from dagster.core.definitions import ( ConfigDefinition, DependencyDefinition, diff --git a/python_modules/dagster/dagster/version.py b/python_modules/dagster/dagster/version.py new file mode 100644 index 0000000000000..312d10d3004c4 --- /dev/null +++ b/python_modules/dagster/dagster/version.py @@ -0,0 +1 @@ +__version__ = '0.2.0.dev10' diff --git a/python_modules/dagster/setup.py b/python_modules/dagster/setup.py index 06bfcc1ca7d09..5b220646cc237 100644 --- a/python_modules/dagster/setup.py +++ b/python_modules/dagster/setup.py @@ -16,9 +16,13 @@ def long_description(): return fh.read() +version = {} +with open("dagster/version.py") as fp: + exec(fp.read(), version) + setup( name='dagster', - version='0.2.0.dev10', + version=version['__version__'], author='Elementl', author_email='schrockn@elementl.com', license='Apache-2.0', From 087d95da015af22a50d21c3aa80e8079b3d887ff Mon Sep 17 00:00:00 2001 From: Mikhail Novikov Date: Mon, 10 Sep 2018 14:39:01 +0300 Subject: [PATCH 077/103] Replace IncomingValue with values and exceptions --- .../dagster/dagster/core/compute_nodes.py | 64 +++---- .../core_tests/test_config_type_system.py | 157 +++++++++--------- .../dagster/dagster/core/execution.py | 9 +- python_modules/dagster/dagster/core/types.py | 53 ++---- 4 files changed, 125 insertions(+), 158 deletions(-) diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index fc7eed884a0a7..33f1aaab8a57c 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -72,13 +72,17 @@ def __new__(cls, compute_node, output_name): def __str__(self): return ( 'ComputeNodeOutputHandle' - '(cn="{cn.compute_node.friendly_name}", output_name="{cn.output_name}")'.format(cn=self) + '(cn="{cn.compute_node.friendly_name}", output_name="{cn.output_name}")'.format( + cn=self + ) ) def __repr__(self): return ( 'ComputeNodeOutputHandle' - '(cn="{cn.compute_node.friendly_name}", output_name="{cn.output_name}")'.format(cn=self) + '(cn="{cn.compute_node.friendly_name}", output_name="{cn.output_name}")'.format( + cn=self + ) ) def __hash__(self): @@ -209,11 +213,13 @@ def _yield_transform_results(context, compute_node, conf, inputs): def _collect_result_list(context, compute_node, conf, inputs): for result in _yield_transform_results(context, compute_node, conf, inputs): - context.debug('Solid {solid} emitted output "{output}" value {value}'.format( - solid=compute_node.solid.name, - output=result.output_name, - value=repr(result.value), - )) + context.debug( + 'Solid {solid} emitted output "{output}" value {value}'.format( + solid=compute_node.solid.name, + output=result.output_name, + value=repr(result.value), + ) + ) yield result @@ -228,12 +234,9 @@ def _execute_core_transform(context, compute_node, conf, inputs): error_str = 'Error occured during core transform' - with context.value('solid', compute_node.solid.name): context.debug( - 'Executing core transform for solid {solid}.'.format( - solid=compute_node.solid.name - ) + 'Executing core transform for solid {solid}.'.format(solid=compute_node.solid.name) ) with time_execution_scope() as timer_result, \ @@ -301,14 +304,15 @@ def _create_compute_node_result(self, result): node_output = self.node_named(result.output_name) - evaluation_result = node_output.dagster_type.evaluate_value(result.value) - if not evaluation_result.success: + try: + evaluation_result = node_output.dagster_type.evaluate_value(result.value) + except DagsterTypeError as e: raise DagsterInvariantViolationError( '''Solid {cn.solid.name} output name {output_name} output {result.value} type failure: {error_msg}'''.format( cn=self, result=result, - error_msg=evaluation_result.error_msg, + error_msg=','.join(e.args), output_name=result.output_name, ) ) @@ -318,7 +322,7 @@ def _create_compute_node_result(self, result): tag=self.tag, success_data=ComputeNodeSuccessData( output_name=result.output_name, - value=evaluation_result.value, + value=evaluation_result, ), ) @@ -411,9 +415,11 @@ def execute_compute_nodes(context, compute_nodes): check.list_param(compute_nodes, 'compute_nodes', of_type=ComputeNode) intermediate_results = {} - context.debug('Entering execute_compute_nodes loop. Order: {order}'.format( - order=[cn.friendly_name for cn in compute_nodes] - )) + context.debug( + 'Entering execute_compute_nodes loop. Order: {order}'.format( + order=[cn.friendly_name for cn in compute_nodes] + ) + ) for compute_node in compute_nodes: if not _all_inputs_covered(compute_node, intermediate_results): @@ -421,13 +427,11 @@ def execute_compute_nodes(context, compute_nodes): expected_outputs = [ni.prev_output_handle for ni in compute_node.node_inputs] context.debug( - 'Not all inputs covered for {compute_name}. Not executing.'.format( - compute_name=compute_node.friendly_name - ) + '\nKeys in result: {result_keys}.'.format( - result_keys=result_keys, - ) + '\nOutputs need for inputs {expected_outputs}'.format( - expected_outputs=expected_outputs, - ) + 'Not all inputs covered for {compute_name}. Not executing.'. + format(compute_name=compute_node.friendly_name) + + '\nKeys in result: {result_keys}.'.format(result_keys=result_keys, ) + + '\nOutputs need for inputs {expected_outputs}'. + format(expected_outputs=expected_outputs, ) ) continue @@ -574,14 +578,13 @@ def create_conf_value(execution_info, solid): solid_configs = execution_info.environment.solids config_input = solid_configs[name].config if name in solid_configs else {} - evaluation_result = solid.config_def.config_type.evaluate_value(config_input) - if evaluation_result.success: - return evaluation_result.value - else: + try: + return solid.config_def.config_type.evaluate_value(config_input) + except DagsterTypeError as e: raise DagsterTypeError( 'Error evaluating config for {solid_name}: {error_msg}'.format( solid_name=solid.name, - error_msg=evaluation_result.error_msg, + error_msg=','.join(e.args), ) ) @@ -647,7 +650,6 @@ def create_compute_node_graph(execution_info): output_handle = topo_solid.output_handle(output_def.name) cn_output_node_map[output_handle] = subgraph.terminal_cn_output_handle - return _create_compute_node_graph(compute_nodes) diff --git a/python_modules/dagster/dagster/core/core_tests/test_config_type_system.py b/python_modules/dagster/dagster/core/core_tests/test_config_type_system.py index db73ce6a513c9..8806d10338af2 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_config_type_system.py +++ b/python_modules/dagster/dagster/core/core_tests/test_config_type_system.py @@ -1,7 +1,9 @@ +import pytest from dagster import ( ConfigDefinition, types, Field, + DagsterTypeError, ) @@ -14,7 +16,7 @@ def test_int_field(): 'int_field': Field(types.Int), }) - assert config_def.config_type.evaluate_value({'int_field': 1}).value == {'int_field': 1} + assert config_def.config_type.evaluate_value({'int_field': 1}) == {'int_field': 1} def test_int_fails(): @@ -22,9 +24,11 @@ def test_int_fails(): 'int_field': Field(types.Int), }) - assert not config_def.config_type.evaluate_value({'int_field': 'fjkdj'}).success + with pytest.raises(DagsterTypeError): + config_def.config_type.evaluate_value({'int_field': 'fjkdj'}) - assert not config_def.config_type.evaluate_value({'int_field': True}).success + with pytest.raises(DagsterTypeError): + config_def.config_type.evaluate_value({'int_field': True}) def test_default_arg(): @@ -34,7 +38,7 @@ def test_default_arg(): } ) - assert config_def.config_type.evaluate_value({}).value == {'int_field': 2} + assert config_def.config_type.evaluate_value({}) == {'int_field': 2} def _single_required_string_config_dict(): @@ -82,21 +86,24 @@ def _validate(config_def, value): def test_single_required_string_field_config_type(): - assert _validate(_single_required_string_config_dict(), {'string_field': 'value'}).success - assert _validate(_single_required_string_config_dict(), {'string_field': None}).success + assert _validate(_single_required_string_config_dict(), {'string_field': 'value'}) == { + 'string_field': 'value' + } + assert _validate(_single_required_string_config_dict(), {'string_field': None}) == { + 'string_field': None + } - assert not _validate(_single_required_string_config_dict(), {}).success + with pytest.raises(DagsterTypeError): + _validate(_single_required_string_config_dict(), {}) - assert not _validate(_single_required_string_config_dict(), {'extra': 'yup'}).success + with pytest.raises(DagsterTypeError): + _validate(_single_required_string_config_dict(), {'extra': 'yup'}) - assert not _validate( - _single_required_string_config_dict(), { - 'string_field': 'yupup', - 'extra': 'yup' - } - ).success + with pytest.raises(DagsterTypeError): + _validate(_single_required_string_config_dict(), {'string_field': 'yupup', 'extra': 'yup'}) - assert not _validate(_single_required_string_config_dict(), {'string_field': 1}).success + with pytest.raises(DagsterTypeError): + _validate(_single_required_string_config_dict(), {'string_field': 1}) def test_multiple_required_fields_passing(): @@ -106,67 +113,70 @@ def test_multiple_required_fields_passing(): 'field_one': 'value_one', 'field_two': 'value_two', }, - ).success + ) == { + 'field_one': 'value_one', + 'field_two': 'value_two', + } + assert _validate( _multiple_required_fields_config_dict(), { 'field_one': 'value_one', 'field_two': None, }, - ).success + ) == { + 'field_one': 'value_one', + 'field_two': None, + } def test_multiple_required_fields_failing(): - assert not _validate(_multiple_required_fields_config_dict(), {}).success + with pytest.raises(DagsterTypeError): + _validate(_multiple_required_fields_config_dict(), {}) - assert not _validate(_multiple_required_fields_config_dict(), {'field_one': 'yup'}).success + with pytest.raises(DagsterTypeError): + _validate(_multiple_required_fields_config_dict(), {'field_one': 'yup'}) - assert not _validate( - _multiple_required_fields_config_dict(), { - 'field_one': 'yup', - 'extra': 'yup' - } - ).success + with pytest.raises(DagsterTypeError): + _validate(_multiple_required_fields_config_dict(), {'field_one': 'yup', 'extra': 'yup'}) - assert not _validate( - _multiple_required_fields_config_dict(), { - 'field_one': 'value_one', - 'field_two': 2 - } - ).success + with pytest.raises(DagsterTypeError): + _validate( + _multiple_required_fields_config_dict(), { + 'field_one': 'value_one', + 'field_two': 2 + } + ) def test_single_optional_field_passing(): - assert _validate(_single_optional_string_config_dict(), { - 'optional_field': 'value' - }).value == { + assert _validate(_single_optional_string_config_dict(), {'optional_field': 'value'}) == { 'optional_field': 'value' } - assert _validate(_single_optional_string_config_dict(), {}).value == {} + assert _validate(_single_optional_string_config_dict(), {}) == {} - assert _validate(_single_optional_string_config_dict(), { - 'optional_field': None - }).value == { + assert _validate(_single_optional_string_config_dict(), {'optional_field': None}) == { 'optional_field': None } def test_single_optional_field_failing(): - assert not _validate(_single_optional_string_config_dict(), {'optional_field': 1}).success + with pytest.raises(DagsterTypeError): + + _validate(_single_optional_string_config_dict(), {'optional_field': 1}) - assert not _validate(_single_optional_string_config_dict(), {'dlkjfalksdjflksaj': 1}).success + with pytest.raises(DagsterTypeError): + _validate(_single_optional_string_config_dict(), {'dlkjfalksdjflksaj': 1}) def test_single_optional_field_passing_with_default(): - assert _validate(_single_optional_string_field_config_dict_with_default(), {}).value == { + assert _validate(_single_optional_string_field_config_dict_with_default(), {}) == { 'optional_field': 'some_default' } assert _validate( - _single_optional_string_field_config_dict_with_default(), { - 'optional_field': 'override' - } - ).value == { + _single_optional_string_field_config_dict_with_default(), {'optional_field': 'override'} + ) == { 'optional_field': 'override' } @@ -177,7 +187,7 @@ def test_mixed_args_passing(): 'optional_arg': 'value_one', 'required_arg': 'value_two', } - ).value == { + ) == { 'optional_arg': 'value_one', 'required_arg': 'value_two', } @@ -186,7 +196,7 @@ def test_mixed_args_passing(): _mixed_required_optional_string_config_dict_with_default(), { 'required_arg': 'value_two', } - ).value == { + ) == { 'optional_arg': 'some_default', 'required_arg': 'value_two', } @@ -196,7 +206,7 @@ def test_mixed_args_passing(): 'required_arg': 'value_two', 'optional_arg_no_default': 'value_three', } - ).value == { + ) == { 'optional_arg': 'some_default', 'required_arg': 'value_two', 'optional_arg_no_default': 'value_three', @@ -257,11 +267,9 @@ def _nested_optional_config_with_no_default(): def test_single_nested_config(): - assert _validate(_single_nested_config(), { - 'nested': { - 'int_field': 2 - } - }).value == { + assert _validate(_single_nested_config(), {'nested': { + 'int_field': 2 + }}) == { 'nested': { 'int_field': 2 } @@ -269,35 +277,26 @@ def test_single_nested_config(): def test_single_nested_config_failures(): - assert not _validate(_single_nested_config(), {'nested': 'dkjfdk'}).success + with pytest.raises(DagsterTypeError): + _validate(_single_nested_config(), {'nested': 'dkjfdk'}) - assert not _validate(_single_nested_config(), {'nested': {'int_field': 'dkjfdk'}}).success + with pytest.raises(DagsterTypeError): + _validate(_single_nested_config(), {'nested': {'int_field': 'dkjfdk'}}) - assert not _validate( - _single_nested_config(), { - 'nested': { - 'int_field': { - 'too_nested': 'dkjfdk' - } - } - } - ).success + with pytest.raises(DagsterTypeError): + _validate(_single_nested_config(), {'nested': {'int_field': {'too_nested': 'dkjfdk'}}}) def test_nested_optional_with_default(): - assert _validate(_nested_optional_config_with_default(), { - 'nested': { - 'int_field': 2 - } - }).value == { + assert _validate(_nested_optional_config_with_default(), {'nested': { + 'int_field': 2 + }}) == { 'nested': { 'int_field': 2 } } - assert _validate(_nested_optional_config_with_default(), { - 'nested': {} - }).value == { + assert _validate(_nested_optional_config_with_default(), {'nested': {}}) == { 'nested': { 'int_field': 3 } @@ -305,18 +304,12 @@ def test_nested_optional_with_default(): def test_nested_optional_with_no_default(): - assert _validate(_nested_optional_config_with_no_default(), { - 'nested': { - 'int_field': 2 - } - }).value == { + assert _validate(_nested_optional_config_with_no_default(), {'nested': { + 'int_field': 2 + }}) == { 'nested': { 'int_field': 2 } } - assert _validate(_nested_optional_config_with_no_default(), { - 'nested': {} - }).value == { - 'nested': {} - } + assert _validate(_nested_optional_config_with_no_default(), {'nested': {}}) == {'nested': {}} diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index 01bd7ad05eb13..510039cc163e7 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -227,13 +227,14 @@ def yield_context(pipeline, environment): context_definition = pipeline.context_definitions[context_name] config_type = context_definition.config_def.config_type - evaluation_result = config_type.evaluate_value(environment.context.config) - if not evaluation_result.success: + try: + evaluation_result = config_type.evaluate_value(environment.context.config) + except DagsterTypeError as e: raise DagsterTypeError( - 'Invalid config value: {error_msg}'.format(error_msg=evaluation_result.error_msg) + 'Invalid config value: {error_msg}'.format(error_msg=','.join(e.args)) ) - thing = context_definition.context_fn(pipeline, evaluation_result.value) + thing = context_definition.context_fn(pipeline, evaluation_result) return _wrap_in_yield(thing) diff --git a/python_modules/dagster/dagster/core/types.py b/python_modules/dagster/dagster/core/types.py index 08f3a2114851e..312132f893b46 100644 --- a/python_modules/dagster/dagster/core/types.py +++ b/python_modules/dagster/dagster/core/types.py @@ -2,6 +2,7 @@ from six import (string_types, integer_types) from dagster import check +from dagster.core.errors import DagsterTypeError class DagsterType(object): @@ -22,7 +23,7 @@ def __repr__(self): return 'DagsterType({name})'.format(name=self.name) def evaluate_value(self, _value): - '''Subclasses must implement this method. Check if the value is a valid one and output :py:class:`IncomingValueResult`. + '''Subclasses must implement this method. Check if the value is a valid one and return a processed version of it. If value is invalid, raise `DagsterTypeError`. Args: value: The value to check @@ -63,12 +64,12 @@ def is_python_valid_value(self, _value): def evaluate_value(self, value): if not self.is_python_valid_value(value): - return IncomingValueResult.create_failure( + raise DagsterTypeError( 'Expected valid value for {type_name} but got {value}'.format( type_name=self.name, value=repr(value) ) ) - return IncomingValueResult.create_success(value) + return value class _DagsterAnyType(DagsterType): @@ -84,7 +85,7 @@ def process_value(self, value): return value def evaluate_value(self, value): - return IncomingValueResult.create_success(value) + return value def nullable_isinstance(value, typez): @@ -108,12 +109,12 @@ def is_python_valid_value(self, value): def evaluate_value(self, value): if not self.is_python_valid_value(value): - return IncomingValueResult.create_failure( + raise DagsterTypeError( 'Expected valid value for {type_name} but got {value}'.format( type_name=self.name, value=repr(value) ) ) - return IncomingValueResult.create_success(value) + return value class _DagsterStringType(DagsterScalarType): @@ -208,7 +209,7 @@ def __init__(self, name, fields, ctor, description=None): def evaluate_value(self, value): if value is not None and not isinstance(value, dict): - return IncomingValueResult.create_failure('Incoming value for composite must be dict') + raise DagsterTypeError('Incoming value for composite must be dict') return process_incoming_composite_value(self, value, self.ctor) @@ -229,34 +230,6 @@ def __init__(self, fields): ) -class IncomingValueResult(namedtuple('_IncomingValueResult', 'success value error_msg')): - '''Result of a dagster typecheck. - - Attributes: - success (bool): whether value is a valid one. - value (any): the actual value - error_msg (str): error message - ''' - - def __new__(cls, success, value, error_msg): - return super(IncomingValueResult, cls).__new__( - cls, - check.bool_param(success, 'success'), - value, - check.opt_str_param(error_msg, 'error_msg'), - ) - - @staticmethod - def create_success(value): - '''Create a succesful IncomingValueResult out of a value''' - return IncomingValueResult(success=True, value=value, error_msg=None) - - @staticmethod - def create_failure(error_msg): - '''Create a failing IncomingValueResult with a error_msg''' - return IncomingValueResult(success=False, value=None, error_msg=error_msg) - - def process_incoming_composite_value(dagster_composite_type, incoming_value, ctor): check.inst_param(dagster_composite_type, 'dagster_composite_type', DagsterCompositeType) incoming_value = check.opt_dict_param(incoming_value, 'incoming_value', key_type=str) @@ -269,7 +242,7 @@ def process_incoming_composite_value(dagster_composite_type, incoming_value, cto for received_arg in received_args: if received_arg not in defined_args: - return IncomingValueResult.create_failure( + raise DagsterTypeError( 'Field {received} not found. Defined fields: {defined}'.format( defined=repr(defined_args), received=received_arg, @@ -283,7 +256,7 @@ def process_incoming_composite_value(dagster_composite_type, incoming_value, cto check.invariant(not field_def.default_provided) if expected_field not in received_args: - return IncomingValueResult.create_failure( + raise DagsterTypeError( 'Did not not find {expected}. Defined fields: {defined}'.format( expected=expected_field, defined=repr(defined_args), @@ -297,15 +270,13 @@ def process_incoming_composite_value(dagster_composite_type, incoming_value, cto evaluation_result = field_def.dagster_type.evaluate_value( incoming_value[expected_field] ) - if not evaluation_result.success: - return evaluation_result - fields_to_pass[expected_field] = evaluation_result.value + fields_to_pass[expected_field] = evaluation_result elif field_def.default_provided: fields_to_pass[expected_field] = field_def.default_value else: check.invariant(field_def.is_optional and not field_def.default_provided) - return IncomingValueResult.create_success(ctor(fields_to_pass)) + return ctor(fields_to_pass) String = _DagsterStringType(name='String', description='A string.') From 2a2af4cea1e0279126b7c8b13d49d5e024f0eae3 Mon Sep 17 00:00:00 2001 From: Mikhail Novikov Date: Mon, 10 Sep 2018 15:47:50 +0300 Subject: [PATCH 078/103] Fixed dagit version --- python_modules/dagit/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python_modules/dagit/setup.py b/python_modules/dagit/setup.py index f6973a83e6654..81a9f7cd9f191 100644 --- a/python_modules/dagit/setup.py +++ b/python_modules/dagit/setup.py @@ -22,7 +22,7 @@ def long_description(): setup( name='dagit', - version='0.2.0.dev3', + version=version['__version__'], author='Elementl', author_email='schrockn@elementl.com', license='Apache-2.0', From 878639d5fa57d6da0b6b9cfc776e4692db82bc6a Mon Sep 17 00:00:00 2001 From: Mikhail Novikov Date: Mon, 10 Sep 2018 15:56:30 +0300 Subject: [PATCH 079/103] DagsterTypeError >== DagsterEvaluateValueError --- python_modules/dagster/dagster/__init__.py | 2 ++ .../dagster/dagster/core/compute_nodes.py | 5 +-- .../core_tests/test_config_type_system.py | 32 +++++++++---------- python_modules/dagster/dagster/core/errors.py | 5 +++ .../dagster/dagster/core/execution.py | 3 +- python_modules/dagster/dagster/core/types.py | 17 +++++----- .../dagster/docs/apidocs/errors.rst | 2 ++ 7 files changed, 39 insertions(+), 27 deletions(-) diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index 62446272dc429..e834f7911c114 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -36,6 +36,7 @@ DagsterTypeError, DagsterUserCodeExecutionError, DagsterExpectationFailedError, + DagsterEvaluateValueError, ) import dagster.config as config @@ -73,6 +74,7 @@ 'DagsterTypeError', 'DagsterUserCodeExecutionError', 'DagsterExpectationFailedError', + 'DagsterEvaluateValueError', # types 'config', diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index 33f1aaab8a57c..2dca1a883397f 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -38,6 +38,7 @@ DagsterInvariantViolationError, DagsterTypeError, DagsterUserCodeExecutionError, + DagsterEvaluateValueError, ) from .types import DagsterType @@ -306,7 +307,7 @@ def _create_compute_node_result(self, result): try: evaluation_result = node_output.dagster_type.evaluate_value(result.value) - except DagsterTypeError as e: + except DagsterEvaluateValueError as e: raise DagsterInvariantViolationError( '''Solid {cn.solid.name} output name {output_name} output {result.value} type failure: {error_msg}'''.format( @@ -580,7 +581,7 @@ def create_conf_value(execution_info, solid): try: return solid.config_def.config_type.evaluate_value(config_input) - except DagsterTypeError as e: + except DagsterEvaluateValueError as e: raise DagsterTypeError( 'Error evaluating config for {solid_name}: {error_msg}'.format( solid_name=solid.name, diff --git a/python_modules/dagster/dagster/core/core_tests/test_config_type_system.py b/python_modules/dagster/dagster/core/core_tests/test_config_type_system.py index 8806d10338af2..744ade3ef4f12 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_config_type_system.py +++ b/python_modules/dagster/dagster/core/core_tests/test_config_type_system.py @@ -3,7 +3,7 @@ ConfigDefinition, types, Field, - DagsterTypeError, + DagsterEvaluateValueError, ) @@ -24,10 +24,10 @@ def test_int_fails(): 'int_field': Field(types.Int), }) - with pytest.raises(DagsterTypeError): + with pytest.raises(DagsterEvaluateValueError): config_def.config_type.evaluate_value({'int_field': 'fjkdj'}) - with pytest.raises(DagsterTypeError): + with pytest.raises(DagsterEvaluateValueError): config_def.config_type.evaluate_value({'int_field': True}) @@ -93,16 +93,16 @@ def test_single_required_string_field_config_type(): 'string_field': None } - with pytest.raises(DagsterTypeError): + with pytest.raises(DagsterEvaluateValueError): _validate(_single_required_string_config_dict(), {}) - with pytest.raises(DagsterTypeError): + with pytest.raises(DagsterEvaluateValueError): _validate(_single_required_string_config_dict(), {'extra': 'yup'}) - with pytest.raises(DagsterTypeError): + with pytest.raises(DagsterEvaluateValueError): _validate(_single_required_string_config_dict(), {'string_field': 'yupup', 'extra': 'yup'}) - with pytest.raises(DagsterTypeError): + with pytest.raises(DagsterEvaluateValueError): _validate(_single_required_string_config_dict(), {'string_field': 1}) @@ -131,16 +131,16 @@ def test_multiple_required_fields_passing(): def test_multiple_required_fields_failing(): - with pytest.raises(DagsterTypeError): + with pytest.raises(DagsterEvaluateValueError): _validate(_multiple_required_fields_config_dict(), {}) - with pytest.raises(DagsterTypeError): + with pytest.raises(DagsterEvaluateValueError): _validate(_multiple_required_fields_config_dict(), {'field_one': 'yup'}) - with pytest.raises(DagsterTypeError): + with pytest.raises(DagsterEvaluateValueError): _validate(_multiple_required_fields_config_dict(), {'field_one': 'yup', 'extra': 'yup'}) - with pytest.raises(DagsterTypeError): + with pytest.raises(DagsterEvaluateValueError): _validate( _multiple_required_fields_config_dict(), { 'field_one': 'value_one', @@ -161,11 +161,11 @@ def test_single_optional_field_passing(): def test_single_optional_field_failing(): - with pytest.raises(DagsterTypeError): + with pytest.raises(DagsterEvaluateValueError): _validate(_single_optional_string_config_dict(), {'optional_field': 1}) - with pytest.raises(DagsterTypeError): + with pytest.raises(DagsterEvaluateValueError): _validate(_single_optional_string_config_dict(), {'dlkjfalksdjflksaj': 1}) @@ -277,13 +277,13 @@ def test_single_nested_config(): def test_single_nested_config_failures(): - with pytest.raises(DagsterTypeError): + with pytest.raises(DagsterEvaluateValueError): _validate(_single_nested_config(), {'nested': 'dkjfdk'}) - with pytest.raises(DagsterTypeError): + with pytest.raises(DagsterEvaluateValueError): _validate(_single_nested_config(), {'nested': {'int_field': 'dkjfdk'}}) - with pytest.raises(DagsterTypeError): + with pytest.raises(DagsterEvaluateValueError): _validate(_single_nested_config(), {'nested': {'int_field': {'too_nested': 'dkjfdk'}}}) diff --git a/python_modules/dagster/dagster/core/errors.py b/python_modules/dagster/dagster/core/errors.py index 5d83d4bde41d3..ac0fcc1ab1199 100644 --- a/python_modules/dagster/dagster/core/errors.py +++ b/python_modules/dagster/dagster/core/errors.py @@ -17,6 +17,11 @@ class DagsterUserError(DagsterError): pass +class DagsterEvaluateValueError(DagsterError): + '''Indicates that invalid value was passed to a type's evaluate_value method''' + pass + + class DagsterInvalidDefinitionError(DagsterUserError): '''Indicates that some violation of the definition rules has been violated by the user''' pass diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index 510039cc163e7..b7e7ebceb5118 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -35,6 +35,7 @@ from .errors import ( DagsterInvariantViolationError, DagsterTypeError, + DagsterEvaluateValueError, ) from .compute_nodes import ( @@ -229,7 +230,7 @@ def yield_context(pipeline, environment): try: evaluation_result = config_type.evaluate_value(environment.context.config) - except DagsterTypeError as e: + except DagsterEvaluateValueError as e: raise DagsterTypeError( 'Invalid config value: {error_msg}'.format(error_msg=','.join(e.args)) ) diff --git a/python_modules/dagster/dagster/core/types.py b/python_modules/dagster/dagster/core/types.py index 312132f893b46..4ee48784f7df1 100644 --- a/python_modules/dagster/dagster/core/types.py +++ b/python_modules/dagster/dagster/core/types.py @@ -2,7 +2,8 @@ from six import (string_types, integer_types) from dagster import check -from dagster.core.errors import DagsterTypeError + +from dagster.core.errors import DagsterEvaluateValueError class DagsterType(object): @@ -23,13 +24,13 @@ def __repr__(self): return 'DagsterType({name})'.format(name=self.name) def evaluate_value(self, _value): - '''Subclasses must implement this method. Check if the value is a valid one and return a processed version of it. If value is invalid, raise `DagsterTypeError`. + '''Subclasses must implement this method. Check if the value is a valid one and return a processed version of it. If value is invalid, raise `DagsterEvaluateValueError`. Args: value: The value to check Returns: - IncomingValueResult + value: A transformed value ''' check.not_implemented('Must implement in subclass') @@ -64,7 +65,7 @@ def is_python_valid_value(self, _value): def evaluate_value(self, value): if not self.is_python_valid_value(value): - raise DagsterTypeError( + raise DagsterEvaluateValueError( 'Expected valid value for {type_name} but got {value}'.format( type_name=self.name, value=repr(value) ) @@ -109,7 +110,7 @@ def is_python_valid_value(self, value): def evaluate_value(self, value): if not self.is_python_valid_value(value): - raise DagsterTypeError( + raise DagsterEvaluateValueError( 'Expected valid value for {type_name} but got {value}'.format( type_name=self.name, value=repr(value) ) @@ -209,7 +210,7 @@ def __init__(self, name, fields, ctor, description=None): def evaluate_value(self, value): if value is not None and not isinstance(value, dict): - raise DagsterTypeError('Incoming value for composite must be dict') + raise DagsterEvaluateValueError('Incoming value for composite must be dict') return process_incoming_composite_value(self, value, self.ctor) @@ -242,7 +243,7 @@ def process_incoming_composite_value(dagster_composite_type, incoming_value, cto for received_arg in received_args: if received_arg not in defined_args: - raise DagsterTypeError( + raise DagsterEvaluateValueError( 'Field {received} not found. Defined fields: {defined}'.format( defined=repr(defined_args), received=received_arg, @@ -256,7 +257,7 @@ def process_incoming_composite_value(dagster_composite_type, incoming_value, cto check.invariant(not field_def.default_provided) if expected_field not in received_args: - raise DagsterTypeError( + raise DagsterEvaluateValueError( 'Did not not find {expected}. Defined fields: {defined}'.format( expected=expected_field, defined=repr(defined_args), diff --git a/python_modules/dagster/docs/apidocs/errors.rst b/python_modules/dagster/docs/apidocs/errors.rst index 4aa3d27ad7cab..160cb4758d120 100644 --- a/python_modules/dagster/docs/apidocs/errors.rst +++ b/python_modules/dagster/docs/apidocs/errors.rst @@ -14,3 +14,5 @@ Core dagster error classes. .. autoexception:: DagsterUserCodeExecutionError .. autoexception:: DagsterExpectationFailedError + +.. autoexception:: DagsterEvaluateValueError From 6396e3742251bc316488e50de56d87f94362883a Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Mon, 10 Sep 2018 06:21:11 -0700 Subject: [PATCH 080/103] Remove metric infrastructure This was essentially unused and overcomplicated. We can readd something like this later. --- python_modules/dagster/dagster/__init__.py | 4 +- .../dagster/dagster/cli/pipeline.py | 40 ++-------------- .../dagster/dagster/core/compute_nodes.py | 8 +++- .../dagster/dagster/core/execution.py | 6 ++- .../dagster/dagster/core/execution_context.py | 46 ------------------- .../test_sql_project_pipeline.py | 1 - 6 files changed, 16 insertions(+), 89 deletions(-) diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index 62446272dc429..c5d90e9481679 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -7,8 +7,6 @@ from dagster.core.execution_context import ExecutionContext -from .version import __version__ - from dagster.core.definitions import ( ConfigDefinition, DependencyDefinition, @@ -41,6 +39,8 @@ import dagster.config as config import dagster.core.types as types +from .version import __version__ + __all__ = [ # Definition 'ConfigDefinition', diff --git a/python_modules/dagster/dagster/cli/pipeline.py b/python_modules/dagster/dagster/cli/pipeline.py index b3552b779dde6..f52637e0567c3 100644 --- a/python_modules/dagster/dagster/cli/pipeline.py +++ b/python_modules/dagster/dagster/cli/pipeline.py @@ -223,10 +223,10 @@ def do_execute_command(pipeline, env, printer): pipeline_iter = execute_pipeline_iterator(pipeline, environment) - process_results_for_console(pipeline_iter, printer) + process_results_for_console(pipeline_iter) -def process_results_for_console(pipeline_iter, printer): +def process_results_for_console(pipeline_iter): results = [] for result in pipeline_iter: @@ -240,38 +240,4 @@ def process_results_for_console(pipeline_iter, printer): click_context.exit(1) results.append(result) - # Commenting this out for now - # print_metrics_to_console(results, printer) - - -def print_metrics_to_console(results, printer): - for result in results: - context = result.context - metrics_of_solid = list(context.metrics_matching_context({'solid': result.solid.name})) - - printer('Metrics for {name}'.format(name=result.solid.name)) - - for input_def in result.solid.input_defs: - metrics_for_input = list( - context.metrics_covering_context( - { - 'solid': result.solid.name, - 'input': input_def.name, - } - ) - ) - if metrics_for_input: - printer(' Input {input_name}'.format(input_name=input_def.name)) - for metric in metrics_for_input: - printer( - '{indent}{metric_name}: {value}'.format( - indent=' ' * 8, metric_name=metric.metric_name, value=metric.value - ) - ) - - for metric in metrics_of_solid: - printer( - '{indent}{metric_name}: {value}'.format( - indent=' ' * 4, metric_name=metric.metric_name, value=metric.value - ) - ) + return results diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index fc7eed884a0a7..f4cb0ef6dd68d 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -241,7 +241,13 @@ def _execute_core_transform(context, compute_node, conf, inputs): all_results = list(_collect_result_list(context, compute_node, conf, inputs)) - context.metric('core_transform_time_ms', timer_result.millis) + context.debug( + 'Finished executing transform for solid {solid}. Time elapsed: {millis:.3f} ms'.format( + solid=compute_node.solid.name, + millis=timer_result.millis, + ), + execution_time_ms=timer_result.millis, + ) for result in all_results: yield result diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index 01bd7ad05eb13..4c7365629ce48 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -143,7 +143,8 @@ def success(self): @property def transformed_values(self): - '''Return dictionary of transformed results, with keys being output names. Returns None if execution result isn't a success.''' + '''Return dictionary of transformed results, with keys being output names. + Returns None if execution result isn't a success.''' if self.success and self.transforms: return { result.success_data.output_name: result.success_data.value @@ -153,7 +154,8 @@ def transformed_values(self): return None def transformed_value(self, output_name=DEFAULT_OUTPUT): - '''Returns transformed value either for DEFAULT_OUTPUT or for the output given as outputname. Returns None if execution result isn't a success''' + '''Returns transformed value either for DEFAULT_OUTPUT or for the output + given as output_name. Returns None if execution result isn't a success''' check.str_param(output_name, 'output_name') if self.success: for result in self.transforms: diff --git a/python_modules/dagster/dagster/core/execution_context.py b/python_modules/dagster/dagster/core/execution_context.py index 56a9fd0cf43e1..6570bb7083cb6 100644 --- a/python_modules/dagster/dagster/core/execution_context.py +++ b/python_modules/dagster/dagster/core/execution_context.py @@ -1,4 +1,3 @@ -import copy import itertools import json import uuid @@ -165,48 +164,3 @@ def value(self, key, value): yield self._context_dict.pop(key) - - def metric(self, metric_name, value): - check.str_param(metric_name, 'metric_name') - check.not_none_param(value, 'value') - - keys = list(self._context_dict.keys()) - keys.append(metric_name) - if isinstance(value, float): - format_string = 'metric:{metric_name}={value:.3f} {kv_message}' - else: - format_string = 'metric:{metric_name}={value} {kv_message}' - value = json.dumps(value) - - self._logger.info( - format_string.format( - metric_name=metric_name, - value=value, - kv_message=_kv_message(self._context_dict.items()), - ), - extra=self._context_dict - ) - - self._metrics.append( - Metric( - context_dict=copy.copy(self._context_dict), metric_name=metric_name, value=value - ) - ) - - def _dict_covers(self, needle_dict, haystack_dict): - for key, value in needle_dict.items(): - if not key in haystack_dict: - return False - if value != haystack_dict[key]: - return False - return True - - def metrics_covering_context(self, needle_dict): - for metric in self._metrics: - if self._dict_covers(needle_dict, metric.context_dict): - yield metric - - def metrics_matching_context(self, needle_dict): - for metric in self._metrics: - if needle_dict == metric.context_dict: - yield metric diff --git a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py index 8374d9805308b..10a847fd7f701 100644 --- a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py +++ b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py @@ -5,7 +5,6 @@ InputDefinition, PipelineContextDefinition, PipelineDefinition, - config, execute_pipeline, ) From 06b82a56762ead10caf3cf3db87bb62876cd756f Mon Sep 17 00:00:00 2001 From: Ben Gotow Date: Tue, 11 Sep 2018 01:35:36 -0700 Subject: [PATCH 081/103] Initial styling pass (#170) * Add DEVELOPMENT.md outlining dev setup steps * Pull in Open Sans, Source Code Pro from Google Fonts * Make descriptions collapsible, remove uniform leading spaces before markdowning * Add the Octopus logo and brand mark * Stylize pipeline header, remove section cards in favor of section item cards * Fix relative size of headers in solid cards, right justify type signatures, prevent text overflow * Types are cool, make hoverable items very obvious and light blue * Give the left sidebar a bit of color (TODO: Needed odd Typescript hack) * Fix minor PR issues --- .circleci/config.yml | 6 + DEVELOPMENT.md | 42 +++++++ .../dagit/webapp/jest/fileTransformer.js | 8 ++ .../dagit/dagit/webapp/package.json | 19 ++- .../dagit/dagit/webapp/public/index.html | 14 +++ .../dagit/dagit/webapp/src/Config.tsx | 4 +- .../dagit/dagit/webapp/src/Description.tsx | 108 +++++++++++++++++- .../dagit/dagit/webapp/src/Page.tsx | 5 +- .../dagit/dagit/webapp/src/Pipeline.tsx | 44 +++---- .../dagit/dagit/webapp/src/Pipelines.tsx | 17 ++- .../dagit/dagit/webapp/src/Solid.tsx | 13 ++- .../dagit/dagit/webapp/src/SolidListItem.tsx | 26 +++-- .../dagit/webapp/src/SolidTypeSignature.tsx | 7 +- .../dagit/webapp/src/TypeWithTooltip.tsx | 15 ++- .../__tests__/__snapshots__/App.test.tsx.snap | 12 +- .../dagit/webapp/src/images/nav-logo.png | Bin 0 -> 15082 bytes 16 files changed, 279 insertions(+), 61 deletions(-) create mode 100644 DEVELOPMENT.md create mode 100644 python_modules/dagit/dagit/webapp/jest/fileTransformer.js create mode 100644 python_modules/dagit/dagit/webapp/src/images/nav-logo.png diff --git a/.circleci/config.yml b/.circleci/config.yml index a4a624b112b8e..8547385f1d83d 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -111,6 +111,12 @@ jobs: cd python_modules/dagit/dagit/webapp yarn run jest + - run: + name: Check coding style + command: | + cd python_modules/dagit/dagit/webapp + yarn run check-prettier + - store_artifacts: path: test-reports destination: test-reports diff --git a/DEVELOPMENT.md b/DEVELOPMENT.md new file mode 100644 index 0000000000000..7e1a2c4bcb247 --- /dev/null +++ b/DEVELOPMENT.md @@ -0,0 +1,42 @@ +Setup for developing the Dagit UI + +1. Install Python 3.6. + * You can't use Python 3.7+ yet because of https://github.com/apache/arrow/issues/1125 + +2. Create a virtual environment + ``` + python3 -m venv ~/venvs/dagit/ + source ~/venvs/dagit/bin/activate + pip + ``` + +3. Install all the dependencies, and make the dagit and dagster packages available by name: + ``` + cd ./dagster + pip install -r dev-requirements.txt + pip install -e . + cd ../dagit + pip install -r dev-requirements.txt + pip install -e . + ``` + +4. Run the GraphQL server from a directory that contains a repository.yml file. + For example: + + ``` + cd ./dagster/dagster/dagster_examples + python3.6 /path/to/python_modules/dagit/bin/dagit -p 3333 + ``` + +5. Run the JS component of the Dagit UI, pointing it to the GraphQL server: + ``` + cd ./dagit/dagit/webapp + yarn install + REACT_APP_GRAPHQL_URI="http://localhost:3333/graphql" yarn start + ``` + +6. You can run tests for python by running `tox` in `python_modules/dagit` or `python_modules/dagster`. + +7. Inside webapp directory, you can run tests too. `yarn test` runs all tests. In development it's handy to run `yarn run jest --watch` to have interactive test runner. + +8. Some webapp tests use *snapshots* - auto-generated results to which the test render tree is compared. Those tests are supposed to break when you change something, check that the change is sensible and run `yarn run jest -u` to update snapshot to the new result. You can also update snapshots interactively when you are in `--watch` mode. diff --git a/python_modules/dagit/dagit/webapp/jest/fileTransformer.js b/python_modules/dagit/dagit/webapp/jest/fileTransformer.js new file mode 100644 index 0000000000000..cc01d2859953c --- /dev/null +++ b/python_modules/dagit/dagit/webapp/jest/fileTransformer.js @@ -0,0 +1,8 @@ +// fileTransformer.js +const path = require("path"); + +module.exports = { + process(src, filename, config, options) { + return "module.exports = " + JSON.stringify(path.basename(filename)) + ";"; + } +}; diff --git a/python_modules/dagit/dagit/webapp/package.json b/python_modules/dagit/dagit/webapp/package.json index 456dd7d09db4e..771d3a6a171c1 100644 --- a/python_modules/dagit/dagit/webapp/package.json +++ b/python_modules/dagit/dagit/webapp/package.json @@ -51,27 +51,22 @@ "start": "react-scripts-ts start", "build": "react-scripts-ts build", "eject": "react-scripts-ts eject", - "test": "npm run ts && npm run jest", + "prettier": "prettier --write \"./src/**/*.tsx\"", + "test": "npm run ts && npm run jest && npm run check-prettier", "jest": "jest", + "check-prettier": "prettier --list-different \"./src/**/*.tsx\"", "ts": "tsc -p .", "download-schema": "apollo schema:download --endpoint $REACT_APP_GRAPHQL_URI ./src/schema.json", "generate-types": "apollo codegen:generate --queries \"./src/**/*.tsx\" --target typescript types --schema ./src/schema.json", "watch-types": "apollo codegen:generate --queries \"./src/**/*.tsx\" --target typescript types --schema ./src/schema.json --watch" }, "jest": { - "roots": [ - "/src" - ], + "roots": ["/src"], "transform": { - "^.+\\.(ts|tsx)$": "ts-jest" + "^.+\\.(ts|tsx)$": "ts-jest", + "\\.(jpg|jpeg|png|gif|eot|otf|webp|svg|ttf|woff|woff2|mp4|webm|wav|mp3|m4a|aac|oga)$": "/jest/fileTransformer.js" }, - "moduleFileExtensions": [ - "js", - "jsx", - "json", - "ts", - "tsx" - ], + "moduleFileExtensions": ["js", "jsx", "json", "ts", "tsx"], "testRegex": "/__tests__/.*\\.test\\.(ts|tsx)$", "testURL": "http://localhost" } diff --git a/python_modules/dagit/dagit/webapp/public/index.html b/python_modules/dagit/dagit/webapp/public/index.html index 4c5e3649ff206..f7a264f59d666 100644 --- a/python_modules/dagit/dagit/webapp/public/index.html +++ b/python_modules/dagit/dagit/webapp/public/index.html @@ -5,6 +5,7 @@ + + + + + Dagit diff --git a/python_modules/dagit/dagit/webapp/src/Config.tsx b/python_modules/dagit/dagit/webapp/src/Config.tsx index 7b4dca300404b..a2fb22eabdc23 100644 --- a/python_modules/dagit/dagit/webapp/src/Config.tsx +++ b/python_modules/dagit/dagit/webapp/src/Config.tsx @@ -1,7 +1,7 @@ import * as React from "react"; import gql from "graphql-tag"; import styled from "styled-components"; -import { UL, H6 } from "@blueprintjs/core"; +import { UL, H6, Colors } from "@blueprintjs/core"; import SpacedCard from "./SpacedCard"; import TypeWithTooltip from "./TypeWithTooltip"; import Description from "./Description"; @@ -78,5 +78,7 @@ const ConfigCard = styled(SpacedCard)` const DescriptionWrapper = styled.div` max-width: 400px; + margin-top: 10px; margin-bottom: 10px; + color: ${Colors.GRAY2}; `; diff --git a/python_modules/dagit/dagit/webapp/src/Description.tsx b/python_modules/dagit/dagit/webapp/src/Description.tsx index 4cd9a0f46d9a7..7426ec9ce12e2 100644 --- a/python_modules/dagit/dagit/webapp/src/Description.tsx +++ b/python_modules/dagit/dagit/webapp/src/Description.tsx @@ -8,15 +8,117 @@ interface IDescriptionProps { description: string | null; } +interface IDescriptionState { + hasMore: boolean; + expanded: boolean; +} + +const MaxHeight = 320; + +/* +If `input` begins with whitespace and every line contains at least that whitespace, +it removes it. Otherwise, return the original string. +*/ +function removeLeadingSpaces(input: string) { + const leadingSpaces = /^\n?( +)/.exec(input); + if (leadingSpaces == null) { + return input; + } + + const lines = input.split("\n"); + if (!lines.every(l => l.substr(0, leadingSpaces[1].length).trim() === "")) { + return input; + } + + return lines.map(l => l.substr(leadingSpaces[1].length)).join("\n"); +} + export default class Description extends React.Component< IDescriptionProps, - {} + IDescriptionState > { + private _container: React.RefObject = React.createRef(); + + public state: IDescriptionState = { + hasMore: false, + expanded: false + }; + + componentDidMount() { + this.updateHandleState(); + } + + componentDidUpdate() { + this.updateHandleState(); + } + + updateHandleState() { + if (!this._container.current) { + return; + } + const hasMore = this._container.current.clientHeight > MaxHeight; + if (hasMore !== this.state.hasMore) { + this.setState({ hasMore }); + } + } + render() { if (!this.props.description || this.props.description.length === 0) { return null; - } else { - return ; } + + const { expanded, hasMore } = this.state; + return ( + this.setState({ expanded: !expanded })} + > + {!expanded && hasMore && } + {hasMore && ( + + {expanded ? "Show Less" : "Show More"} + + )} + +
+ +
+
+ ); } } + +const Container = styled.div` + overflow: hidden; + font-size: 0.8rem; + position: relative; +`; + +const Mask = styled.div` + position: absolute; + top: 0; + left: 0; + right: 0; + bottom: 0; + background: linear-gradient( + to bottom, + rgba(255, 255, 255, 0) 0%, + rgba(255, 255, 255, 0) 70%, + rgba(255, 255, 255, 1) + ); + pointer-events: none; + border-bottom: 1px solid #eee; +`; + +const ShowMoreHandle = styled.a` + line-height: 20px; + position: absolute; + padding: 0 14px; + bottom: 0; + left: 50%; + height: 20px; + transform: translate(-50%); + background: #eee; +`; diff --git a/python_modules/dagit/dagit/webapp/src/Page.tsx b/python_modules/dagit/dagit/webapp/src/Page.tsx index 1140d50ee34e2..beefc6c9d299b 100644 --- a/python_modules/dagit/dagit/webapp/src/Page.tsx +++ b/python_modules/dagit/dagit/webapp/src/Page.tsx @@ -1,5 +1,6 @@ import * as React from "react"; import { Alignment, Navbar } from "@blueprintjs/core"; +import navBarImage from "./images/nav-logo.png"; export default class Page extends React.Component { public render() { @@ -7,7 +8,9 @@ export default class Page extends React.Component { <> - Dagit + + + diff --git a/python_modules/dagit/dagit/webapp/src/Pipeline.tsx b/python_modules/dagit/dagit/webapp/src/Pipeline.tsx index c1c963888f0e1..39f69c329ff64 100644 --- a/python_modules/dagit/dagit/webapp/src/Pipeline.tsx +++ b/python_modules/dagit/dagit/webapp/src/Pipeline.tsx @@ -4,7 +4,7 @@ import styled from "styled-components"; import { History } from "history"; import { Switch, Route, match } from "react-router"; import { Link } from "react-router-dom"; -import { Card, H2, H5, Text, Code, UL, H6 } from "@blueprintjs/core"; +import { Card, H3, H5, Text, Code, UL, H6 } from "@blueprintjs/core"; import SpacedCard from "./SpacedCard"; import Config from "./Config"; import Solid from "./Solid"; @@ -69,16 +69,12 @@ export default class Pipeline extends React.Component { render={({ match }) => ( <> -

- - {this.props.pipeline.name} - -

+ + {this.props.pipeline.name} +
-

- {match.params.solid} -

+ {match.params.solid}
)} @@ -88,9 +84,7 @@ export default class Pipeline extends React.Component { render={() => ( <> -

- {this.props.pipeline.name} -

+ {this.props.pipeline.name}
)} @@ -165,14 +159,14 @@ export default class Pipeline extends React.Component { renderSolidList = () => { return ( <> - -
Context
+
+

Context

{this.renderContext()} - - -
Solids
-
    {this.renderSolids()}
-
+
+
+

Solids

+ {this.renderSolids()} +
); }; @@ -190,11 +184,19 @@ export default class Pipeline extends React.Component { } } -const PipelineGraphWrapper = styled(Card)` +const BreadcrumbText = styled.h2` + font-family: "Source Code Pro", monospace; + font-weight: 500; +`; + +const Section = styled.div` + margin-bottom: 30px; +`; + +const PipelineGraphWrapper = styled(Section)` height: 500px; width: 100%; display: flex; - margin-bottom: 10px; `; const PipelineCard = styled(Card)` diff --git a/python_modules/dagit/dagit/webapp/src/Pipelines.tsx b/python_modules/dagit/dagit/webapp/src/Pipelines.tsx index f1fdde2dc448d..81ef67b80f3be 100644 --- a/python_modules/dagit/dagit/webapp/src/Pipelines.tsx +++ b/python_modules/dagit/dagit/webapp/src/Pipelines.tsx @@ -3,7 +3,13 @@ import gql from "graphql-tag"; import styled from "styled-components"; import { History } from "history"; import { withRouter } from "react-router-dom"; -import { Tabs, Tab, NonIdealState } from "@blueprintjs/core"; +import { + Tabs, + ITabsProps, + Tab, + NonIdealState, + Colors +} from "@blueprintjs/core"; import Pipeline from "./Pipeline"; import { PipelinesFragment } from "./types/PipelinesFragment"; @@ -61,14 +67,14 @@ export default class Pipelines extends React.Component { public render() { return ( - {this.renderTabs()} - + {this.renderPipeline()} ); @@ -80,3 +86,8 @@ const PipelinesContainer = styled.div` display: flex; width: 100%; `; + +// XXX(freiksenet): Some weirdness caused by weird blueprint type hierarchy +const LeftTabs = styled((Tabs as any) as React.StatelessComponent)` + background: ${Colors.LIGHT_GRAY4}; +`; diff --git a/python_modules/dagit/dagit/webapp/src/Solid.tsx b/python_modules/dagit/dagit/webapp/src/Solid.tsx index 8c790c45cd509..a08b516181ec6 100644 --- a/python_modules/dagit/dagit/webapp/src/Solid.tsx +++ b/python_modules/dagit/dagit/webapp/src/Solid.tsx @@ -132,12 +132,10 @@ export default class Solid extends React.Component { public render() { return ( -
- {this.props.solid.name} -
+ {this.props.solid.name} @@ -152,6 +150,14 @@ export default class Solid extends React.Component { } } +const SolidHeader = styled.h3` + font-family: "Source Code Pro", monospace; + margin-bottom: 8px; + margin-top: 0; + overflow: hidden; + text-overflow: ellipsis; +`; + const Cards = styled.div` display: flex; flex-wrap: wrap; @@ -175,7 +181,6 @@ const TypeSignatureWrapper = styled.div` const DescriptionWrapper = styled.div` margin-bottom: 10px; - max-width: 500px; `; const TypeWrapper = styled.div` diff --git a/python_modules/dagit/dagit/webapp/src/SolidListItem.tsx b/python_modules/dagit/dagit/webapp/src/SolidListItem.tsx index 4c598b613c517..688fa7da3a39e 100644 --- a/python_modules/dagit/dagit/webapp/src/SolidListItem.tsx +++ b/python_modules/dagit/dagit/webapp/src/SolidListItem.tsx @@ -2,7 +2,7 @@ import * as React from "react"; import gql from "graphql-tag"; import styled from "styled-components"; import { Link } from "react-router-dom"; -import { Text, Code } from "@blueprintjs/core"; +import { Text, Code, Card } from "@blueprintjs/core"; import SolidTypeSignature from "./SolidTypeSignature"; import Description from "./Description"; import { SolidListItemFragment } from "./types/SolidListItemFragment"; @@ -30,22 +30,30 @@ export default class SolidListItem extends React.Component< render() { return ( - - - {this.props.solid.name} - + + + {this.props.solid.name} + - + ); } } -const ListItem = styled.li` +const TitleCode = styled.h3` + font-family: "Source Code Pro", monospace; + margin-top: 0; + overflow: hidden; + text-overflow: ellipsis; +`; + +const SolidCard = styled(Card)` + position: relative; &&&& { margin-bottom: 10px; } @@ -58,5 +66,7 @@ const DescriptionWrapper = styled.div` `; const SolidTypeSignatureWrapper = styled.span` - margin-left: 10px; + position: absolute; + right: 18px; + top: 18px; `; diff --git a/python_modules/dagit/dagit/webapp/src/SolidTypeSignature.tsx b/python_modules/dagit/dagit/webapp/src/SolidTypeSignature.tsx index 346e4fbf6a13d..1d6deb4751f45 100644 --- a/python_modules/dagit/dagit/webapp/src/SolidTypeSignature.tsx +++ b/python_modules/dagit/dagit/webapp/src/SolidTypeSignature.tsx @@ -1,7 +1,7 @@ import * as React from "react"; import gql from "graphql-tag"; import styled from "styled-components"; -import { Code } from "@blueprintjs/core"; +import { Code, Colors } from "@blueprintjs/core"; import TypeWithTooltip from "./TypeWithTooltip"; import { SolidTypeSignatureFragment } from "./types/SolidTypeSignatureFragment"; @@ -57,6 +57,9 @@ export default class SolidTypeSignature extends React.Component< const TypeSignature = styled(Code)` && { - padding: 5px; + background: ${Colors.LIGHT_GRAY5}; + padding: 5px 10px; + box-shadow: none; + color: black; } `; diff --git a/python_modules/dagit/dagit/webapp/src/TypeWithTooltip.tsx b/python_modules/dagit/dagit/webapp/src/TypeWithTooltip.tsx index ad25f65650d09..d163584192948 100644 --- a/python_modules/dagit/dagit/webapp/src/TypeWithTooltip.tsx +++ b/python_modules/dagit/dagit/webapp/src/TypeWithTooltip.tsx @@ -29,16 +29,25 @@ export default class TypeWithTooltip extends React.Component< {this.props.type.description} } > - {this.props.type.name} + {this.props.type.name} ); } else { - return {this.props.type.name}; + return {this.props.type.name}; } } } -const TypeName = styled(Code)` +const TypeName = styled.code` + background: #d6ecff; + border: none; + padding: 1px 4px; + border-bottom: 1px solid #2491eb; + border-radius: 0.25em; + font-weight: 500; +`; + +const TypeNameWithHelp = styled(TypeName)` cursor: help; `; diff --git a/python_modules/dagit/dagit/webapp/src/__tests__/__snapshots__/App.test.tsx.snap b/python_modules/dagit/dagit/webapp/src/__tests__/__snapshots__/App.test.tsx.snap index dfcdcd053db66..de690892d2857 100644 --- a/python_modules/dagit/dagit/webapp/src/__tests__/__snapshots__/App.test.tsx.snap +++ b/python_modules/dagit/dagit/webapp/src/__tests__/__snapshots__/App.test.tsx.snap @@ -11,7 +11,13 @@ Array [
- Dagit +
,
?*;K}4m@CRE zDb3wXaUGJDd4Rb*`*)S(HLhbtQ6n$DzY29RO3C9Ge2eI@I+&FA41l4ql2DUhW=mID1s zfsDX%+%FqWP=s7#6x04ErApYK=>ce( z4T=;$ZDa!r0r&`A9(z+Ym?aKcXL+cQJ8{~+e4DI1|_qApc)>~v%UgXAlp&aTXgjRv9*2G zCi^~1fh-07Q7DiCJTW%ziN^DQQ9t5}UI~=a)8!^jfaLIem0dH@!={ec*@yxS;7DM} z*#~uDwsdERt=ns`_Mor<6HH{EfxEmwU!z8Fao4Vf;x#NQ%j}b8*$7z*WGRq21^R@g z;~uLW>+-r6KvjLIEt~91vmI9#;3OMesIUhs^4O$NE~eXSZ^lg1WUcM(tROFs>99B2 zxG&6pvB}T2)>&AlgUckdBG@1G7GRQuHN9+>R_Hrs;gFa1e1ivji=Ta$r9hSfSqfxI z0jz0z4ig@&8?Sm@KY_M;HrYpJ5(L2YrZc^4+ISDEt3Ak?T3T4H*UMZko&9dnU)i6R zt%RkK!p@se#x5J}Vu$K|Y_HE`g0$hh3t4+A7)m7=w|k*EpRR^F_UBByYe(Di$Fqu` zp6Hc*o~1yR0vS`FN3b08_~9|C+x_nVqU+O@PdaNu9OCQ~J?x|s#{TfqU)joyo0uCm z>a~rH?2XNTXCLm^$u@0$AM#9#opJmaHtpnMw*O!~`>e5(S>POP`%7nr@z*d{0$?Jd zG8yo?qj=V%HOrbi2xzrk8$F)J(^%K{-Nk6Ig<8Mc^S zSPC5M5!3CeKtm2=cD;Zp`jbpH*0K3=BV?Fa4l|vG%Erx7AWMN>DUbk`6P|1;kc`8~ zoZTtO$ftL)VXv9fOBLFh*Py^TtwdvPkC(aKItv>n)3C(m$UmqSu@>)G*5(?)K71<- z&G;_tjO37GYE1W}zSGg(Myb2Yq$wXCH>ly7-AvX%&U%!!@ALDr5waA>Qs5ti0}G8XJ?VM5Je&F%@zmwO5mIi*bT z<}($`bK1FM?<1$ZSDOGgcf7 zg^HE1ED)T#$kTx8QraN%ZY^lg#tI z$6WRAu@-G43k*7$sRdYO>z*!UP8TEC%@1~8#Any@JsGca-IQG18VvjY1i-|0$$<@1 zXJ2_TQ{-tDIR!~EU=lzTr2$*1R;Ib@Jh}Ser7yhTOEG!BM4A`$P zt@bt5OJ2v*3>w6lS9;f#*I!>?wD>1UDKA6UIUS;#QK(E2W>FV9u04wim}qdRv?way zi`u=5zPqNwt8d=4Xi>kmJAt*Y-JFC*t7P0dAJU~Us6syE?^o%TLGe}iV~~dR+3MGqywh9y@|knW zHANdT5cN?R>R`#ktCzpJzPCEiyD62IpTaeMD%|hb{gK8}v`%zwlZ$?xL6K-3G?c`g zhJmLkvYjS1=JoDhzGzWOOjS2np7^_le6vHnPT~9)Qd-zw7+aRJ+{~{zv&=A)tqqKM z^O%~Gj~*Y@c=um|>S6<;AqhFqZo+~F z4G>=sVtj%&UC&jqa@EqMyECVA`Y9(*udGsBuIB+b-F+gO3pNu_|C+gG`I2{|&j#|h za^AesP^WP{I?FsPHI4_*=0$&Wp|fiMqKH*lv;ych#M5DrKyb}Ncpupb3i(0Z9`2L8 zhH?Ek&kaA_&E~hJokxbETh@e*Ms|@0Y41?o&w*Wv!LMr?&R53uYqNG zcI9iYeHx8BP`|`AHTna$q1?d&NDv>0^DvloE1uKefQE`$RbLIZ2k*tw`eYzrE_$>^ zj;p#uJUrnZ-Cn^Tam4hOFPVCos_?VDhIMVl%qt#U`}!-ZyVIv}an3pC=!HXu-!A}^ z0KY+C8{Gg+I|o}j)JtA=otX51%kf|^7NULc4T|98DKo3?>x+Zy&epP2OR6Qw+N3*9 zzg&Qbf_eJN3jC9o`uRZA2TUK01MD>EJD@Ex9&a(ZWjKm38F5HHRb1??j&B%N}>)+HFz5pD4MNfl(CIOWx zS52L=^|4=b~)&H`UM>Y1kzHOc2X9?`hN1n33#KNt--T~pLQc8xhl ztjMJemWD?T#UiE>@*dZK(E{ah4IruN0}qra%Qh#{T_aUB_lk-0Auc_pg5&^;Tqczh zq^dktF>CH4X%F^vzSp#F1LxE0tPbv|1e&vNq?2u`am#PI$rBRl69ApN0ZN0Z1IMQ& zAcA<%maow-ffM}^*Gqp9po&d}rLs;yr)jPqhNOC5?AzY&Ccj;E4aUQrSn3tT$BdLs zWlDH9kOzP>AA{@MhU>j})){>tpAM6uhOamt`TlC!w7J9M-+#gP13Dpp!QYlr@#uS` zG#mx7=3|p*RoxJuNMq1|qqB`cV_z7v+=-9bANTgm+aDRFm%(mzuAEHUJ$ttdhUT-N6 zOPEQy_g)jmy-cYd_PC6Y3LJk-4m1w&26967UsGpY{@$u(uYMSfpp71KRz3t$XK+Q7 z#X874Nc^AvtOI87YXM|;;2RTwQUS*Zyr>2GJ+CU(Upq{5Ur30Mp(tiit6#efJiZlEuCEq;oFU5=<3Sb-(&QUlyKWtgwrGR#^78UM4KCNaKv^hsTop`}4}cRsod6v415HJz zVait_YI!Xt^JF(>=pCu3vrMb^frkSI+%Qdk4zWf!+l;a{gl_R(jW1_LOC z9g5zk^K18WA(lZ;yDgn`@o`OoLuG`rJ^p#B_mS}?{|15EQ+=iZhmYPd98(Asc6=%( zJvZ8=I~mas8)1fH@}|j*I86Jcri5gn#JM>N>>DFs36di+j>W;M3yL8OUh-JZHP}FQ zSx9u}I^Y*n7ZgIGk_-dtu82@z?#O}(s}I0`M~2y?oy_j2l6~4_g@4~u;^J_azDx_b zTrOw<#y~%D6(*#c!Hq!Ca-{Ky2$m4Jz~3KCpFVs2iWN(@^ps~h&o7;R`6!i}*TjPY z>P~9LPto8fQeTJ$)Cp@}c{4b#=c}v#c+|c`%PT8KBL4@`*%CRp2vDvT%={5-fIi;5 zEU$yHUhOEGCrz4kfam8g^D6q+U?egh10bZpHG!}B&|BNKZA+PRk`|u?(-hS^|KC$PF@L0{XLNLH%NcJCB>^~@aK8BeM@^w zOP};xy?S*s!1vAMlkJg0n_VtvytfVG7oM#?K; zsymmW4O{?7Sm6Ndz9QQ!6}Jbw7$Hu>ESXUtr4nbC8kU}q^!;jk*H?WoFysd-R$!2) z5a!Kmy1?(>?iK3mXo;VJt2#S&iGbNos%2qS{s;qTUVkjdRn@!!v*1wYio)*vT1dNqm{f?x9T=;PiW#G?JxYeoSbuLY_^hD87IgF%|(TR|`ZqI1SHjuvA1aP|Srn zia1Ct8=2M7!mOr4jE6e0(a|QAic90+$Qx$0xnvnq5oicL5LigmNEo1 zguPniilbc8u=}b>$g0ouK-dOtv z;CU%|gmMPpKP;NqZAYtcnmlLD2$X!I&8<2Lu*(->D+`Ic-(0a`e~Liq$bqpfOO|xD z)gOARwJv?2v@;@}B1u4-^vMk-NhVfTS8w;#9#{iTu_+CpOp;adN8w;FrVX93Fv0cs z&Kd7$u3@(|;ANy#KL}@2isy(#6WTs*EyJ2~4NcO{KwpJEdO`FVhJpdF=K}k|@gE#g zh|V^gP8$L@8D#aoHnzXEk$v>(Ue@I6fCL#FfNfd?Q9FQ5@z=8aU>(Z~)iE{TgI{3q z^OS+Z>If2G}?6#j@^3$D@ocWv6kKqABAdw2ow9FJ&yz>XM-a zG^n9qcn-_|7_N-O1K`*$2zSW^Gb$X3xXI^0N;TYBKIMgP4he2Oux8D&eVOJ>>8Sph z1CvF|?3h6H1bwDuI7x`i>k?Cq?9BjBpay^`HNGkVX{pT_9~mtYoMmSr&g`Zg5_pE_ zlmeGDaG)V0`#OLiV=e7|<_}@qkZjWoF|(zP8NPZJXorzGmd!b4JAOO5fG(PSioX!g zS?x_&dUvAe_$m_z3zKMn8x5osGmMQ!-vW*3sUr&tzVu+(Rddl=e`r0nf4!K%T>`9N zQZ#M$?CVHlmr*>G_pR8(5JQhhS#t4;!op7}ND|^6ak0ImB|%vZ#^%QI!u{6cL-ES=O%EKXedYpe!p$o zqJi5PkwS$9*VlvY4K&F*1HRjhM>nF5uT{*fdS=S(x&H)(+c~s{yBxNn&ba?b*BDf> zLg!=g6Ul3YX{~QOuxGO~FU<^9TbgGZik#@I1C|}R+_bkq#>97~?ddwyDUWyQyu=BX z1Wb_09`bN=V^8Vgp3}tV_=32%iMQ0dD1M)5)>XfJsA9(K=V1H!05Jg002;u=iJycv zISvYid0|CBBH?Q0bq$R$gQ|mo>Hv<3uEA{bi4tZzR<2x}kx7oT%rE;jRtZBdt*RR9 z!fCIW*44Ch_-kxA&>xD+OzHqP8`GTqk@B!7OTi9C^V5{hP)O)OO9it){fGcl%_+nm zl%N18!ajIVc49U735n18eo`=G_(h=V zr7^&VWZ*K%?EE)WrZbmS&~lbn&i*~>@Kfh=TT`#FE6S^?9$mL&3CUPf4la#NAi#HO z!O2bzLiP`rFJJEKE2F8EbI(FI`*BpQ*jIddPl_toa=yQI`I3K4A-*sb@dP);xvHj4 zj|WWZP56GY3(c|7L9#Lktzc`yErn~?W(><^A;Xp?8?o2@L&RAHldI}Auf5hi@KUJ5 zF%iY?2b_N&U;WQ?c2-6KAcNNFPzwrXBK_w+xlFE{Q-vB}yYCT(dFJulbnoX3&%zPhV6n= zn3zD7hjGBrn1`v|kd-pbbu<{^9^sgXQAJ1j&}?scJy|wr_ZI;e-veOAC}dz-QDhm` zH<9N4Wa)Z(1{X#npw*9gO~VY&Yw*)&+h8FySoiR0u%S#jMCNhfN;Sdy>gtB_nKN&N z1Fnam(7MKE8j=ns$`Sn(c6;DVXux#uLP5I0sN;ATK2L{;=`J`&c|qvvv+G`Y<(QeU zQ=*rULjW_SDaB0|iAE$)1}h$n&&&f_;Yg=WAsbTGg}8PV+UzI52-rkA0}U0c6M~_Y z(We?#8BLlI_F2KsJaUz*HRfpu^@8U8Khj6fL6bmpXRb z#6vcnW=AtP3Mh`E0lgZy+o+h~KSI(t9#ZvkR3M-0IA$j%gJ@={ju=Sa)SgW&qQ!sN zeqraBXuk#m;p-E2y!2J~X|raXVM%2sIzenyAOXy5pWnD~V^Vi!;D`=`L;5r955Mc= z28kC%MXrMx&@a;AnowFnYz4aRxSF7aP8LtY8VKM{F58m5?%^5vdB&V?^_iG<;0QSaLdNsZC(ePQvluzVBqO(V zh9#Eru)llNq)E-hQL^eD8E7BHJS2{B&iFJaaZV0-3W5@bLkR<`O^%BP_VU8DR2TF1~GugJm41Q!d|iy z(iG1NRo;yF1yPWqUudw0!QbU|Nc&hX>0@4T0RgGAn?>KoTH!~Z_S3K3%VN4(3K>hX zjl|BNKPpPzGH;RhLV)yCj7-?x@$ohQlK222l6%h}3R*##DheQOn9L0E-{qB8eGRfv zY%0r!{-8iaS;^0Y*yNGE@4nCdEK*|o6dn1XQLx0i7)BkEs9?ouOUhvAf>%IgqKPWgvYC%v~T=n+LgFcRqAM9RpzZ z0%V1?F)!k8!X4X%H@{j_lxv(|3-F+QfZx30iYuObCB@|>Y}!b}m=Lf%IIbZRqSz0x ztc5ib_)}`<9)h!uJw!?Qadc`9cc%JVa9yUW}dy|_?gJ{VcgO})) zkaLm!P#0g;OD}cgPO)5YN_Em;m5nM?*}*o!ngfD`gD^X4ISg54Fb32h6or2jxdF5A z)&q}U6d3-UmBSTjc{$cy+y%=FTB1WmZ~2?vgCYG0P<9wJ`KrB1sf*0S?T}rjLcU<5 z+zZeJjoD!MgDCLW3fL}lN-}DLJKLYmm$)o`BOs&m*s7#dH@zTFPma zRZpW6U5SU?vOk-<)YRkK0%3ao`|bq2TtGVl0eGxz2WE1ul6{W=h zWL#d7gi*l9c=!NA;YU2!v2xMtOJn3!+M(Z^+gWyyO&#;@_=4X?c#h=8Bk|W;^ zg;84_mgr4PD;>eKD8~s9meSm#!`SIa+%NwX8J>5VZRf|!g^6t72e+mA1$0aUypP|A zna0^-n0tsRG*>p{<=2v1vcB>Q1lqPSx7Y%%X%cZw(Ll!S;QnvD>86|hv|z!4J|`r2 zDf_Hz){SO?szH&+*%{NP-;ldv#UCK==_P>6?d0x%=_Lk(GmQ(;qq;=~y~WMw9c)pn zfsRNU0na0u)Pn=vH8K!bt9lB43#Apj4k_0-=|xdN!`zUxXpjA|4VKi04D(MisK0!K z1nFO?T0ky+VM$V%c{;iE#NozZO>BmAf)Xv6KgN>$C9K*0T(>W8|F&(f#aM+<5_@{b zGWh^z`_~{>JX>_MswPtFgN74NYG<1VB=utc^oJx1&>A{A9LM91s=|Z z<#Se+hkPL8+>LX?_xLQN#olw@AtpL}u|P?sC}j3$t%5t%4Gq7Z>K}98+M!`nJsv=j z%K>^cu7q3Byut$Lu%e+?Yq9v8nfjfe=9iwfETD+0VT zLg3yqb(2d=<+z0Upnhndv-j3EOh@v?y=6h~gbeNg=ih-uWVmZbr_PMU+*nI*Js%IK zje87Tws_4wMIbFF7BJE7+f%fGdM+FsXxmpREAP^oHVlAysgrK@;I)qIJ07UYd>;A2 zja+!$x-VYCkD(p{=;091xWEM+?SUgoB7H6TmVgyc_G)nG(yp2xC_Y~sl$?}4Hey!- z8!Qj~th{ODch|fQN4Z~xn}pNtt%t~1FB)odb7grPGDnpi_Jb>gSiyr-9Z+Dmhb#t0 zk>%m6y<7llE*58n9);zRco7A(!jJU+R+AlQ1q?z#WcdyQD5HD7qgBQ)aehKHs=D&L zTN$~R-r)W$48T#U4ikCX(-GpBVLv8;%?)f=aJ0(dMcn1Gm*@C2ZX19aTQpt$k3i76 zLRGbEk&Ue}hF?o4dzG!9HXk@C7j3FuhJ`}!!3Il= z0@>kOxg*z9ev8%bBT%$@MSCMod<)`mu`+g8D_~6l7?6QW91Zd5kX*7`6NlS2O`_1s%hEQ=@Bk1xS^E5rr?NUf1k1WZUUf10(HxD1bR^Rz< zK6m9_4$cKzk7&QKQK!4GY@J_z%z!0!_WzK$wRm|K&4&1IVVC=_-dh)MbwkX8J>)6z z`9u?9woMA6(f9Gc$$*mLP$vXuc$tWJMA{Ia&Up_{Lg6WuRZH-vXQQCjcg$r_>z<=5H=AWOtWWRyD1wUh+iuizFA_ z4dr`eTB~U(OQ6GB2#}&ZJ4^)q%xv7tTI&yE?{_Dw2WRo9tjKb}t1w!A;iMd59i>l4)elBbUOZ{DS&lTv$hS96fUI zNdUP^0E9)flZckm7}OTH#TP>@R{M|#77qb9p32m|Kg%ly*ZO}?XisG7} zx?Q#sg7&*}uKle|k@YkwB%#5V>d`spd|SuCUH^(o-wA;0n#>|qB%s97`H(Ku7gMP; zPHlAIi@~X7c<@&$g*f6s&J6*Kyv^~b$4hAC7vlT;)V6UU$KvW^Q!6XaihG{&EjTr7 zK}^1sN+px&v#&fA2IP-3OL+y{?l-Z@`M+=>nlY;iZPRF45jcbW*^+l);&2!lJ7Z+8 zGo%0^Vay--S!i$AzY#LYf5up7qjBU67*hq9=;v2d&biU~JpMYS1#64m*az+X6Yy{O zW(360<6yvSwF2hkErZiZ}3B`MmT26B8Cr4wPSyuZulk6}AU;`qEF} zebEgEsDJ)rtFKDMy(hrj&&5)3lJ|l3m`q~<6U|VFQSm5keT+-meH+8Mkj>Y@`D;HK z-pqcaXx3v=bDs}mbi@H@!po2a=fA8IXFt(K2s&y4OI-UJ*@5x1ft7aS&~>B_5SGPP zs;g|x1yez>+0xzLul0tuvPD-Dhsj6$ZDl!JWL>W0hjf=X0vs~UDYEL&!|&D3~-%Tx*MTdEmvQ~zV-PXDbe+X zRJHs|r^CU39;<~0=z&?GXD&B5>LH-wDR5DI8NT6S3{pC~8+KB$0oqeq>b?mI$I$s@$l)@E&4r4@?<2RJZ0vbNk{Vy5_YN3EBIC|{p%-3lTUAH zN%2;APSyYB50RyiDE^qBBO0H+1Nd2ne||Y-#^o)mm%q9``Y!q#3zqbBeua5x-onkt zjVZayvW$O|nrkWm5L^z8t4Lo1*TWd(nXn^^=F{~{Wjp`kUehg3M#G+SKhiGKbouc9 z-3LC30_Dj2-kgA~;P+}dc@yCF0#-o)WS#KDcq!lxEhCs_|ezd0=XiPzhUjEv?V65Bs~DHZAW_3+zldZPwhgAw!-X zKS+r?g;xA`Cbu4T#!vei&5XTPPvdCNuR68tn7JVtk=IPEs`~k=B}?}8mhbB7yd9Hg zb^HkXu%1CY)d`rmrp`7D0{GiS$ef4mmTyC(@DAF12PB&f8fJ0Df-??HEynW+m~1YF zjQwKlWGjVLL~eY!SWaWY7rmOM5cg3&WA@`X*km%m8oN7=bQ!c~4ro^ij~wStshqnS z?FVm6;{763tqp}jf%^JUV%#{NhUpC;Y=sWWCj18#UO(m~4>3Dn^LSY^T#H{i*C zMxQ7DU1YV^RlQJDD#k%v@I#8@1Sq;kXU@ZdBM->kB=3@V-oWU2U1*PYMxp8wCj{=4^gk`~2 zhA)Vz+UIpcpAXXa^w%2+DNz?tzKJcN*R?br+z@rj+gJ6c&H3u_K;amUkJ8W}h5YR0 zRfBSN_LWB}@AvEySgMG(i1T#@tUTZ?ihbisob%&+p6r@tdOx^aaDxxwo5NJcgaA4M zTxf|=2%H=YKrMp^0Oy6vp?E$7fLerK*wQ7WjL=Yscfvy4z=Q2u6Jx|aAs2O>%y^4Q-N;vQ|^8RA_ zWx_{&86HHLu+OJ18#Zk41cJ z`+=WVJQXs`8!*rwLI+q6kU9v?uFd8Yxq^-603%zO0U;R24rCQR+%&BN@0-!Lc7Z)y z0Eyw(4K;N8;h$ah=AQW#kzF3(6omC1j!ywzi^IJ}H9M z%QbiN(4cQGEL01M@6(EgVw)ok3k(%lE`iHiAD0D#OFo%;2I3v>TYOT<)bZiYkOz+?j>FK3`-zX-tl`ZKA5 zt}}JA>zW#zUBdaV0Uj|u9eo5lW0f&Jf9Pw;(#1ccgW5Wq#m_Lg-2(jBN#9SC#nA-X zZY6^X_^WCa7Wr5NX9Fbkv{Rv?Yy9 zd7wT-s^{7R{_m2X(4+ajY}%uzz{Zs;aWbQJ2PVdM;wO9nCS>x;rR`z)>*($_G;%6| z%Z4X$jx=LGAAS4!-wTg%I;O<)p;kE+qhvHXNfG!*7nZ{CgQma-9BaVRc{h|~TQO?h zK9I+@lPuIx5-+=Vd5*^NPFGy&y=u;|8Q9X8Oa3##QWDn+ptOEz1g+=x+&{E_!d;)b z&HR>cDkVc_04Qx)rJ{Qb9GCm&j*ie@4?e(x$3NaUi+cx6h2gpvd)wjt&0Ove8|&)k zkBtDcqwxN|#5{qq30H%hO*#weF!bHJ-%pR#-7#+Q4Qv&EMJ zS>b4SzJqrlP}1hkBQYR9XJMjA!{PVh(;UrxXOsDQ?jZF;P18<~!Avm5FeSQc>b!X` zu3EIHPhi-*xw@^qynGRJmAs>A;jd%9yB5>?7y?an+=S(MvO42~Nm5NRSy0p3fI0l> zPGK$IwAgkioL1{>UcI{Rl1pa)tGiI$3?|`jk}3{rD-RcjmpO`jy?&rp@{`QZ9tYqT_*Gjv#ei(`E;))?Zg%fBhXPg5{C)`=0&^ zHS`;iKS{Diw$g(bt!@40_i1AF%GdT38`CAMGwi|WcLum#GnFo)Ncw>4Efd@wx1 zH}O|{%0foL_b^NbBNK3yX06VxU(|hip#MAN5mFqv63QMZQ`UfkPJKWkCWerKhHn8k zeh5v!6I(Jr<{{%#!_Yc5Zd^?Jqhp%xNP)dv8Oq6!l6-{OGNvxVQsE;^c|08swU<)5 zUlGXRIEA00qBl9eAIIquP)O0%HXf1R?M~3k1vo#N*mhn}S@~c2hVnma8brOHwkQr2VdYwM=;&XJC(-G&WIeQ?jVP|wd_ zqiOCb;527L_;W5$a0L3OPGA_B#lgeU%+;9;>Qi% z)`C!>))}}I9sWk{&iM*=xpN65scu|zqqCbk(d~YQ>HF21yGz<8&fn$=YRb9NQ}8?0 z<2?ab1f@3)z>s1mm%(rEY#z30%yC%ow>SM8xXHU@*npE@J;=@QkDCMXsmAY@rp1+I z`rYT?w8WB7C@*nSYVSE>xhY-YFq)a0KA+df&u_;b+avz-dQYAFEe5wYKYwVxAHMA4 zL+inVE%C=j#K+0xK2l~r%1jKc2MbYVMhB2_(AAsY&T^nj&PNX!rD>8pO_idL6u?W3 z3mRB3v$7p*Mx9}>eTov^7m~8ANKrac-Qkp24|f{ZlO)a2{039`Jr zyeRB)*_%C_W%t*0ju4?##1p4apPq|ZAveCig^+$rhT;u6n<{6>kRq%~5-`zgd3a}M zLtDa&-adT7UD6#sd0&pFbC6oETBh6FXAJH*-N3q0uu~REwrHN6eB#89)m9uk+o|Q~ z#jquWj%OSgov(0dQ~~bJg|;-uHb25Ng{8NNIi(!~U^(1zKCai*y|-=t#PFEkOOMvL z=R%?7KBVf}#Sx%{={aM2u+RKE%C7d6HK$GxCj{!Q5}Nl3?1dZ(87aJXg%_KaxO4A! zOQV|h-aaY&?iio|47=6VR*%Q+Zt-?VkK1J$x(Qd-07shkO*QNaT9&8tqKn1{<`1xc zNC1ZB^?GyQ9xKNqT%M3>lmMXyfs2Ee5v5h*RzukG1iaGUnU|MGJ9Lh)ddC2*vsLVw z0x5&#@MjO@c#Pa~@VZxG<2;lo3MnAK;feiRhJBp50v1Gt*o*8oG!HG8hhf14t*kg5 zSb8DsH3vbfPS)78ZX#_re5a}%@OHEfoaBN|Q~S$-?-e%VB|q_z#yrcd%!G-?1BiGU zgitDMCdumkGc51!WxVnA8aTx`p?KmXp}3v}|9&DZzhS%nuI4hnxqa1}(4Lc#dp2Y# zkflJD0{ugQlx3Fr0d<0OYqwyJ+!ff(1K|wdGJ<;$AGksQQ5>iPwG1pDaXhXY+9eHO z2<(G2j!tgGDN|qrU`C{bF$Kd@w%iV5_m>qt{Fm*GL-&z+&&2uC6)Zhe2Fr{ak^k*j zYKQ*S^`G@I(C`N}zQgMns}~cB&pZv<@Sj3Hbv)!q$lI!cpZNQ?v$E}3pzNQ{ zkS5Nh%e?(3kYeisV5Wu|Jb-qT!KZ`(kwmD-rceFc8jFv-R8Zby!jp`?Lfxeb)i zt{ZP^!urv6-$(Dm=aDj@@T}8uHqDb#)y{%x2nv>@U;)S3@)oNXRuj>(PBt4td{NO2f!G`F5Rkkdzxf|}?vLQ=>ECsR@7yt^S z1(wnBhd(QG3XH~)rclPfI%F7}zJk-_Ik0qs5f%@@3aW)$ybcb#_Iq@7|JHqZO*RKf z8GwtN_(p@Lqm;p=uzEVY6kmZoZB} zaEb8d(XxG#Vwq=RoB9OEJz*yYNh2wKz)E5zkFzsmD>%%J5OhA@qRKZK8cI9I4fEy8 zkh}nPbUy^Qc@Yehu);Us#Fzb+3GL~cyYr9Q8AO`&%0|ypAWMONPzt21;K_tO6BmB0 zhic2_!65Eo>>-Yvv`4-!p&18vXezoMkfz}SXSo485B{tM*xDUkhU6q4)eQIDaJ~Oc zMbnGnOagA*to5q2p4`z?mtA&e(jeJ1Sqfw+kTC@Y6f8rvr(kbNJx)I3ZP1`&6|G@Q zm||K6j)0J|29CDYy12TmE!_6Of$xn9IKM)Z%DF5jcZw3gW7Ajz@9lrG4z|8p^T_x< z_Zd6$$zEqEkflJD0{uaO0bYhPHsQ&p0+?IE2`xVn3K@tT`61Qq-R5s@*tzcqqg$Xw zC(U_6IOkbe=oh|IrJ)+Fp}@Af`-Ud&-%Ci9eUPO Date: Tue, 11 Sep 2018 04:35:03 -0700 Subject: [PATCH 082/103] Add graphviz flags (#169) Add only_solids --- .../dagster/dagster/cli/pipeline.py | 5 ++- python_modules/dagster/dagster/graphviz.py | 37 ++++++++++++------- 2 files changed, 26 insertions(+), 16 deletions(-) diff --git a/python_modules/dagster/dagster/cli/pipeline.py b/python_modules/dagster/dagster/cli/pipeline.py index f52637e0567c3..ecfef257fcf73 100644 --- a/python_modules/dagster/dagster/cli/pipeline.py +++ b/python_modules/dagster/dagster/cli/pipeline.py @@ -173,9 +173,10 @@ def format_argument_dict(arg_def_dict): @click.command(name='graphviz', help="graphviz <>") @repository_config_argument @click.argument('name') -def graphviz_command(conf, name): +@click.option('--only-solids', is_flag=True) +def graphviz_command(conf, name, only_solids): pipeline = pipeline_from_conf(conf, name) - build_graphviz_graph(pipeline).view(cleanup=True) + build_graphviz_graph(pipeline, only_solids).view(cleanup=True) LOGGING_DICT = { diff --git a/python_modules/dagster/dagster/graphviz.py b/python_modules/dagster/dagster/graphviz.py index 3f7a8449a5c85..fd34ff792fd16 100644 --- a/python_modules/dagster/dagster/graphviz.py +++ b/python_modules/dagster/dagster/graphviz.py @@ -5,7 +5,7 @@ from dagster.core.definitions import PipelineDefinition -def build_graphviz_graph(pipeline): +def build_graphviz_graph(pipeline, only_solids): check.inst_param(pipeline, 'pipeline', PipelineDefinition) graphviz_graph = graphviz.Graph('pipeline', directory='/tmp/graphviz') for solid in pipeline.solids: @@ -13,18 +13,27 @@ def build_graphviz_graph(pipeline): graphviz_graph.attr('node', color='grey', shape='box') - for solid in pipeline.solids: - for input_def in solid.input_defs: - scoped_name = solid.name + '.' + input_def.name - graphviz_graph.node(scoped_name) - - for solid in pipeline.solids: - for input_def in solid.input_defs: - scoped_name = solid.name + '.' + input_def.name - graphviz_graph.edge(scoped_name, solid.name) - input_handle = solid.input_handle(input_def.name) - if pipeline.dependency_structure.has_dep(input_handle): - output_handle = pipeline.dependency_structure.get_dep(input_handle) - graphviz_graph.edge(output_handle.solid.name, scoped_name) + if only_solids: + for solid in pipeline.solids: + for input_def in solid.input_defs: + input_handle = solid.input_handle(input_def.name) + if pipeline.dependency_structure.has_dep(input_handle): + output_handle = pipeline.dependency_structure.get_dep(input_handle) + graphviz_graph.edge(output_handle.solid.name, solid.name) + + else: + for solid in pipeline.solids: + for input_def in solid.input_defs: + scoped_name = solid.name + '.' + input_def.name + graphviz_graph.node(scoped_name) + + for solid in pipeline.solids: + for input_def in solid.input_defs: + scoped_name = solid.name + '.' + input_def.name + graphviz_graph.edge(scoped_name, solid.name) + input_handle = solid.input_handle(input_def.name) + if pipeline.dependency_structure.has_dep(input_handle): + output_handle = pipeline.dependency_structure.get_dep(input_handle) + graphviz_graph.edge(output_handle.solid.name, scoped_name) return graphviz_graph From 9c593e8af2bfc533c11e46dba34fd007774f0c85 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Wed, 12 Sep 2018 06:16:03 -0700 Subject: [PATCH 083/103] Fix bug and add graphql tests There was a bug in the graphql so I fixed it and then added a test as well as some test scaffolding so we can test individual graphql tests in the future. --- python_modules/dagit/dagit/schema.py | 65 ++++++---- .../dagit/dagit_tests/test_graphql.py | 111 ++++++++++++++++++ 2 files changed, 153 insertions(+), 23 deletions(-) create mode 100644 python_modules/dagit/dagit_tests/test_graphql.py diff --git a/python_modules/dagit/dagit/schema.py b/python_modules/dagit/dagit/schema.py index 1ab61cde5f63d..131e8788414d1 100644 --- a/python_modules/dagit/dagit/schema.py +++ b/python_modules/dagit/dagit/schema.py @@ -1,5 +1,17 @@ import graphene -from dagster.core.types import (DagsterScalarType, DagsterCompositeType) + +from dagster import ( + ConfigDefinition, + ExpectationDefinition, + InputDefinition, + OutputDefinition, + PipelineContextDefinition, + PipelineDefinition, + SolidDefinition, + check, +) + +from dagster.core.types import DagsterCompositeType class Query(graphene.ObjectType): @@ -7,8 +19,9 @@ class Query(graphene.ObjectType): pipelines = graphene.NonNull(graphene.List(lambda: graphene.NonNull(Pipeline))) def resolve_pipeline(self, info, name): + check.str_param(name, 'name') repository = info.context['repository_container'].repository - return repository.get_pipeline(name) + return Pipeline(repository.get_pipeline(name)) def resolve_pipelines(self, info): repository = info.context['repository_container'].repository @@ -26,9 +39,9 @@ class Pipeline(graphene.ObjectType): def __init__(self, pipeline): super(Pipeline, self).__init__(name=pipeline.name, description=pipeline.description) - self._pipeline = pipeline + self._pipeline = check.inst_param(pipeline, 'pipeline', PipelineDefinition) - def resolve_solids(self, info): + def resolve_solids(self, _info): return [ Solid( solid, self._pipeline.dependency_structure.deps_of_solid_with_input(solid.name), @@ -36,7 +49,7 @@ def resolve_solids(self, info): ) for solid in self._pipeline.solids ] - def resolve_contexts(self, info): + def resolve_contexts(self, _info): return [ PipelineContext(name=name, context=context) for name, context in self._pipeline.context_definitions.items() @@ -50,9 +63,9 @@ class PipelineContext(graphene.ObjectType): def __init__(self, name, context): super(PipelineContext, self).__init__(name=name, description=context.description) - self._context = context + self._context = check.inst_param(context, 'context', PipelineContextDefinition) - def resolve_config(self, info): + def resolve_config(self, _info): return Config(self._context.config_def) @@ -65,7 +78,9 @@ class Solid(graphene.ObjectType): def __init__(self, solid, depends_on=None, depended_by=None): super(Solid, self).__init__(name=solid.name, description=solid.description) - self._solid = solid + + self._solid = check.inst_param(solid, 'solid', SolidDefinition) + if depends_on: self._depends_on = { input_handle.input_def.name: output_handle @@ -82,19 +97,19 @@ def __init__(self, solid, depends_on=None, depended_by=None): else: self._depended_by = {} - def resolve_inputs(self, info): + def resolve_inputs(self, _info): return [ Input(input_definition, self, self._depends_on.get(input_definition.name)) for input_definition in self._solid.input_defs ] - def resolve_outputs(self, info): + def resolve_outputs(self, _info): return [ Output(output_definition, self, self._depended_by.get(output_definition.name, [])) for output_definition in self._solid.output_defs ] - def resolve_config(self, info): + def resolve_config(self, _info): return Config(self._solid.config_def) @@ -112,19 +127,21 @@ def __init__(self, input_definition, solid, depends_on): description=input_definition.description, solid=solid, ) - self._input_definition = input_definition + self._input_definition = check.inst_param( + input_definition, 'input_definition', InputDefinition + ) self._depends_on = depends_on - def resolve_type(self, info): + def resolve_type(self, _info): return Type.from_dagster_type(dagster_type=self._input_definition.dagster_type) - def resolve_expectations(self, info): + def resolve_expectations(self, _info): if self._input_definition.expectations: return [Expectation(expectation for expectation in self._input_definition.expectations)] else: return [] - def resolve_depends_on(self, info): + def resolve_depends_on(self, _info): return Output( self._depends_on.output_def, Solid(self._depends_on.solid), @@ -142,6 +159,7 @@ class Output(graphene.ObjectType): depended_by = graphene.List(lambda: graphene.NonNull(Input)) def __init__(self, output_definition, solid, depended_by): + check.inst_param(output_definition, 'output_definition', OutputDefinition) super(Output, self).__init__( name=output_definition.name, description=output_definition.description, @@ -150,10 +168,10 @@ def __init__(self, output_definition, solid, depended_by): self._output_definition = output_definition self._depended_by = depended_by - def resolve_type(self, info): + def resolve_type(self, _info): return Type.from_dagster_type(dagster_type=self._output_definition.dagster_type) - def resolve_expectations(self, info): + def resolve_expectations(self, _info): if self._output_definition.expectations: return [ Expectation(expectation) for expectation in self._output_definition.expectations @@ -161,7 +179,7 @@ def resolve_expectations(self, info): else: return [] - def resolve_depends_on(self, info): + def resolve_depends_on(self, _info): return [ Input( depended_by.input_def, @@ -177,6 +195,7 @@ class Expectation(graphene.ObjectType): description = graphene.String() def __init__(self, expectation): + check.inst_param(expectation, 'expectation', ExpectationDefinition) super(Expectation, self).__init__( name=expectation.name, description=expectation.description ) @@ -187,9 +206,9 @@ class Config(graphene.ObjectType): def __init__(self, config_def): super(Config, self).__init__() - self._config_def = config_def + self._config_def = check.inst_param(config_def, 'config_def', ConfigDefinition) - def resolve_type(self, info): + def resolve_type(self, _info): return Type.from_dagster_type(dagster_type=self._config_def.config_type) @@ -198,7 +217,7 @@ class Type(graphene.Interface): description = graphene.String() @classmethod - def from_dagster_type(self, dagster_type): + def from_dagster_type(cls, dagster_type): if isinstance(dagster_type, DagsterCompositeType): return CompositeType(dagster_type) else: @@ -233,7 +252,7 @@ def __init__(self, dagster_type): ) self._dagster_type = dagster_type - def resolve_fields(self, info): + def resolve_fields(self, _info): return [TypeField(name=k, field=v) for k, v in self._dagster_type.field_dict.items()] @@ -253,7 +272,7 @@ def __init__(self, name, field): ) self._field = field - def resolve_type(self, info): + def resolve_type(self, _info): return Type.from_dagster_type(dagster_type=self._field.dagster_type) diff --git a/python_modules/dagit/dagit_tests/test_graphql.py b/python_modules/dagit/dagit_tests/test_graphql.py new file mode 100644 index 0000000000000..c13b01c771bbc --- /dev/null +++ b/python_modules/dagit/dagit_tests/test_graphql.py @@ -0,0 +1,111 @@ +from dagster import ( + DependencyDefinition, + PipelineDefinition, + InputDefinition, + OutputDefinition, + RepositoryDefinition, + lambda_solid, +) +import dagster.pandas as dagster_pd + +from dagit.schema import create_schema + +from graphql import graphql + + +@lambda_solid( + inputs=[InputDefinition('num', dagster_pd.DataFrame)], + output=OutputDefinition(dagster_type=dagster_pd.DataFrame), +) +def sum_solid(num): + sum_df = num.copy() + sum_df['sum'] = sum_df['num1'] + sum_df['num2'] + return sum_df + + +@lambda_solid( + inputs=[InputDefinition('sum_df', dagster_pd.DataFrame)], + output=OutputDefinition(dagster_type=dagster_pd.DataFrame), +) +def sum_sq_solid(sum_df): + sum_sq_df = sum_df.copy() + sum_sq_df['sum_sq'] = sum_df['sum']**2 + return sum_sq_df + + +def define_pipeline_one(): + return PipelineDefinition( + name='pandas_hello_world', + solids=[dagster_pd.load_csv_solid('load_num_csv'), sum_solid, sum_sq_solid], + dependencies={ + 'sum_solid': { + 'num': DependencyDefinition('load_num_csv') + }, + 'sum_sq_solid': { + 'sum_df': DependencyDefinition(sum_solid.name), + }, + }, + ) + + +def define_pipeline_two(): + return PipelineDefinition( + name='pandas_hello_world_two', + solids=[dagster_pd.load_csv_solid('load_num_csv'), sum_solid], + dependencies={ + 'sum_solid': { + 'num': DependencyDefinition('load_num_csv') + }, + }, + ) + + +def define_repo(): + return RepositoryDefinition( + name='test', + pipeline_dict={ + 'pandas_hello_world': define_pipeline_one, + 'pandas_hello_world_two': define_pipeline_two, + } + ) + + +class FakeRepositoryContainer: + def __init__(self, repository): + self.repository = repository + + def reload(self): + raise Exception('not implemented for test') + + +def execute_dagster_graphql(repo, query): + return graphql( + create_schema(), + query, + context={'repository_container': FakeRepositoryContainer(repo)}, + ) + + +def test_pipelines(): + result = execute_dagster_graphql(define_repo(), '{ pipelines { name } }') + assert result.data + assert not result.errors + + assert set([p['name'] for p in result.data['pipelines']]) == set( + ['pandas_hello_world', 'pandas_hello_world_two'] + ) + + +def test_pipeline_by_name(): + result = execute_dagster_graphql( + define_repo(), ''' + { + pipeline(name: "pandas_hello_world_two") { + name + } + }''' + ) + + assert result.data + assert not result.errors + assert result.data['pipeline']['name'] == 'pandas_hello_world_two' From c057006ee53e561496b8b536ee557740801b4530 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Mon, 10 Sep 2018 06:30:28 -0700 Subject: [PATCH 084/103] Add tutorial part eight ... and part of nine. And some API changes to make this stuff more smooth --- .../dagit/dagit_tests/test_graphql.py | 4 +- .../dagster_ge_tests/test_pandas_ge.py | 6 +- python_modules/dagster/dagster/__init__.py | 1 + .../dagster/dagster/cli/pipeline.py | 7 +- .../dagster/dagster/cli/repository_config.py | 39 ++--- python_modules/dagster/dagster/config.py | 5 + .../core/core_tests/test_naming_collisions.py | 2 +- .../core/core_tests/test_solid_with_config.py | 25 ++++ .../dagster/dagster/core/definitions.py | 14 +- .../dagster/dagster/core/execution.py | 42 ++++-- python_modules/dagster/dagster/core/types.py | 1 + .../pandas_hello_world/pipeline.py | 6 +- .../dagster/dagster/pandas/__init__.py | 2 +- .../test_pandas_hello_world_library_slide.py | 4 +- .../pandas/pandas_tests/test_pandas_solids.py | 10 +- .../pandas_tests/test_pandas_user_error.py | 4 +- .../dagster/dagster/utils/__init__.py | 6 + .../env_step_one_field_error.yml | 8 ++ .../env_step_one_type_error.yml | 8 ++ .../env_step_one_works.yml | 8 ++ .../env_step_three_type_mismatch.yml | 8 ++ .../env_step_two_works.yml | 8 ++ .../repository.yml | 3 + .../test_intro_tutorial_part_eight.py | 134 ++++++++++++++++++ .../test_intro_tutorial_part_nine.py | 88 ++++++++++++ python_modules/dagster/docs/index.rst | 2 + .../docs/intro_tutorial/part_eight.rst | 131 +++++++++++++++++ .../dagster/docs/intro_tutorial/part_nine.rst | 24 ++++ .../docs/intro_tutorial/part_seven.rst | 19 +-- 29 files changed, 549 insertions(+), 70 deletions(-) create mode 100644 python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/env_step_one_field_error.yml create mode 100644 python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/env_step_one_type_error.yml create mode 100644 python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/env_step_one_works.yml create mode 100644 python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/env_step_three_type_mismatch.yml create mode 100644 python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/env_step_two_works.yml create mode 100644 python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/repository.yml create mode 100644 python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/test_intro_tutorial_part_eight.py create mode 100644 python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/test_intro_tutorial_part_nine.py create mode 100644 python_modules/dagster/docs/intro_tutorial/part_eight.rst create mode 100644 python_modules/dagster/docs/intro_tutorial/part_nine.rst diff --git a/python_modules/dagit/dagit_tests/test_graphql.py b/python_modules/dagit/dagit_tests/test_graphql.py index c13b01c771bbc..a7474cf93a758 100644 --- a/python_modules/dagit/dagit_tests/test_graphql.py +++ b/python_modules/dagit/dagit_tests/test_graphql.py @@ -15,7 +15,7 @@ @lambda_solid( inputs=[InputDefinition('num', dagster_pd.DataFrame)], - output=OutputDefinition(dagster_type=dagster_pd.DataFrame), + output=OutputDefinition(dagster_pd.DataFrame), ) def sum_solid(num): sum_df = num.copy() @@ -25,7 +25,7 @@ def sum_solid(num): @lambda_solid( inputs=[InputDefinition('sum_df', dagster_pd.DataFrame)], - output=OutputDefinition(dagster_type=dagster_pd.DataFrame), + output=OutputDefinition(dagster_pd.DataFrame), ) def sum_sq_solid(sum_df): sum_sq_df = sum_df.copy() diff --git a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py index 1427dc586b749..023ba3cb22809 100644 --- a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py +++ b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py @@ -35,7 +35,7 @@ def col_exists(name, col_name): 'num_df', dagster_pd.DataFrame, expectations=[col_exists('num1_exists', 'num1')] ) ], - output=OutputDefinition(dagster_type=dagster_pd.DataFrame) + output=OutputDefinition(dagster_pd.DataFrame) ) def sum_solid(num_df): return _sum_solid_impl(num_df) @@ -49,7 +49,7 @@ def sum_solid(num_df): expectations=[col_exists('failing', 'not_a_column')], ) ], - output=OutputDefinition(dagster_type=dagster_pd.DataFrame) + output=OutputDefinition(dagster_pd.DataFrame) ) def sum_solid_fails_input_expectation(num_df): return _sum_solid_impl(num_df) @@ -67,7 +67,7 @@ def sum_solid_fails_input_expectation(num_df): ], ), ], - output=OutputDefinition(dagster_type=dagster_pd.DataFrame) + output=OutputDefinition(dagster_pd.DataFrame) ) def sum_solid_expectations_config(num_df): return _sum_solid_impl(num_df) diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index 98b5fdae477a9..387036fe38425 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -48,6 +48,7 @@ 'DependencyDefinition', 'ExpectationDefinition', 'ExpectationResult', + 'Field', 'InputDefinition', 'OutputDefinition', 'PipelineContextDefinition', diff --git a/python_modules/dagster/dagster/cli/pipeline.py b/python_modules/dagster/dagster/cli/pipeline.py index ecfef257fcf73..44801bb48498a 100644 --- a/python_modules/dagster/dagster/cli/pipeline.py +++ b/python_modules/dagster/dagster/cli/pipeline.py @@ -16,6 +16,7 @@ from dagster.core.execution import execute_pipeline_iterator from dagster.core.errors import DagsterExecutionFailureReason from dagster.graphviz import build_graphviz_graph +from dagster.utils import load_yaml_from_path from dagster.utils.indenting_printer import IndentingPrinter from .repository_config import ( @@ -188,12 +189,6 @@ def graphviz_command(conf, name, only_solids): } -def load_yaml_from_path(path): - check.str_param(path, 'path') - with open(path, 'r') as ff: - return yaml.load(ff) - - @click.command(name='execute', help="execute <>") @repository_config_argument @click.argument('name') diff --git a/python_modules/dagster/dagster/cli/repository_config.py b/python_modules/dagster/dagster/cli/repository_config.py index f4d3dd1240936..a8700e5b755b4 100644 --- a/python_modules/dagster/dagster/cli/repository_config.py +++ b/python_modules/dagster/dagster/cli/repository_config.py @@ -11,6 +11,8 @@ RepositoryDefinition, ) +from dagster.utils import load_yaml_from_path + class RepositoryInfo(namedtuple('_RepositoryInfo', 'repository module fn module_name fn_name')): pass @@ -18,25 +20,24 @@ class RepositoryInfo(namedtuple('_RepositoryInfo', 'repository module fn module_ def load_repository_from_file(file_path): check.str_param(file_path, 'file_path') - with open(file_path, 'r') as ff: - config = yaml.load(ff) - repository_config = check.dict_elem(config, 'repository') - module_name = check.opt_str_elem(repository_config, 'module') - file_name = check.opt_str_elem(repository_config, 'file') - fn_name = check.str_elem(repository_config, 'fn') - - if module_name: - module, fn, repository = load_repository_from_module_name(module_name, fn_name) - else: - module, fn, repository = load_repository_from_python_file(file_name, fn_name) - - return RepositoryInfo( - repository=repository, - module=module, - fn=fn, - module_name=module_name, - fn_name=fn_name, - ) + config = load_yaml_from_path(file_path) + repository_config = check.dict_elem(config, 'repository') + module_name = check.opt_str_elem(repository_config, 'module') + file_name = check.opt_str_elem(repository_config, 'file') + fn_name = check.str_elem(repository_config, 'fn') + + if module_name: + module, fn, repository = load_repository_from_module_name(module_name, fn_name) + else: + module, fn, repository = load_repository_from_python_file(file_name, fn_name) + + return RepositoryInfo( + repository=repository, + module=module, + fn=fn, + module_name=module_name, + fn_name=fn_name, + ) def _create_repo_from_module(module, fn_name): diff --git a/python_modules/dagster/dagster/config.py b/python_modules/dagster/dagster/config.py index 70d806327e61a..8eb4c919be816 100644 --- a/python_modules/dagster/dagster/config.py +++ b/python_modules/dagster/dagster/config.py @@ -1,6 +1,7 @@ from collections import namedtuple from dagster import check +from dagster.utils import load_yaml_from_path DEFAULT_CONTEXT_NAME = 'default' @@ -86,3 +87,7 @@ def construct_environment(yml_config_object): solids=_construct_solids(yml_config_object), context=_construct_context(yml_config_object), ) + + +def load_environment(path): + return construct_environment(load_yaml_from_path(path)) diff --git a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py index 356fc87e8fce0..4975c0369be01 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py +++ b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py @@ -27,7 +27,7 @@ def _value_t_fn(_context, conf, _inputs): name=name, description=description, inputs=[], - outputs=[OutputDefinition(dagster_type=types.String)], + outputs=[OutputDefinition(types.String)], config_def=ConfigDefinition.config_dict({ 'value': Field(types.String) }), diff --git a/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py b/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py index 237deb07fb99d..b760c12567873 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py +++ b/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py @@ -2,6 +2,7 @@ from dagster import ( ConfigDefinition, + DagsterInvariantViolationError, Field, PipelineDefinition, SolidDefinition, @@ -65,3 +66,27 @@ def _t_fn(*_args): 'some_config': 1 })}), ) + + +def test_solid_not_found(): + def _t_fn(*_args): + raise Exception('should not reach') + + solid = SolidDefinition( + name='find_me_solid', + inputs=[], + outputs=[], + transform_fn=_t_fn, + ) + + pipeline = PipelineDefinition(solids=[solid]) + + with pytest.raises(DagsterInvariantViolationError): + execute_pipeline( + pipeline, + config.Environment(solids={ + 'not_found': config.Solid({ + 'some_config': 1, + }), + }), + ) diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index 37128310c6927..d1ce850fad389 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -517,6 +517,16 @@ def __validate_dependency_structure(self, name, solids, dependency_structure): '{input_def.name}'.format(input_def=input_def) ) + @property + def display_name(self): + '''Name suitable for exception messages, logging etc. If pipeline + is unnamed the method with return "<>". + + Returns: + str: Display name of pipeline + ''' + return self.name if self.name else '<>' + @property def solids(self): '''Return the solids in the pipeline. @@ -644,13 +654,13 @@ class OutputDefinition(object): the default name of "result". Attributes: - name (str): Name of the output. Defaults to "result". dagster_type (DagsterType): Type of the output. Defaults to types.Any. + name (str): Name of the output. Defaults to "result". expectations List[ExpectationDefinition]: Expectations for this output. description (str): Description of the output. Optional. ''' - def __init__(self, name=None, dagster_type=None, expectations=None, description=None): + def __init__(self, dagster_type=None, name=None, expectations=None, description=None): self.name = check.opt_str_param(name, 'name', DEFAULT_OUTPUT) self.dagster_type = check.opt_inst_param( diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index f3fb77df5dcd6..b748b8543d3a8 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -195,15 +195,15 @@ def _do_throw_on_error(execution_result): execution_result.reraise_user_error() -def _wrap_in_yield(thing): - if isinstance(thing, ExecutionContext): +def _wrap_in_yield(context_or_generator): + if isinstance(context_or_generator, ExecutionContext): def _wrap(): - yield thing + yield context_or_generator return _wrap() - return thing + return context_or_generator def _validate_environment(environment, pipeline): @@ -218,6 +218,17 @@ def _validate_environment(environment, pipeline): ) ) + for solid_name in environment.solids.keys(): + if not pipeline.has_solid(solid_name): + available_solids = [s.name for s in pipeline.solids] + raise DagsterInvariantViolationError( + 'Solid {solid_name} specified in config for pipeline {pipeline} not found.'.format( + solid_name=solid_name, + pipeline=pipeline.display_name, + ) + \ + ' Available solids in pipeline are {solids}.'.format(solids=available_solids) + ) + @contextmanager def yield_context(pipeline, environment): @@ -237,8 +248,8 @@ def yield_context(pipeline, environment): 'Invalid config value: {error_msg}'.format(error_msg=','.join(e.args)) ) - thing = context_definition.context_fn(pipeline, evaluation_result) - return _wrap_in_yield(thing) + context_or_generator = context_definition.context_fn(pipeline, evaluation_result) + return _wrap_in_yield(context_or_generator) def execute_pipeline_iterator(pipeline, environment): @@ -347,12 +358,13 @@ def _execute_graph( check.bool_param(throw_on_error, 'throw_on_error') results = [] - with yield_context(execution_graph.pipeline, environment) as context: - with context.value('pipeline', execution_graph.pipeline.name or '<>'): - for result in _execute_graph_iterator(context, execution_graph, environment): - if throw_on_error: - if not result.success: - _do_throw_on_error(result) - - results.append(result) - return PipelineExecutionResult(context, results) + with yield_context(execution_graph.pipeline, environment) as context, \ + context.value('pipeline', execution_graph.pipeline.display_name): + + for result in _execute_graph_iterator(context, execution_graph, environment): + if throw_on_error: + if not result.success: + _do_throw_on_error(result) + + results.append(result) + return PipelineExecutionResult(context, results) diff --git a/python_modules/dagster/dagster/core/types.py b/python_modules/dagster/dagster/core/types.py index 4ee48784f7df1..326e14f3fe491 100644 --- a/python_modules/dagster/dagster/core/types.py +++ b/python_modules/dagster/dagster/core/types.py @@ -293,3 +293,4 @@ def process_incoming_composite_value(dagster_composite_type, incoming_value, cto Int = _DagsterIntType() Bool = _DagsterBoolType() Any = _DagsterAnyType() +Dict = PythonObjectType('Dict', dict) diff --git a/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py b/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py index 81821ccf4dd28..a2cfc1465ed57 100644 --- a/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py +++ b/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py @@ -11,7 +11,7 @@ @lambda_solid( inputs=[InputDefinition('num', dagster_pd.DataFrame)], - output=OutputDefinition(dagster_type=dagster_pd.DataFrame), + output=OutputDefinition(dagster_pd.DataFrame), ) def sum_solid(num): sum_df = num.copy() @@ -21,7 +21,7 @@ def sum_solid(num): @lambda_solid( inputs=[InputDefinition('sum_df', dagster_pd.DataFrame)], - output=OutputDefinition(dagster_type=dagster_pd.DataFrame), + output=OutputDefinition(dagster_pd.DataFrame), ) def sum_sq_solid(sum_df): sum_sq_df = sum_df.copy() @@ -31,7 +31,7 @@ def sum_sq_solid(sum_df): @lambda_solid( inputs=[InputDefinition('sum_sq_solid', dagster_pd.DataFrame)], - output=OutputDefinition(dagster_type=dagster_pd.DataFrame), + output=OutputDefinition(dagster_pd.DataFrame), ) def always_fails_solid(**_kwargs): raise Exception('I am a programmer and I make error') diff --git a/python_modules/dagster/dagster/pandas/__init__.py b/python_modules/dagster/dagster/pandas/__init__.py index aacebfc29d5ce..b0fbe327d8fd1 100644 --- a/python_modules/dagster/dagster/pandas/__init__.py +++ b/python_modules/dagster/dagster/pandas/__init__.py @@ -37,7 +37,7 @@ def _t_fn(_context, conf, _inputs): return SolidDefinition( name=name, inputs=[], - outputs=[OutputDefinition(dagster_type=DataFrame)], + outputs=[OutputDefinition(DataFrame)], transform_fn=_t_fn, config_def=ConfigDefinition.config_dict({ 'path': Field(types.Path), diff --git a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_library_slide.py b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_library_slide.py index b7f9f2793993e..3374ce51a55b5 100644 --- a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_library_slide.py +++ b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_library_slide.py @@ -116,7 +116,7 @@ def transform_fn(_context, inputs): name='hello_world', inputs=[table_input], transform_fn=transform_fn, - output=OutputDefinition(dagster_type=dagster_pd.DataFrame) + output=OutputDefinition(dagster_pd.DataFrame) ) return hello_world @@ -124,7 +124,7 @@ def transform_fn(_context, inputs): def create_decorator_based_solid(): @lambda_solid( inputs=[InputDefinition('num_csv', dagster_pd.DataFrame)], - output=OutputDefinition(dagster_type=dagster_pd.DataFrame), + output=OutputDefinition(dagster_pd.DataFrame), ) def hello_world(num_csv): num_csv['sum'] = num_csv['num1'] + num_csv['num2'] diff --git a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py index 6f685233fa823..f1e2314bbfdc0 100644 --- a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py +++ b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py @@ -34,7 +34,7 @@ def _dataframe_solid(name, inputs, transform_fn): name=name, inputs=inputs, transform_fn=transform_fn, - output=OutputDefinition(dagster_type=dagster_pd.DataFrame), + output=OutputDefinition(dagster_pd.DataFrame), ) @@ -117,7 +117,7 @@ def transform(context, inputs): name='sum_table', inputs=[csv_input], transform_fn=transform, - output=OutputDefinition(dagster_type=dagster_pd.DataFrame), + output=OutputDefinition(dagster_pd.DataFrame), ) output_df = execute_transform_in_temp_csv_files(solid_def) @@ -178,7 +178,7 @@ def transform(_context, inputs): @lambda_solid( inputs=[InputDefinition('num_csv', dagster_pd.DataFrame)], - output=OutputDefinition(dagster_type=dagster_pd.DataFrame), + output=OutputDefinition(dagster_pd.DataFrame), ) def sum_table(num_csv): check.inst_param(num_csv, 'num_csv', pd.DataFrame) @@ -188,7 +188,7 @@ def sum_table(num_csv): @lambda_solid( inputs=[InputDefinition('sum_df', dagster_pd.DataFrame)], - output=OutputDefinition(dagster_type=dagster_pd.DataFrame), + output=OutputDefinition(dagster_pd.DataFrame), ) def sum_sq_table(sum_df): sum_df['sum_squared'] = sum_df['sum'] * sum_df['sum'] @@ -197,7 +197,7 @@ def sum_sq_table(sum_df): @lambda_solid( inputs=[InputDefinition('sum_table_renamed', dagster_pd.DataFrame)], - output=OutputDefinition(dagster_type=dagster_pd.DataFrame), + output=OutputDefinition(dagster_pd.DataFrame), ) def sum_sq_table_renamed_input(sum_table_renamed): sum_table_renamed['sum_squared'] = sum_table_renamed['sum'] * sum_table_renamed['sum'] diff --git a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_user_error.py b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_user_error.py index bf765a3e51bcf..b4b20f3d872fe 100644 --- a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_user_error.py +++ b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_user_error.py @@ -24,7 +24,7 @@ def _dataframe_solid(name, inputs, transform_fn): name=name, inputs=inputs, transform_fn=transform_fn, - output=OutputDefinition(dagster_type=dagster_pd.DataFrame), + output=OutputDefinition(dagster_pd.DataFrame), ) @@ -34,7 +34,7 @@ def test_wrong_output_value(): @lambda_solid( name="test_wrong_output", inputs=[csv_input], - output=OutputDefinition(dagster_type=dagster_pd.DataFrame), + output=OutputDefinition(dagster_pd.DataFrame), ) def df_solid(num_csv): return 'not a dataframe' diff --git a/python_modules/dagster/dagster/utils/__init__.py b/python_modules/dagster/dagster/utils/__init__.py index 8f7979dbae965..86e19693d0ecf 100644 --- a/python_modules/dagster/dagster/utils/__init__.py +++ b/python_modules/dagster/dagster/utils/__init__.py @@ -1,5 +1,6 @@ import inspect import os +import yaml from dagster import check @@ -16,3 +17,8 @@ def script_relative_path(file_path): check.str_param(file_path, 'file_path') scriptdir = inspect.stack()[1][1] return os.path.join(os.path.dirname(os.path.abspath(scriptdir)), file_path) + +def load_yaml_from_path(path): + check.str_param(path, 'path') + with open(path, 'r') as ff: + return yaml.load(ff) diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/env_step_one_field_error.yml b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/env_step_one_field_error.yml new file mode 100644 index 0000000000000..fee595474631a --- /dev/null +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/env_step_one_field_error.yml @@ -0,0 +1,8 @@ +context: + config: + log_level: DEBUG + +solids: + double_the_word_with_typed_config: + config: + wrong_word: quux diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/env_step_one_type_error.yml b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/env_step_one_type_error.yml new file mode 100644 index 0000000000000..135c764ae3f4a --- /dev/null +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/env_step_one_type_error.yml @@ -0,0 +1,8 @@ +context: + config: + log_level: DEBUG + +solids: + double_the_word_with_typed_config: + config: + word: 1 diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/env_step_one_works.yml b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/env_step_one_works.yml new file mode 100644 index 0000000000000..5763d4c3b9c26 --- /dev/null +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/env_step_one_works.yml @@ -0,0 +1,8 @@ +context: + config: + log_level: DEBUG + +solids: + double_the_word_with_typed_config: + config: + word: quux diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/env_step_three_type_mismatch.yml b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/env_step_three_type_mismatch.yml new file mode 100644 index 0000000000000..d5fdbdf6aeb8d --- /dev/null +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/env_step_three_type_mismatch.yml @@ -0,0 +1,8 @@ +context: + config: + log_level: DEBUG + +solids: + typed_double_word_mismatch: + config: + word: quux diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/env_step_two_works.yml b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/env_step_two_works.yml new file mode 100644 index 0000000000000..bdfe74fa477eb --- /dev/null +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/env_step_two_works.yml @@ -0,0 +1,8 @@ +context: + config: + log_level: DEBUG + +solids: + typed_double_word: + config: + word: baaz diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/repository.yml b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/repository.yml new file mode 100644 index 0000000000000..07b00b2eab3a0 --- /dev/null +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/repository.yml @@ -0,0 +1,3 @@ +repository: + file: test_intro_tutorial_part_eight.py + fn: define_part_eight_repo diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/test_intro_tutorial_part_eight.py b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/test_intro_tutorial_part_eight.py new file mode 100644 index 0000000000000..04a0d5d22ace1 --- /dev/null +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/test_intro_tutorial_part_eight.py @@ -0,0 +1,134 @@ +# pylint: disable=W0622,W0614,W0401 +from collections import defaultdict +import pytest + +from dagster import * + +from dagster.utils import script_relative_path + + +@solid(config_def=ConfigDefinition(types.ConfigDictionary({'word': Field(types.String)}))) +def double_the_word_with_typed_config(_context, conf): + return conf['word'] * 2 + + +@solid( + config_def=ConfigDefinition(types.ConfigDictionary({ + 'word': Field(types.String) + })), + outputs=[OutputDefinition(types.String)], +) +def typed_double_word(_context, conf): + return conf['word'] * 2 + + +@solid( + config_def=ConfigDefinition(types.ConfigDictionary({ + 'word': Field(types.String) + })), + outputs=[OutputDefinition(types.Int)], +) +def typed_double_word_mismatch(_context, conf): + return conf['word'] * 2 + + +@lambda_solid(inputs=[InputDefinition('word')]) +def count_letters(word): + counts = defaultdict(int) + for letter in word: + counts[letter] += 1 + return dict(counts) + + +def define_part_eight_step_one_pipeline(): + return PipelineDefinition( + name='part_eight_step_one', + solids=[double_the_word_with_typed_config, count_letters], + dependencies={ + 'count_letters': { + 'word': DependencyDefinition('double_the_word_with_typed_config'), + }, + }, + ) + + +def define_part_eight_step_two_pipeline(): + return PipelineDefinition( + name='part_eight_step_two', + solids=[typed_double_word, count_letters], + dependencies={ + 'count_letters': { + 'word': DependencyDefinition('typed_double_word'), + }, + }, + ) + + +def define_part_eight_step_three_pipeline(): + return PipelineDefinition( + name='part_eight_step_three', + solids=[typed_double_word_mismatch, count_letters], + dependencies={ + 'count_letters': { + 'word': DependencyDefinition('typed_double_word_mismatch'), + }, + }, + ) + + +def define_part_eight_repo(): + return RepositoryDefinition( + name='part_eight_repo', + pipeline_dict={ + 'part_eight_step_one': define_part_eight_step_one_pipeline, + 'part_eight_step_two': define_part_eight_step_two_pipeline, + 'part_eight_step_three': define_part_eight_step_three_pipeline, + } + ) + + +def test_part_eight_repo_step_one(): + environment = config.load_environment(script_relative_path('env_step_one_works.yml')) + + pipeline_result = execute_pipeline(define_part_eight_step_one_pipeline(), environment) + + assert pipeline_result.success + assert pipeline_result.result_for_solid('double_the_word_with_typed_config' + ).transformed_value() == 'quuxquux' + assert pipeline_result.result_for_solid('count_letters').transformed_value() == { + 'q': 2, + 'u': 4, + 'x': 2, + } + + +def test_part_eight_repo_step_one_wrong_env(): + environment = config.load_environment(script_relative_path('env_step_one_type_error.yml')) + with pytest.raises(DagsterTypeError, match='Expected valid value for String'): + execute_pipeline(define_part_eight_step_one_pipeline(), environment) + + +def test_part_eight_repo_step_one_wrong_field(): + environment = config.load_environment(script_relative_path('env_step_one_field_error.yml')) + + with pytest.raises(DagsterTypeError, match='Field wrong_word not found'): + execute_pipeline(define_part_eight_step_one_pipeline(), environment) + + +def test_part_eight_repo_step_two(): + environment = config.load_environment(script_relative_path('env_step_two_works.yml')) + + pipeline_result = execute_pipeline(define_part_eight_step_two_pipeline(), environment) + + assert pipeline_result.success + assert pipeline_result.result_for_solid('typed_double_word').transformed_value() == 'baazbaaz' + + +def test_part_eight_repo_step_three(): + environment = config.load_environment(script_relative_path('env_step_three_type_mismatch.yml')) + + with pytest.raises( + DagsterInvariantViolationError, + match='Solid typed_double_word_mismatch output name result output quuxquux' + ): + execute_pipeline(define_part_eight_step_three_pipeline(), environment) diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/test_intro_tutorial_part_nine.py b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/test_intro_tutorial_part_nine.py new file mode 100644 index 0000000000000..1a6fed029365d --- /dev/null +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/test_intro_tutorial_part_nine.py @@ -0,0 +1,88 @@ +# pylint: disable=W0622,W0614,W0401 +from dagster import * + + +class PublicCloudConn: + pass + + +def create_public_cloud_conn(_creds): + return PublicCloudConn() + + +def set_value_in_cloud_store(_conn, _key, _value): + # imagine this doing something + pass + + +class PublicCloudStoreResource: + def __init__(self, credentials): + # create credential and store it + self.conn = create_public_cloud_conn(credentials) + + def record_value(self, key, value): + set_value_in_cloud_store(self.conn, key, value) + + +@solid(config_def=ConfigDefinition(types.Int), outputs=[OutputDefinition(types.Int)]) +def injest_a(_context, conf): + return conf + + +@solid( + config_def=ConfigDefinition(types.Int), + outputs=[OutputDefinition(types.Int)], +) +def injest_b(_context, conf): + return conf + + +@solid( + inputs=[InputDefinition('num_one', types.Int), + InputDefinition('num_two', types.Int)], + outputs=[OutputDefinition(types.Int)], +) +def add_ints(_context, _conf, num_one, num_two): + return num_one + num_two + + +@solid( + inputs=[InputDefinition('num_one', types.Int), + InputDefinition('num_two', types.Int)], + outputs=[OutputDefinition(types.Int)], +) +def mult_ints(_context, _conf, num_one, num_two): + return num_one * num_two + + +def define_part_nine_step_one(): + return PipelineDefinition( + name='part_nine_step_one', + solids=[injest_a, injest_b, add_ints, mult_ints], + dependencies={ + 'add_ints': { + 'num_one': DependencyDefinition('injest_a'), + 'num_two': DependencyDefinition('injest_b'), + }, + 'mult_ints': { + 'num_one': DependencyDefinition('injest_a'), + 'num_two': DependencyDefinition('injest_b'), + }, + }, + ) + + +def test_intro_tutorial_part_nine_step_one(): + result = execute_pipeline( + define_part_nine_step_one(), + config.Environment(solids={ + 'injest_a': config.Solid(2), + 'injest_b': config.Solid(3), + }) + ) + + assert result.success + assert result.result_for_solid('injest_a').transformed_value() == 2 + assert result.result_for_solid('injest_b').transformed_value() == 3 + assert result.result_for_solid('add_ints').transformed_value() == 5 + assert result.result_for_solid('mult_ints').transformed_value() == 6 diff --git a/python_modules/dagster/docs/index.rst b/python_modules/dagster/docs/index.rst index d5f7080fda818..fb9726739f747 100644 --- a/python_modules/dagster/docs/index.rst +++ b/python_modules/dagster/docs/index.rst @@ -28,6 +28,8 @@ Intro Tutorial intro_tutorial/part_five intro_tutorial/part_six intro_tutorial/part_seven + intro_tutorial/part_eight + intro_tutorial/part_nine API Reference diff --git a/python_modules/dagster/docs/intro_tutorial/part_eight.rst b/python_modules/dagster/docs/intro_tutorial/part_eight.rst new file mode 100644 index 0000000000000..435178e0cb18a --- /dev/null +++ b/python_modules/dagster/docs/intro_tutorial/part_eight.rst @@ -0,0 +1,131 @@ +Basic Typing +------ + +Dagster includes an optional type system that can be applied to both runtime values +and configuration. We can use these types to both provide runtime type guarantees +as well as improve documentation and understandability. + +There actually *have* been types during all previous parts of this tutorial. If the +use does not specify types for inputs, outputs, or config in dagster, they default +to the Any type, which can accept any and all values. + +We are going to incrementally add typing to the example in part eight. + +Before we had this: + +.. code-block:: python + + from dagster import * + + @solid + def double_the_word(_context, conf): + return conf['word'] * 2 + +We are going to make the configuration of this strongly typed prevent errors and improve +documentation. + +.. code-block:: python + + from dagster import * + + @solid(config_def=ConfigDefinition(types.ConfigDictionary({'word': Field(types.String)}))) + def double_the_word_with_typed_config(_context, conf): + return conf['word'] * 2 + +The previous env.yml file works as before: + +.. code-block:: yaml + + context: + config: + log_level: DEBUG + + solids: + double_the_word_with_typed_config: + config: + word: quux + +Now let's imagine we made a mistake and passed an int to word: + +.. code-block:: yaml + + context: + config: + log_level: DEBUG + + solids: + double_the_word_with_typed_config: + config: + word: 1 + +And then ran it: + +.. code-block:: sh + + $ dagster pipeline execute part_eight -e env.yml + ... + dagster.core.errors.DagsterTypeError: Error evaluating config for double_the_word_with_typed_config: Expected valid value for String but got 1 + +Or if we passed the wrong field: + +.. code-block:: yaml + + context: + config: + log_level: DEBUG + + solids: + double_the_word_with_typed_config: + config: + wrong_word: quux + +And then ran it: + +.. code-block:: sh + + $ dagster pipeline execute part_eight -e env.yml + dagster.core.errors.DagsterTypeError: Error evaluating config for double_the_word_with_typed_config: Field wrong_word not found. Defined fields: {'word'} + +The type system is also used to evaluate the runtime values that flow between solids, +not just config. Types are attached, optionally, to inputs and outputs. If a type is not +specified, it defaults to the Any type. + +.. code-block:: python + + @solid( + config_def=ConfigDefinition(types.ConfigDictionary({ + 'word': Field(types.String) + })), + outputs=[OutputDefinition(types.String)], + ) + def typed_double_word(_context, conf): + return conf['word'] * 2 + +You'll see here that now the output is annotated with a type. This both ensures +that the runtime value conforms requirements specified by the type (in this case +an instanceof check on a string) and also provides metadata to view in tools such +as dagit. That the output is a string is now guaranteed by the system. If you +violate this, execution halts. + +So imagine we made a coding error (mistyped the output) such as: + +.. code-block:: python + + @solid( + config_def=ConfigDefinition(types.ConfigDictionary({ + 'word': Field(types.String) + })), + outputs=[OutputDefinition(types.Int)], + ) + def typed_double_word(_context, conf): + return conf['word'] * 2 + +When we run it, it errors: + +.. code-block:: sh + + $ dagster pipeline execute part_eight -e env.yml + dagster.core.errors.DagsterInvariantViolationError: Solid typed_double_word_mismatch output name result + output quuxquux type failure: Expected valid value for Int but got 'quuxquux' + + diff --git a/python_modules/dagster/docs/intro_tutorial/part_nine.rst b/python_modules/dagster/docs/intro_tutorial/part_nine.rst new file mode 100644 index 0000000000000..3828db9929c8b --- /dev/null +++ b/python_modules/dagster/docs/intro_tutorial/part_nine.rst @@ -0,0 +1,24 @@ +Custom Contexts +--------------- + +So far we have used contexts for configuring logging level only. This barely scratched the surface +of its capabilities. + +Testing data pipelines, for a number of reasons, is notoriously difficult. One of the reasons is +that as one moves a data pipeline from local development, to unit testing, to integration testing, to CI/CD, +to production, or to whatever environment you need to operate in, the operating environment can +change dramatically. + +In order to handle this, whenever the business logic of a pipeline is interacting with external resources +or dealing with pipeline-wide state generally, the dagster user is expected to interact with these resources +and that state via the context object. Examples would include database connections, connections to cloud services, +interactions with scratch directories on your local filesystem, and so on. + +Let's imagine a scenario where we want to record some custom state in a key-value store for our execution runs. +A production time, this key-value store is a live store (e.g. DynamoDB in amazon) but we do not want to interact +this store for unit-testing. Contexts will be our tool to accomplish this. Goal. + +We're going to have a simple pipeline that does some rudimentary arithmetic, but that wants record +the result computations in that key value store. + +# TODO: do this diff --git a/python_modules/dagster/docs/intro_tutorial/part_seven.rst b/python_modules/dagster/docs/intro_tutorial/part_seven.rst index 034fbcc369a1c..d41c10e067aa0 100644 --- a/python_modules/dagster/docs/intro_tutorial/part_seven.rst +++ b/python_modules/dagster/docs/intro_tutorial/part_seven.rst @@ -13,7 +13,7 @@ and a yaml file so that the CLI tool can know about the repository. @solid def double_the_word(_context, conf): - return conf['config_key'] * 2 + return conf['word'] * 2 @lambda_solid(inputs=[InputDefinition('word')]) def count_letters(word): @@ -40,6 +40,7 @@ and a yaml file so that the CLI tool can know about the repository. 'part_seven': define_part_seven_pipeline, }, ) + And now the repository file: .. code-block:: yaml @@ -69,11 +70,11 @@ With these elements in place we can now drive execution from the CLI .. code-block:: sh dagster pipeline execute part_seven -e env.yml - 2018-09-09 11:47:38 - dagster - DEBUG - message="About to execute the compute node graph in the following order ['double_the_word.transform', 'count_letters.transform']" - 2018-09-09 11:47:38 - dagster - DEBUG - message="Entering execute_compute_nodes loop. Order: ['double_the_word.transform', 'count_letters.transform']" - 2018-09-09 11:47:38 - dagster - DEBUG - message="Executing core transform for solid double_the_word." solid=double_the_word - 2018-09-09 11:47:38 - dagster - DEBUG - message="Solid double_the_word emitted output "result" value 'barbar'" solid=double_the_word - 2018-09-09 11:47:38 - dagster - INFO - metric:core_transform_time_ms=0.136 solid=double_the_word - 2018-09-09 11:47:38 - dagster - DEBUG - message="Executing core transform for solid count_letters." solid=count_letters - 2018-09-09 11:47:38 - dagster - DEBUG - message="Solid count_letters emitted output "result" value {'b': 2, 'a': 2, 'r': 2}" solid=count_letters - 2018-09-09 11:47:38 - dagster - INFO - metric:core_transform_time_ms=0.130 solid=count_letters \ No newline at end of file + 2018-09-10 06:29:41 - dagster - DEBUG - orig_message="About to execute the compute node graph in the following order ['double_the_word.transform', 'count_letters.transform']" log_message_id="12c7c3f0-ea99-44ce-bd1e-5c362560795a" + 2018-09-10 06:29:41 - dagster - DEBUG - orig_message="Entering execute_compute_nodes loop. Order: ['double_the_word.transform', 'count_letters.transform']" log_message_id="c6750058-1a1c-49cd-b529-e149cd6fee27" + 2018-09-10 06:29:41 - dagster - DEBUG - orig_message="Executing core transform for solid double_the_word." log_message_id="09eb5f50-3681-4594-a1e5-d1a007630a47" solid="double_the_word" + 2018-09-10 06:29:41 - dagster - DEBUG - orig_message="Solid double_the_word emitted output \"result\" value 'barbar'" log_message_id="44af0ca9-a62c-472d-ae9b-91cd0a66fe8d" solid="double_the_word" + 2018-09-10 06:29:41 - dagster - DEBUG - orig_message="Finished executing transform for solid double_the_word. Time elapsed: 0.269 ms" log_message_id="ad8f806d-6022-4b81-865d-cc33bff03e0f" solid="double_the_word" execution_time_ms=0.2689361572265625 + 2018-09-10 06:29:41 - dagster - DEBUG - orig_message="Executing core transform for solid count_letters." log_message_id="108c91b6-4503-4004-947b-1d5ccb77698d" solid="count_letters" + 2018-09-10 06:29:41 - dagster - DEBUG - orig_message="Solid count_letters emitted output \"result\" value {'b': 2, 'a': 2, 'r': 2}" log_message_id="000cdb0e-7471-42c1-9616-281eaea28f6c" solid="count_letters" + 2018-09-10 06:29:41 - dagster - DEBUG - orig_message="Finished executing transform for solid count_letters. Time elapsed: 0.160 ms" log_message_id="9bb52b66-519c-4301-abec-0ff1b6a62eae" solid="count_letters" execution_time_ms=0.16021728515625 \ No newline at end of file From 99ca551dd10bf875434f29f890c359d2e8b6ad2f Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Wed, 12 Sep 2018 12:16:21 -0700 Subject: [PATCH 085/103] Part Nine Rough Draft --- python_modules/dagster/dagster/config.py | 11 +- .../dagster/dagster/core/execution.py | 5 + .../dagster/dagster/core/execution_context.py | 13 +- .../env_cloud.yml | 13 + .../env_local.yml | 9 + .../env_step_one.yml | 10 + .../repository.yml | 3 + .../test_intro_tutorial_part_nine.py | 283 +++++++++++-- python_modules/dagster/docs/apidocs/types.rst | 3 - .../docs/intro_tutorial/part_eight.rst | 2 +- .../dagster/docs/intro_tutorial/part_nine.rst | 373 +++++++++++++++++- 11 files changed, 677 insertions(+), 48 deletions(-) create mode 100644 python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/env_cloud.yml create mode 100644 python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/env_local.yml create mode 100644 python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/env_step_one.yml create mode 100644 python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/repository.yml diff --git a/python_modules/dagster/dagster/config.py b/python_modules/dagster/dagster/config.py index 8eb4c919be816..1b4c01bbce333 100644 --- a/python_modules/dagster/dagster/config.py +++ b/python_modules/dagster/dagster/config.py @@ -13,8 +13,8 @@ def __new__(cls, name=None, config=None): cls, check.opt_str_param(name, 'name', DEFAULT_CONTEXT_NAME), config, - ) + ) class Solid(namedtuple('Solid', 'config')): def __new__(cls, config): @@ -51,7 +51,10 @@ def __new__(cls, evaluate): def _construct_context(yml_config_object): context_obj = check.opt_dict_elem(yml_config_object, 'context') if context_obj: - return Context(check.opt_str_elem(context_obj, 'name'), context_obj['config']) + return Context( + check.opt_str_elem(context_obj, 'name'), + context_obj.get('config'), + ) else: return None @@ -74,8 +77,8 @@ def _construct_solids(yml_config_object): solid_configs = {} for solid_name, solid_yml_object in solid_dict.items(): - config_dict = check.dict_elem(solid_yml_object, 'config') - solid_configs[solid_name] = Solid(config_dict) + config_value = solid_yml_object['config'] + solid_configs[solid_name] = Solid(config_value) return solid_configs diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index b748b8543d3a8..624c2182acc8e 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -361,6 +361,11 @@ def _execute_graph( with yield_context(execution_graph.pipeline, environment) as context, \ context.value('pipeline', execution_graph.pipeline.display_name): + context.info( + 'Beginning execution of pipeline {pipeline}', + pipeline=execution_graph.pipeline.display_name, + ) + for result in _execute_graph_iterator(context, execution_graph, environment): if throw_on_error: if not result.success: diff --git a/python_modules/dagster/dagster/core/execution_context.py b/python_modules/dagster/dagster/core/execution_context.py index 6570bb7083cb6..a277cb4571685 100644 --- a/python_modules/dagster/dagster/core/execution_context.py +++ b/python_modules/dagster/dagster/core/execution_context.py @@ -9,7 +9,11 @@ from contextlib import contextmanager from dagster import check -from dagster.utils.logging import CompositeLogger +from dagster.utils.logging import ( + CompositeLogger, + INFO, + define_colored_console_logger, +) Metric = namedtuple('Metric', 'context_dict metric_name value') @@ -50,6 +54,13 @@ def __init__(self, loggers=None, resources=None): self._metrics = [] self.resources = resources + @staticmethod + def console_logging(log_level=INFO, resources=None): + return ExecutionContext( + loggers=[define_colored_console_logger('dagster', log_level)], + resources=resources, + ) + def _log(self, method, orig_message, message_props): check.str_param(method, 'method') check.str_param(orig_message, 'orig_message') diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/env_cloud.yml b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/env_cloud.yml new file mode 100644 index 0000000000000..86f6ca2c459c7 --- /dev/null +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/env_cloud.yml @@ -0,0 +1,13 @@ +context: + name: cloud + config: + credentials: + user: some_user + pass: some_password + +solids: + injest_a: + config: 2 + injest_b: + config: 3 + diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/env_local.yml b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/env_local.yml new file mode 100644 index 0000000000000..d4fa7a49bffbe --- /dev/null +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/env_local.yml @@ -0,0 +1,9 @@ +context: + name: local + +solids: + injest_a: + config: 2 + injest_b: + config: 3 + diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/env_step_one.yml b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/env_step_one.yml new file mode 100644 index 0000000000000..8904940fb0b8c --- /dev/null +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/env_step_one.yml @@ -0,0 +1,10 @@ +context: + config: + log_level: DEBUG + +solids: + injest_a: + config: 2 + injest_b: + config: 3 + diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/repository.yml b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/repository.yml new file mode 100644 index 0000000000000..dd5dd422f8638 --- /dev/null +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/repository.yml @@ -0,0 +1,3 @@ +repository: + file: test_intro_tutorial_part_nine.py + fn: define_part_nine_repo diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/test_intro_tutorial_part_nine.py b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/test_intro_tutorial_part_nine.py index 1a6fed029365d..1bc48819b52f5 100644 --- a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/test_intro_tutorial_part_nine.py +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/test_intro_tutorial_part_nine.py @@ -1,64 +1,125 @@ # pylint: disable=W0622,W0614,W0401 -from dagster import * +from collections import namedtuple +from logging import DEBUG +import pytest -class PublicCloudConn: - pass +from dagster import * -def create_public_cloud_conn(_creds): - return PublicCloudConn() +class PublicCloudConn: + def __init__(self, creds): + self.creds = creds def set_value_in_cloud_store(_conn, _key, _value): # imagine this doing something pass - -class PublicCloudStoreResource: +class PublicCloudStore: def __init__(self, credentials): # create credential and store it - self.conn = create_public_cloud_conn(credentials) + self.conn = PublicCloudConn(credentials) - def record_value(self, key, value): + def record_value(self, context, key, value): + context.info('Setting key={key} value={value} in cloud'.format(key=key, value=value)) set_value_in_cloud_store(self.conn, key, value) -@solid(config_def=ConfigDefinition(types.Int), outputs=[OutputDefinition(types.Int)]) -def injest_a(_context, conf): - return conf +class InMemoryStore: + def __init__(self): + self.values = {} + + def record_value(self, context, key, value): + context.info('Setting key={key} value={value} in memory'.format(key=key, value=value)) + self.values[key] = value + + +PartNineResources = namedtuple('PartNineResources', 'store') + + +def define_contextless_solids(): + @solid( + config_def=ConfigDefinition(types.Int), + outputs=[OutputDefinition(types.Int)], + ) + def injest_a(_context, conf): + return conf + + + @solid( + config_def=ConfigDefinition(types.Int), + outputs=[OutputDefinition(types.Int)], + ) + def injest_b(_context, conf): + return conf + + + @solid( + inputs=[InputDefinition('num_one', types.Int), + InputDefinition('num_two', types.Int)], + outputs=[OutputDefinition(types.Int)], + ) + def add_ints(_context, _conf, num_one, num_two): + return num_one + num_two + + + @solid( + inputs=[InputDefinition('num_one', types.Int), + InputDefinition('num_two', types.Int)], + outputs=[OutputDefinition(types.Int)], + ) + def mult_ints(_context, _conf, num_one, num_two): + return num_one * num_two + + return [injest_a, injest_b, add_ints, mult_ints] +def define_contextful_solids(): + @solid( + config_def=ConfigDefinition(types.Int), + outputs=[OutputDefinition(types.Int)], + ) + def injest_a(context, conf): + context.resources.store.record_value(context, 'a', conf) + return conf -@solid( - config_def=ConfigDefinition(types.Int), - outputs=[OutputDefinition(types.Int)], -) -def injest_b(_context, conf): - return conf + + @solid( + config_def=ConfigDefinition(types.Int), + outputs=[OutputDefinition(types.Int)], + ) + def injest_b(context, conf): + context.resources.store.record_value(context, 'b', conf) + return conf -@solid( - inputs=[InputDefinition('num_one', types.Int), - InputDefinition('num_two', types.Int)], - outputs=[OutputDefinition(types.Int)], -) -def add_ints(_context, _conf, num_one, num_two): - return num_one + num_two + @solid( + inputs=[InputDefinition('num_one', types.Int), + InputDefinition('num_two', types.Int)], + outputs=[OutputDefinition(types.Int)], + ) + def add_ints(context, _conf, num_one, num_two): + result = num_one + num_two + context.resources.store.record_value(context, 'add', result) + return result -@solid( - inputs=[InputDefinition('num_one', types.Int), - InputDefinition('num_two', types.Int)], - outputs=[OutputDefinition(types.Int)], -) -def mult_ints(_context, _conf, num_one, num_two): - return num_one * num_two + @solid( + inputs=[InputDefinition('num_one', types.Int), + InputDefinition('num_two', types.Int)], + outputs=[OutputDefinition(types.Int)], + ) + def mult_ints(context, _conf, num_one, num_two): + result = num_one * num_two + context.resources.store.record_value(context, 'mult', result) + return result + return [injest_a, injest_b, add_ints, mult_ints] def define_part_nine_step_one(): return PipelineDefinition( name='part_nine_step_one', - solids=[injest_a, injest_b, add_ints, mult_ints], + solids=define_contextless_solids(), dependencies={ 'add_ints': { 'num_one': DependencyDefinition('injest_a'), @@ -72,13 +133,90 @@ def define_part_nine_step_one(): ) +PartNineResources = namedtuple('PartNineResources', 'store') + +def define_part_nine_step_two(): + return PipelineDefinition( + name='part_nine_step_two', + solids=define_contextful_solids(), + dependencies={ + 'add_ints': { + 'num_one': DependencyDefinition('injest_a'), + 'num_two': DependencyDefinition('injest_b'), + }, + 'mult_ints': { + 'num_one': DependencyDefinition('injest_a'), + 'num_two': DependencyDefinition('injest_b'), + }, + }, + context_definitions={ + 'local': + PipelineContextDefinition( + context_fn=lambda _pipeline_def, _conf: + ExecutionContext.console_logging( + log_level=DEBUG, + resources=PartNineResources(InMemoryStore()) + ) + ), + } + ) + + +def define_part_nine_final(): + return PipelineDefinition( + name='part_nine_final', + solids=define_contextful_solids(), + dependencies={ + 'add_ints': { + 'num_one': DependencyDefinition('injest_a'), + 'num_two': DependencyDefinition('injest_b'), + }, + 'mult_ints': { + 'num_one': DependencyDefinition('injest_a'), + 'num_two': DependencyDefinition('injest_b'), + }, + }, + context_definitions={ + 'local': + PipelineContextDefinition( + context_fn=lambda _pipeline_def, _conf: ExecutionContext.console_logging( + log_level=DEBUG, + resources=PartNineResources(InMemoryStore()) + ) + ), + 'cloud': + PipelineContextDefinition( + context_fn=lambda _pipeline_def, conf: ExecutionContext.console_logging( + resources=PartNineResources(PublicCloudStore(conf['credentials'])) + ), + config_def=ConfigDefinition(config_type=types.ConfigDictionary({ + 'credentials': Field(types.ConfigDictionary({ + 'user' : Field(types.String), + 'pass' : Field(types.String), + })), + })), + ) + } + ) + +def define_part_nine_repo(): + return RepositoryDefinition( + name='part_nine_repo', + pipeline_dict={ + 'part_nine_step_one': define_part_nine_step_one, + 'part_nine_final': define_part_nine_final, + } + ) + def test_intro_tutorial_part_nine_step_one(): result = execute_pipeline( define_part_nine_step_one(), - config.Environment(solids={ - 'injest_a': config.Solid(2), - 'injest_b': config.Solid(3), - }) + config.Environment( + solids={ + 'injest_a': config.Solid(2), + 'injest_b': config.Solid(3), + }, + ) ) assert result.success @@ -86,3 +224,74 @@ def test_intro_tutorial_part_nine_step_one(): assert result.result_for_solid('injest_b').transformed_value() == 3 assert result.result_for_solid('add_ints').transformed_value() == 5 assert result.result_for_solid('mult_ints').transformed_value() == 6 + + + +def test_intro_tutorial_part_nine_final_local_success(): + result = execute_pipeline( + define_part_nine_final(), + config.Environment( + solids={ + 'injest_a': config.Solid(2), + 'injest_b': config.Solid(3), + }, + context=config.Context(name='local') + ) + ) + + assert result.success + assert result.result_for_solid('injest_a').transformed_value() == 2 + assert result.result_for_solid('injest_b').transformed_value() == 3 + assert result.result_for_solid('add_ints').transformed_value() == 5 + assert result.result_for_solid('mult_ints').transformed_value() == 6 + + assert result.context.resources.store.values == { + 'a': 2, + 'b': 3, + 'add': 5, + 'mult': 6, + } + + +def test_intro_tutorial_part_nine_final_cloud_success(): + result = execute_pipeline( + define_part_nine_final(), + config.Environment( + solids={ + 'injest_a': config.Solid(2), + 'injest_b': config.Solid(3), + }, + context=config.Context( + name='cloud', + config={ + 'credentials': { + 'user': 'some_user', + 'pass': 'some_pass', + }, + }, + ), + ), + ) + + assert result.success + +def test_intro_tutorial_part_nine_final_error(): + with pytest.raises(DagsterTypeError, match='Field username not found'): + execute_pipeline( + define_part_nine_final(), + config.Environment( + solids={ + 'injest_a': config.Solid(2), + 'injest_b': config.Solid(3), + }, + context=config.Context( + name='cloud', + config={ + 'credentials': { + 'username': 'some_user', + 'pass': 'some_pass', + }, + }, + ), + ), + ) diff --git a/python_modules/dagster/docs/apidocs/types.rst b/python_modules/dagster/docs/apidocs/types.rst index 571975f94a0aa..ac613b78d1017 100644 --- a/python_modules/dagster/docs/apidocs/types.rst +++ b/python_modules/dagster/docs/apidocs/types.rst @@ -35,6 +35,3 @@ Utilities .. autoclass:: Field :members: - -.. autoclass:: IncomingValueResult - :members: diff --git a/python_modules/dagster/docs/intro_tutorial/part_eight.rst b/python_modules/dagster/docs/intro_tutorial/part_eight.rst index 435178e0cb18a..2e7aa4c2a2209 100644 --- a/python_modules/dagster/docs/intro_tutorial/part_eight.rst +++ b/python_modules/dagster/docs/intro_tutorial/part_eight.rst @@ -1,5 +1,5 @@ Basic Typing ------- +------------ Dagster includes an optional type system that can be applied to both runtime values and configuration. We can use these types to both provide runtime type guarantees diff --git a/python_modules/dagster/docs/intro_tutorial/part_nine.rst b/python_modules/dagster/docs/intro_tutorial/part_nine.rst index 3828db9929c8b..d48866312cd33 100644 --- a/python_modules/dagster/docs/intro_tutorial/part_nine.rst +++ b/python_modules/dagster/docs/intro_tutorial/part_nine.rst @@ -16,9 +16,378 @@ interactions with scratch directories on your local filesystem, and so on. Let's imagine a scenario where we want to record some custom state in a key-value store for our execution runs. A production time, this key-value store is a live store (e.g. DynamoDB in amazon) but we do not want to interact -this store for unit-testing. Contexts will be our tool to accomplish this. Goal. +this store for unit-testing. Contexts will be our tool to accomplish this goal. We're going to have a simple pipeline that does some rudimentary arithmetic, but that wants record the result computations in that key value store. -# TODO: do this +Let's first set up a simple pipeline. We injest two numbers (each in their own trivial solid) +and then two downstream solids add and multiple those numbers, respectively. + +.. code-block:: python + + @solid( + config_def=ConfigDefinition(types.Int), + outputs=[OutputDefinition(types.Int)], + ) + def injest_a(context, conf): + return conf + + + @solid( + config_def=ConfigDefinition(types.Int), + outputs=[OutputDefinition(types.Int)], + ) + def injest_b(context, conf): + return conf + + @solid( + inputs=[InputDefinition('num_one', types.Int), + InputDefinition('num_two', types.Int)], + outputs=[OutputDefinition(types.Int)], + ) + def add_ints(_context, _conf, num_one, num_two): + return num_one + num_two + + + @solid( + inputs=[InputDefinition('num_one', types.Int), + InputDefinition('num_two', types.Int)], + outputs=[OutputDefinition(types.Int)], + ) + def mult_ints(_context, _conf, num_one, num_two): + return num_one * num_two + + + def define_part_nine_step_one(): + return PipelineDefinition( + name='part_nine', + solids=[injest_a, injest_b, add_ints, mult_ints], + dependencies={ + 'add_ints': { + 'num_one': DependencyDefinition('injest_a'), + 'num_two': DependencyDefinition('injest_b'), + }, + 'mult_ints': { + 'num_one': DependencyDefinition('injest_a'), + 'num_two': DependencyDefinition('injest_b'), + }, + }, + ) + +Now we configure execution using a env.yml file: + +.. code-block:: yaml + + context: + config: + log_level: DEBUG + + solids: + injest_a: + config: 2 + injest_b: + config: 3 + +And you should see some log spew indicating execution. + +Now imagine we want to log some of the values passing through these solids +into some sort of key value store in cloud storage: + +Let's say we have a module called ``cloud`` that allows for interaction +with this key value store. You have to create an instance of a ``PublicCloudConn`` +class and then pass that to a function ``set_value_in_cloud_store`` to interact +with the service. + +.. code-block:: python + + from cloud import (PublicCloudConn, set_value_in_cloud_store) + + # imagine implementations such as the following + # class PublicCloudConn: + # def __init__(self, creds): + # self.creds = creds + + + # def set_value_in_cloud_store(_conn, _key, _value): + # # imagine this doing something + # pass + + conn = PublicCloudConn({'user': some_user', 'pass' : 'some_pwd'}) + set_value_in_cloud_store(conn, 'some_key', 'some_value') + + +Naively let's add this to one of our transforms: + +.. code-block:: python + + @solid( + config_def=ConfigDefinition(types.Int), + outputs=[OutputDefinition(types.Int)], + ) + def injest_a(_context, conf): + conn = PublicCloudConn('some_user', 'some_pwd') + set_value_in_cloud_store(conn, 'a', conf) + return conf + +As coded above this is a bad idea on any number of dimensions. One the username/password +combo is hard coded. We could pass it in as a configuration of the solid. However that +is only in scope for that particular solid. So now the configuration would be passed into +each and every solid that needs it. This sucks. The connection would have to be created within +every solid. Either you would have to implement your own connection pooling or take the hit +of a new connection per solid. This also sucks. + +More subtley, what was previously a nice, isolated, testable piece of software is now hard-coded +to interact with some externalized resource and requires an internet connection, access to +a cloud service, and that could intermittently fail, and that would be slow relative to pure +in-memory compute. This code is no longer testable in any sort of reliable way. + +This is where the concept of the context shines. What we want to do is attach an object to the +context object -- which a single instance of is flowed through the entire execution -- that +provides an interface to that cloud store that caches that connection and also provides a +swappable implementation of that store for test isolation. We want code that ends up looking like +this: + +.. code-block:: python + + @solid( + config_def=ConfigDefinition(types.Int), + outputs=[OutputDefinition(types.Int)], + ) + def injest_a(context, conf): + # The store should be an interface to the cloud store + # We will explain the ``resources`` property later. + context.resources.store.record_value(context, 'a', conf) + return conf + +The user will be able have complete control the creation of the ``store`` object attached to +the ``resources`` object, which allows a pipeline designer to insert seams of testability. + +This ends up accomplishing our goal of being able to test this pipeline in multiple environments +with *zero changes to the core business logic.* The only thing that will vary between environments +is configuration and the context generated because of that configuration. + +We need this store object, and two implementations of it. One that talks to the public cloud +service, and one that is an in-memory implementation of this. + +.. code-block:: python + + class PublicCloudStore: + def __init__(self, credentials): + self.conn = PublicCloudConn(credentials) + + def record_value(self, context, key, value): + context.info('Setting key={key} value={value} in cloud' + .format( + key=key, + value=value, + ) + ) + set_value_in_cloud_store(self.conn, key, value) + + + class InMemoryStore: + def __init__(self): + self.values = {} + + def record_value(self, context, key, value): + context.info('Setting key={key} value={value} in memory'. + format( + key=key, + value=value, + ) + ) + self.values[key] = value + + +Now we need to create one of these stores and put them into the context. The pipeline author must +create a :py:class:`PipelineContextDefinition`. + +It two primrary attributes: + +1) A configuration definition that allows the pipeline author to define what configuration +is needed to create the ExecutionContext. +2) A function that returns an instance of an ExecutionContext. This context is flowed through +the entire execution. + +First let's create the context suitable for local testing: + +..code-block:: python + + PartNineResources = namedtuple('PartNineResources', 'store') + + PipelineContextDefinition( + context_fn=lambda _pipeline_def, _conf: + ExecutionContext.console_logging( + log_level=DEBUG, + resources=PartNineResources(InMemoryStore()) + ) + ) + +This context requires *no* configuration so it is not specified. We then +provide a lambda which creates an ExecutionContext. You'll notice that we pass +in a log_level and a "resources" object. The resources object can be any +python object. What is demonstrated above is a convention. The resources +object that the user creates will be passed through the execution. + +So if we return to the implementation of the solids that includes the interaction +with the key-value store, you can see how this will invoke the in-memory store object +which is attached the resources property of the context. + +.. code-block:: python + + @solid( + config_def=ConfigDefinition(types.Int), + outputs=[OutputDefinition(types.Int)], + ) + def injest_a(context, conf): + context.resources.store.record_value(context, 'a', conf) + return conf + + @solid( + config_def=ConfigDefinition(types.Int), + outputs=[OutputDefinition(types.Int)], + ) + def injest_b(context, conf): + context.resources.store.record_value(context, 'b', conf) + return conf + + + @solid( + inputs=[InputDefinition('num_one', types.Int), + InputDefinition('num_two', types.Int)], + outputs=[OutputDefinition(types.Int)], + ) + def add_ints(context, _conf, num_one, num_two): + result = num_one + num_two + context.resources.store.record_value(context, 'add', result) + return result + + + @solid( + inputs=[InputDefinition('num_one', types.Int), + InputDefinition('num_two', types.Int)], + outputs=[OutputDefinition(types.Int)], + ) + def mult_ints(context, _conf, num_one, num_two): + result = num_one * num_two + context.resources.store.record_value(context, 'mult', result) + return result + +Now we need to declare the pipeline to use this PipelineContextDefinition. +We do so with the following: + +.. code-block:: python + + return PipelineDefinition( + name='part_nine', + solids=[injest_a, injest_b, add_ints, mult_ints], + dependencies={ + 'add_ints': { + 'num_one': DependencyDefinition('injest_a'), + 'num_two': DependencyDefinition('injest_b'), + }, + 'mult_ints': { + 'num_one': DependencyDefinition('injest_a'), + 'num_two': DependencyDefinition('injest_b'), + }, + }, + context_definitions={ + 'local': PipelineContextDefinition( + context_fn=lambda _pipeline_def, _conf: + ExecutionContext.console_logging( + log_level=DEBUG, + resources=PartNineResources(InMemoryStore()) + ), + ) + ), + } + ) + +You'll notice that we have "named" the context local. Now when we invoke that context, +we config it with that name. + +.. code-block:: yaml + + context: + name: local + + solids: + injest_a: + config: 2 + injest_b: + config: 3 + +Now run the pipeline and you should see logging indicating the execution is occuring. + +Now let us add a different context definition that substitutes in the production +version of that store. + +.. code-block:: python + + PipelineDefinition( + name='part_nine', + solids=[injest_a, injest_b, add_ints, mult_ints], + dependencies={ + 'add_ints': { + 'num_one': DependencyDefinition('injest_a'), + 'num_two': DependencyDefinition('injest_b'), + }, + 'mult_ints': { + 'num_one': DependencyDefinition('injest_a'), + 'num_two': DependencyDefinition('injest_b'), + }, + }, + context_definitions={ + 'local': PipelineContextDefinition( + context_fn=lambda _pipeline_def, _conf: + ExecutionContext.console_logging( + log_level=DEBUG, + resources=PartNineResources(InMemoryStore()) + ) + ), + 'cloud': PipelineContextDefinition( + context_fn=lambda _pipeline_def, conf: + ExecutionContext.console_logging( + resources=PartNineResources( + PublicCloudStore(conf['credentials'], + ) + ) + ), + config_def=ConfigDefinition( + config_type=types.ConfigDictionary({ + 'credentials': Field(types.ConfigDictionary({ + 'user' : Field(types.String), + 'pass' : Field(types.String), + })), + }), + ), + ) + } + ) + +Notice the *second* context definition. It + +1) Accepts configuration, the specifies that in a typed fashion. +2) Creates a different version of that store, to which it passes configuration. + +Now when you invoke this pipeline with the following yaml file: + +.. code-block:: yaml + + context: + name: cloud + config: + credentials: + user: some_user + pass: some_password + + solids: + injest_a: + config: 2 + injest_b: + config: 3 + +It will create the production version of that store. Note that you have +not change the implementation of any solid to do this. Only the configuration +changes. From 7f8fba1f4c3cd279fc5be226d710dff6936cb465 Mon Sep 17 00:00:00 2001 From: Mikhail Novikov Date: Thu, 13 Sep 2018 11:59:43 +0300 Subject: [PATCH 086/103] Better error instructions --- python_modules/dagit/dagit/app.py | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/python_modules/dagit/dagit/app.py b/python_modules/dagit/dagit/app.py index 36fe9799f4715..be26a3e869145 100644 --- a/python_modules/dagit/dagit/app.py +++ b/python_modules/dagit/dagit/app.py @@ -54,7 +54,15 @@ def static_view(path, file): def index_view(_path): - return send_file(os.path.join(os.path.dirname(__file__), './webapp/build/index.html')) + try: + return send_file(os.path.join(os.path.dirname(__file__), './webapp/build/index.html')) + except FileNotFoundError: + text = """

Can't find webapp files. Probably webapp isn't built. If you are using dagit, then probably it's a corrupted installation or a bug. However, if you are developing dagit locally, you problem can be fixed as follows:

+ +
cd ./python_modules/dagit/dagit/webapp
+yarn
+yarn build
""" + return text, 500 def create_app(repository_container): From 90ac4fc59f951a49a2b25ff1ef86e8292ca7088a Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Wed, 12 Sep 2018 20:05:16 -0700 Subject: [PATCH 087/103] context, conf ==> info Instead of the transform function taking a separate context and conf object we are going to take a single object, info, thas was context and config properties. This reduces the number of parameters you have to deal with and will also allow additional concepts in the future. --- .../dagster_ge_tests/test_pandas_ge.py | 1 - python_modules/dagster/dagster/__init__.py | 5 +- .../dagster/dagster/core/compute_nodes.py | 8 ++- .../core/core_tests/test_custom_context.py | 22 +++---- .../core/core_tests/test_decorators.py | 48 +++++---------- .../core/core_tests/test_definition_errors.py | 4 +- .../core/core_tests/test_naming_collisions.py | 16 ++--- .../core/core_tests/test_pipeline_errors.py | 13 ++-- .../core_tests/test_pipeline_execution.py | 6 +- .../core/core_tests/test_solid_with_config.py | 4 +- .../test_transform_only_pipeline.py | 10 ++-- .../dagster/dagster/core/decorators.py | 40 ++++++------- .../dagster/dagster/core/definitions.py | 60 ++----------------- .../dagster/dagster/core/execution.py | 5 +- .../dagster/dagster/core/execution_context.py | 17 ++++++ .../dagster/dagster/core/test_utils.py | 52 ++++++++++++++++ .../dagster/dagster/pandas/__init__.py | 12 ++-- .../test_pandas_hello_world_library_slide.py | 4 +- ...est_pandas_hello_world_no_library_slide.py | 18 +++--- .../pandas/pandas_tests/test_pandas_solids.py | 8 ++- .../subquery_builder_experimental.py | 12 ++-- .../dagster/dagster/sqlalchemy/templated.py | 8 +-- python_modules/dagster/dagster/version.py | 2 +- .../test_intro_tutorial_part_eight.py | 12 ++-- .../test_intro_tutorial_part_five.py | 8 +-- .../test_intro_tutorial_part_four.py | 6 +- .../test_intro_tutorial_part_nine.py | 41 +++++++------ .../test_intro_tutorial_part_seven.py | 4 +- .../dagster/docs/apidocs/execution.rst | 3 + .../docs/intro_tutorial/part_eight.rst | 16 ++--- .../dagster/docs/intro_tutorial/part_five.rst | 13 ++-- .../dagster/docs/intro_tutorial/part_four.rst | 13 ++-- .../docs/intro_tutorial/part_seven.rst | 4 +- python_modules/dagster/setup.py | 2 +- 34 files changed, 261 insertions(+), 236 deletions(-) diff --git a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py index 023ba3cb22809..4786a99a943c2 100644 --- a/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py +++ b/python_modules/dagster-ge/dagster_ge_tests/test_pandas_ge.py @@ -7,7 +7,6 @@ InputDefinition, OutputDefinition, PipelineDefinition, - config, execute_pipeline, lambda_solid, ) diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index 387036fe38425..658ef5706af42 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -5,7 +5,10 @@ execute_pipeline_iterator, ) -from dagster.core.execution_context import ExecutionContext +from dagster.core.execution_context import ( + ExecutionContext, + TransformExecutionInfo, +) from dagster.core.definitions import ( ConfigDefinition, diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index cb83c82ea0808..7086a0a7a6bbb 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -31,7 +31,10 @@ SolidOutputHandle, ) -from .execution_context import ExecutionContext +from .execution_context import ( + ExecutionContext, + TransformExecutionInfo, +) from .errors import ( DagsterExpectationFailedError, @@ -183,8 +186,9 @@ class ComputeNodeTag(Enum): EXPECTATION_INPUT = 'expectation_input' + def _yield_transform_results(context, compute_node, conf, inputs): - gen = compute_node.solid.transform_fn(context, conf, inputs) + gen = compute_node.solid.transform_fn(TransformExecutionInfo(context, conf), inputs) if isinstance(gen, Result): raise DagsterInvariantViolationError( diff --git a/python_modules/dagster/dagster/core/core_tests/test_custom_context.py b/python_modules/dagster/dagster/core/core_tests/test_custom_context.py index 929a2c749f819..295935c25d1c4 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_custom_context.py +++ b/python_modules/dagster/dagster/core/core_tests/test_custom_context.py @@ -24,9 +24,9 @@ def test_default_context(): called = {} @solid(inputs=[], outputs=[OutputDefinition()]) - def default_context_transform(context, _): + def default_context_transform(info): called['yes'] = True - for logger in context._logger.loggers: + for logger in info.context._logger.loggers: assert logger.level == ERROR pipeline = PipelineDefinition(solids=[default_context_transform]) @@ -37,8 +37,8 @@ def default_context_transform(context, _): def test_default_context_with_log_level(): @solid(inputs=[], outputs=[OutputDefinition()]) - def default_context_transform(context, _): - for logger in context._logger.loggers: + def default_context_transform(info): + for logger in info.context._logger.loggers: assert logger.level == INFO pipeline = PipelineDefinition(solids=[default_context_transform]) @@ -57,8 +57,8 @@ def default_context_transform(context, _): def test_default_value(): def _get_config_test_solid(config_key, config_value): @solid(inputs=[], outputs=[OutputDefinition()]) - def config_test(context, _): - assert context.resources == {config_key: config_value} + def config_test(info): + assert info.context.resources == {config_key: config_value} return config_test @@ -89,8 +89,8 @@ def config_test(context, _): def test_custom_contexts(): @solid(inputs=[], outputs=[OutputDefinition()]) - def custom_context_transform(context, _): - assert context.resources == {'field_one': 'value_two'} + def custom_context_transform(info): + assert info.context.resources == {'field_one': 'value_two'} pipeline = PipelineDefinition( solids=[custom_context_transform], @@ -133,9 +133,9 @@ def test_yield_context(): events = [] @solid(inputs=[], outputs=[OutputDefinition()]) - def custom_context_transform(context, _): - assert context.resources == {'field_one': 'value_two'} - assert context._context_dict['foo'] == 'bar' # pylint: disable=W0212 + def custom_context_transform(info): + assert info.context.resources == {'field_one': 'value_two'} + assert info.context._context_dict['foo'] == 'bar' # pylint: disable=W0212 events.append('during') def _yield_context(_pipeline, config_value): diff --git a/python_modules/dagster/dagster/core/core_tests/test_decorators.py b/python_modules/dagster/dagster/core/core_tests/test_decorators.py index b9514b6220d11..8903b4bdf2259 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_decorators.py +++ b/python_modules/dagster/dagster/core/core_tests/test_decorators.py @@ -84,7 +84,7 @@ def hello_world(): def test_solid(): @solid(outputs=[OutputDefinition()]) - def hello_world(_context, _conf): + def hello_world(_info): return {'foo': 'bar'} result = execute_single_solid( @@ -116,7 +116,7 @@ def hello_world(): def test_solid_yield(): @solid(outputs=[OutputDefinition()]) - def hello_world(_context, _): + def hello_world(_info): yield Result(value={'foo': 'bar'}) result = execute_single_solid( @@ -132,7 +132,7 @@ def hello_world(_context, _): def test_solid_result_return(): @solid(outputs=[OutputDefinition()]) - def hello_world(_context, _conf): + def hello_world(_info): return Result(value={'foo': 'bar'}) result = execute_single_solid( @@ -148,7 +148,7 @@ def hello_world(_context, _conf): def test_solid_multiple_outputs(): @solid(outputs=[OutputDefinition(name="left"), OutputDefinition(name="right")]) - def hello_world(_context, _conf): + def hello_world(_info): return MultipleResults( Result(value={'foo': 'left'}, output_name='left'), Result(value={'foo': 'right'}, output_name='right') @@ -169,7 +169,7 @@ def hello_world(_context, _conf): def test_dict_multiple_outputs(): @solid(outputs=[OutputDefinition(name="left"), OutputDefinition(name="right")]) - def hello_world(_context, _conf): + def hello_world(_info): return MultipleResults.from_dict({ 'left': { 'foo': 'left' @@ -210,7 +210,7 @@ def hello_world(): def test_solid_with_name(): @solid(name="foobar", outputs=[OutputDefinition()]) - def hello_world(_context, _conf): + def hello_world(_info): return {'foo': 'bar'} result = execute_single_solid( @@ -256,13 +256,13 @@ def test_solid_definition_errors(): with pytest.raises(DagsterInvalidDefinitionError, match='positional vararg'): @solid(inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()]) - def vargs(context, conf, foo, *args): + def vargs(info, foo, *args): pass with pytest.raises(DagsterInvalidDefinitionError): @solid(inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()]) - def wrong_name(context, conf, bar): + def wrong_name(info, bar): pass with pytest.raises(DagsterInvalidDefinitionError): @@ -274,37 +274,19 @@ def wrong_name(context, conf, bar): ], outputs=[OutputDefinition()] ) - def wrong_name_2(context, conf, foo): + def wrong_name_2(info, foo): pass with pytest.raises(DagsterInvalidDefinitionError): @solid(inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()]) - def no_context(conf, foo): + def no_info(foo): pass with pytest.raises(DagsterInvalidDefinitionError): @solid(inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()]) - def no_context_with_both(conf, foo): - pass - - with pytest.raises(DagsterInvalidDefinitionError): - - @solid(inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()]) - def no_conf(context, foo): - pass - - with pytest.raises(DagsterInvalidDefinitionError): - - @solid(inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()]) - def yes_context(_context, foo): - pass - - with pytest.raises(DagsterInvalidDefinitionError): - - @solid(inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()]) - def extras(_context, _conf, foo, bar): + def extras(_info, foo, bar): pass @solid( @@ -312,7 +294,7 @@ def extras(_context, _conf, foo, bar): InputDefinition(name="bar")], outputs=[OutputDefinition()] ) - def valid_kwargs(context, conf, **kwargs): + def valid_kwargs(info, **kwargs): pass @solid( @@ -320,7 +302,7 @@ def valid_kwargs(context, conf, **kwargs): InputDefinition(name="bar")], outputs=[OutputDefinition()] ) - def valid(context, conf, foo, bar): + def valid(info, foo, bar): pass @@ -352,8 +334,8 @@ def test_any_config_definition(): conf_value = 234 @solid(config_def=ConfigDefinition()) - def hello_world(context, conf): - assert conf == conf_value + def hello_world(info): + assert info.config == conf_value called['yup'] = True result = execute_single_solid( diff --git a/python_modules/dagster/dagster/core/core_tests/test_definition_errors.py b/python_modules/dagster/dagster/core/core_tests/test_definition_errors.py index d7e39268655d1..a59359dca9f73 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_definition_errors.py +++ b/python_modules/dagster/dagster/core/core_tests/test_definition_errors.py @@ -16,13 +16,13 @@ def solid_a_b_list(): name='A', inputs=[], outputs=[OutputDefinition()], - transform_fn=lambda context, conf, inputs: None, + transform_fn=lambda _info, _inputs: None, ), SolidDefinition( name='B', inputs=[InputDefinition('b_input')], outputs=[], - transform_fn=lambda context, conf, inputs: None, + transform_fn=lambda _info, _inputs: None, ) ] diff --git a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py index 4975c0369be01..b2c803d9349fd 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py +++ b/python_modules/dagster/dagster/core/core_tests/test_naming_collisions.py @@ -15,13 +15,15 @@ types, ) +from dagster.core.test_utils import single_output_transform + def define_pass_value_solid(name, description=None): check.str_param(name, 'name') check.opt_str_param(description, 'description') - def _value_t_fn(_context, conf, _inputs): - yield Result(conf['value']) + def _value_t_fn(info, _inputs): + yield Result(info.config['value']) return SolidDefinition( name=name, @@ -36,7 +38,7 @@ def _value_t_fn(_context, conf, _inputs): def test_execute_solid_with_input_same_name(): - a_thing_solid = SolidDefinition.single_output_transform( + a_thing_solid = single_output_transform( 'a_thing', inputs=[InputDefinition(name='a_thing')], transform_fn=lambda context, inputs: inputs['a_thing'] + inputs['a_thing'], @@ -63,14 +65,14 @@ def test_execute_solid_with_input_same_name(): def test_execute_two_solids_with_same_input_name(): input_def = InputDefinition(name='a_thing') - solid_one = SolidDefinition.single_output_transform( + solid_one = single_output_transform( 'solid_one', inputs=[input_def], transform_fn=lambda context, inputs: inputs['a_thing'] + inputs['a_thing'], output=dagster.OutputDefinition(), ) - solid_two = SolidDefinition.single_output_transform( + solid_two = single_output_transform( 'solid_two', inputs=[input_def], transform_fn=lambda context, inputs: inputs['a_thing'] + inputs['a_thing'], @@ -117,14 +119,14 @@ def test_execute_two_solids_with_same_input_name(): def test_execute_dep_solid_different_input_name(): pass_to_first = define_pass_value_solid('pass_to_first') - first_solid = SolidDefinition.single_output_transform( + first_solid = single_output_transform( 'first_solid', inputs=[InputDefinition(name='a_thing')], transform_fn=lambda context, inputs: inputs['a_thing'] + inputs['a_thing'], output=dagster.OutputDefinition(), ) - second_solid = SolidDefinition.single_output_transform( + second_solid = single_output_transform( 'second_solid', inputs=[ InputDefinition(name='an_input'), diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py index 029081566ed72..1f3539fd979e8 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py @@ -15,7 +15,10 @@ execute_pipeline, ) -from dagster.core.test_utils import execute_single_solid +from dagster.core.test_utils import ( + execute_single_solid, + single_output_transform, +) from dagster.core.errors import DagsterUserCodeExecutionError @@ -41,7 +44,7 @@ def root_transform(_context, _args): passed_rows.append({name: 'transform_called'}) return passed_rows - return SolidDefinition.single_output_transform( + return single_output_transform( name=name, inputs=[], transform_fn=root_transform, @@ -53,7 +56,7 @@ def create_root_transform_failure_solid(name): def failed_transform(**_kwargs): raise Exception('Transform failed') - return SolidDefinition.single_output_transform( + return single_output_transform( name=name, inputs=[], transform_fn=failed_transform, @@ -82,7 +85,7 @@ def transform_fn(_context, inputs): check.failed('user error') return [inputs['A'], inputs['B'], {'C': 'transform_called'}] - solid_c = SolidDefinition.single_output_transform( + solid_c = single_output_transform( name='C', inputs=[InputDefinition(name='A'), InputDefinition(name='B')], transform_fn=transform_fn, @@ -145,7 +148,7 @@ def _tn(*_args, **_kwargs): def test_single_transform_returning_result(): - solid_inst = SolidDefinition.single_output_transform( + solid_inst = single_output_transform( 'test_return_result', inputs=[], transform_fn=lambda *_args, **_kwargs: Result(None), diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py index 444682140a3f0..b9bb4f00fc425 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py @@ -19,6 +19,8 @@ execute_pipeline_iterator, ExecutionContext, SolidExecutionResult, PipelineExecutionResult ) +from dagster.core.test_utils import single_output_transform + from dagster.core.utility_solids import define_stub_solid # protected members @@ -56,7 +58,7 @@ def transform(_context, inputs): def create_solid_with_deps(name, *solid_deps): inputs = [InputDefinition(solid_dep.name) for solid_dep in solid_deps] - return SolidDefinition.single_output_transform( + return single_output_transform( name=name, inputs=inputs, transform_fn=make_transform(name), @@ -68,7 +70,7 @@ def create_root_solid(name): input_name = name + '_input' inp = InputDefinition(input_name) - return SolidDefinition.single_output_transform( + return single_output_transform( name=name, inputs=[inp], transform_fn=make_transform(name), diff --git a/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py b/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py index b760c12567873..0dee7001a9454 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py +++ b/python_modules/dagster/dagster/core/core_tests/test_solid_with_config.py @@ -17,8 +17,8 @@ def test_basic_solid_with_config(): did_get = {} - def _t_fn(_context, conf, _inputs): - did_get['yep'] = conf + def _t_fn(info, _inputs): + did_get['yep'] = info.config solid = SolidDefinition( name='solid_with_context', diff --git a/python_modules/dagster/dagster/core/core_tests/test_transform_only_pipeline.py b/python_modules/dagster/dagster/core/core_tests/test_transform_only_pipeline.py index fccd0a57b2d83..82df35fe9bb98 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_transform_only_pipeline.py +++ b/python_modules/dagster/dagster/core/core_tests/test_transform_only_pipeline.py @@ -8,6 +8,8 @@ execute_pipeline, ) +from dagster.core.test_utils import single_output_transform + def _set_key_value(ddict, key, value): ddict[key] = value @@ -17,14 +19,14 @@ def _set_key_value(ddict, key, value): def test_execute_solid_with_dep_only_inputs_no_api(): did_run_dict = {} - step_one_solid = SolidDefinition.single_output_transform( + step_one_solid = single_output_transform( name='step_one_solid', inputs=[], transform_fn=lambda context, args: _set_key_value(did_run_dict, 'step_one', True), output=OutputDefinition(), ) - step_two_solid = SolidDefinition.single_output_transform( + step_two_solid = single_output_transform( name='step_two_solid', inputs=[InputDefinition('step_one_solid')], transform_fn=lambda context, args: _set_key_value(did_run_dict, 'step_two', True), @@ -56,14 +58,14 @@ def test_execute_solid_with_dep_only_inputs_no_api(): def test_execute_solid_with_dep_only_inputs_with_api(): did_run_dict = {} - step_one_solid = SolidDefinition.single_output_transform( + step_one_solid = single_output_transform( name='step_one_solid', inputs=[], transform_fn=lambda context, args: _set_key_value(did_run_dict, 'step_one', True), output=OutputDefinition(), ) - step_two_solid = SolidDefinition.single_output_transform( + step_two_solid = single_output_transform( name='step_two_solid', transform_fn=lambda context, args: _set_key_value(did_run_dict, 'step_two', True), inputs=[InputDefinition(step_one_solid.name)], diff --git a/python_modules/dagster/dagster/core/decorators.py b/python_modules/dagster/dagster/core/decorators.py index 8e7d57cba5f50..69ffb18a06e07 100644 --- a/python_modules/dagster/dagster/core/decorators.py +++ b/python_modules/dagster/dagster/core/decorators.py @@ -122,7 +122,7 @@ def __call__(self, fn): if not self.name: self.name = fn.__name__ - _validate_transform_fn(self.name, fn, self.input_defs, ['context', 'conf']) + _validate_transform_fn(self.name, fn, self.input_defs, ['info']) transform_fn = _create_solid_transform_wrapper(fn, self.input_defs, self.outputs) return SolidDefinition( name=self.name, @@ -215,30 +215,30 @@ def solid( .. code-block:: python @solid - def hello_world(context, conf): + def hello_world(info): print('hello') @solid() - def hello_world(context, conf): + def hello_world(info): print('hello') @solid(outputs=[OutputDefinition()]) - def hello_world(context, conf): + def hello_world(info): return {'foo': 'bar'} @solid(outputs=[OutputDefinition()]) - def hello_world(context, conf): + def hello_world(info): return Result(value={'foo': 'bar'}) @solid(outputs=[OutputDefinition()]) - def hello_world(context, conf): + def hello_world(info): yield Result(value={'foo': 'bar'}) @solid(outputs=[ OutputDefinition(name="left"), OutputDefinition(name="right"), ]) - def hello_world(context, conf): + def hello_world(info): return MultipleResults.from_dict({ 'left': {'foo': 'left'}, 'right': {'foo': 'right'}, @@ -248,15 +248,15 @@ def hello_world(context, conf): inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()] ) - def hello_world(context, conf, foo): + def hello_world(info, foo): return foo @solid( inputs=[InputDefinition(name="foo")], outputs=[OutputDefinition()], ) - def hello_world(context, conf, foo): - context.info('log something') + def hello_world(info, foo): + info.context.info('log something') return foo @solid( @@ -264,9 +264,9 @@ def hello_world(context, conf, foo): outputs=[OutputDefinition()], config_def=ConfigDefinition(types.ConfigDictionary({'str_value' : Field(types.String)})), ) - def hello_world(context, conf, foo): - # conf is a dictionary with 'str_value' key - return foo + def hello_world(info, foo): + # info.config is a dictionary with 'str_value' key + return foo + info.config['str_value'] ''' if callable(name): @@ -293,7 +293,7 @@ def _create_lambda_solid_transform_wrapper(fn, input_defs, output_def): input_names = [input_def.name for input_def in input_defs] @wraps(fn) - def transform(_context, _conf, inputs): + def transform(_info, inputs): kwargs = {} for input_name in input_names: kwargs[input_name] = inputs[input_name] @@ -312,12 +312,12 @@ def _create_solid_transform_wrapper(fn, input_defs, output_defs): input_names = [input_def.name for input_def in input_defs] @wraps(fn) - def transform(context, conf, inputs): + def transform(info, inputs): kwargs = {} for input_name in input_names: kwargs[input_name] = inputs[input_name] - result = fn(context, conf, **kwargs) + result = fn(info, **kwargs) if inspect.isgenerator(result): for item in result: @@ -369,23 +369,23 @@ def _validate_transform_fn(solid_name, transform_fn, inputs, expected_positional except FunctionValidationError as e: if e.error_type == FunctionValidationError.TYPES['vararg']: raise DagsterInvalidDefinitionError( - "solid '{solid_name}' transform function has positional vararg parameter '{e.param}'. Transform functions should only have keyword arguments that match input names and optionally a first positional parameter named 'context'.". + "solid '{solid_name}' transform function has positional vararg parameter '{e.param}'. Transform functions should only have keyword arguments that match input names and a first positional parameter named 'info'.". format(solid_name=solid_name, e=e) ) elif e.error_type == FunctionValidationError.TYPES['missing_name']: raise DagsterInvalidDefinitionError( - "solid '{solid_name}' transform function has parameter '{e.param}' that is not one of the solid inputs. Transform functions should only have keyword arguments that match input names and optionally a first positional parameter named 'context'.". + "solid '{solid_name}' transform function has parameter '{e.param}' that is not one of the solid inputs. Transform functions should only have keyword arguments that match input names and a first positional parameter named 'info'.". format(solid_name=solid_name, e=e) ) elif e.error_type == FunctionValidationError.TYPES['missing_positional']: raise DagsterInvalidDefinitionError( - "solid '{solid_name}' transform function do not have required positional parameter '{e.param}'. Transform functions should only have keyword arguments that match input names and optionally a first positional parameter named 'context'.". + "solid '{solid_name}' transform function do not have required positional parameter '{e.param}'. Transform functions should only have keyword arguments that match input names and a first positional parameter named 'info'.". format(solid_name=solid_name, e=e) ) elif e.error_type == FunctionValidationError.TYPES['extra']: undeclared_inputs_printed = ", '".join(e.missing_names) raise DagsterInvalidDefinitionError( - "solid '{solid_name}' transform function do not have parameter(s) '{undeclared_inputs_printed}', which are in solid's inputs. Transform functions should only have keyword arguments that match input names and optionally a first positional parameter named 'context'.". + "solid '{solid_name}' transform function do not have parameter(s) '{undeclared_inputs_printed}', which are in solid's inputs. Transform functions should only have keyword arguments that match input names and a first positional parameter named 'info'.". format(solid_name=solid_name, undeclared_inputs_printed=undeclared_inputs_printed) ) else: diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index d1ce850fad389..46cd2b5be5aaf 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -15,10 +15,7 @@ from dagster.config import DEFAULT_CONTEXT_NAME -from .errors import ( - DagsterInvalidDefinitionError, - DagsterInvariantViolationError, -) +from .errors import DagsterInvalidDefinitionError from .execution_context import ExecutionContext @@ -808,8 +805,8 @@ class SolidDefinition(object): .. code-block:: python - def _read_csv(context, inputs, config_dict): - yield Result(pandas.read_csv(config_dict['path'])) + def _read_csv(info, inputs): + yield Result(pandas.read_csv(info.config['path'])) SolidDefinition( name='read_csv', @@ -825,9 +822,8 @@ def _read_csv(context, inputs, config_dict): transform_fn (callable): Callable with the signature ( - context: ExecutionContext, + info: TransformExecutionInfo, inputs: Dict[str, Any], - conf: Any ) : Iterable outputs (List[OutputDefinition]): Outputs of the solid. config_def (ConfigDefinition): How the solid configured. @@ -861,54 +857,6 @@ def __init__(self, name, inputs, transform_fn, outputs, config_def=None, descrip self._input_dict = _build_named_dict(inputs) self._output_dict = _build_named_dict(outputs) - @staticmethod - def single_output_transform(name, inputs, transform_fn, output, description=None): - '''It is commmon to want a Solid that has only inputs, a single output (with the default - name), and no config. So this is a helper function to do that. This transform function - must return the naked return value (as opposed to a Result object). - - Args: - name (str): Name of the solid. - inputs (List[InputDefinition]): Inputs of solid. - transform_fn (callable): - Callable with the signature - (context: ExecutionContext, inputs: Dict[str, Any]) : Any - output (OutputDefinition): Output of the solid. - description (str): Descripion of the solid. - - Returns: - SolidDefinition: - - Example: - - .. code-block:: python - - SolidDefinition.single_output_transform( - 'add_one', - inputs=InputDefinition('num', types.Int), - output=OutputDefinition(types.Int), - transform_fn=lambda context, inputs: inputs['num'] + 1 - ) - - ''' - - def _new_transform_fn(context, _conf, inputs): - value = transform_fn(context, inputs) - if isinstance(value, Result): - raise DagsterInvariantViolationError( - '''Single output transform Solid {name} returned a Result. Just return - value directly without wrapping it in Result''' - ) - yield Result(output_name=DEFAULT_OUTPUT, value=value) - - return SolidDefinition( - name=name, - inputs=inputs, - transform_fn=_new_transform_fn, - outputs=[output], - description=description, - ) - def input_handle(self, name): check.str_param(name, 'name') return self._input_handles[name] diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index 624c2182acc8e..3fd184d7e69c9 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -46,7 +46,10 @@ execute_compute_nodes, ) -from .execution_context import ExecutionContext +from .execution_context import ( + ExecutionContext, + TransformExecutionInfo, +) class PipelineExecutionResult(object): diff --git a/python_modules/dagster/dagster/core/execution_context.py b/python_modules/dagster/dagster/core/execution_context.py index a277cb4571685..d28c4cda4b5a5 100644 --- a/python_modules/dagster/dagster/core/execution_context.py +++ b/python_modules/dagster/dagster/core/execution_context.py @@ -175,3 +175,20 @@ def value(self, key, value): yield self._context_dict.pop(key) + + +class TransformExecutionInfo(namedtuple('_TransformExecutionInfo', 'context config')): + '''An instance of TransformExecutionInfo is passed every solid transform function. + + Attributes: + + context (ExecutionContext): Context instance for this pipeline invocation + config (Any): Config object for current solid + ''' + + def __new__(cls, context, config): + return super(TransformExecutionInfo, cls).__new__( + cls, + check.inst_param(context, 'context', ExecutionContext), + config, + ) \ No newline at end of file diff --git a/python_modules/dagster/dagster/core/test_utils.py b/python_modules/dagster/dagster/core/test_utils.py index f4941132214b2..e4a1afbdc8840 100644 --- a/python_modules/dagster/dagster/core/test_utils.py +++ b/python_modules/dagster/dagster/core/test_utils.py @@ -1,13 +1,17 @@ from dagster import ( + DagsterInvariantViolationError, ExecutionContext, PipelineDefinition, PipelineContextDefinition, + Result, SolidDefinition, check, config, execute_pipeline, ) +from dagster.core.definitions import DEFAULT_OUTPUT + def execute_single_solid(context, solid, environment=None, throw_on_error=True): check.inst_param(context, 'context', ExecutionContext) @@ -36,3 +40,51 @@ def execute_single_solid(context, solid, environment=None, throw_on_error=True): ) return pipeline_result + + +def single_output_transform(name, inputs, transform_fn, output, description=None): + '''It is commmon to want a Solid that has only inputs, a single output (with the default + name), and no config. So this is a helper function to do that. This transform function + must return the naked return value (as opposed to a Result object). + + Args: + name (str): Name of the solid. + inputs (List[InputDefinition]): Inputs of solid. + transform_fn (callable): + Callable with the signature + (context: ExecutionContext, inputs: Dict[str, Any]) : Any + output (OutputDefinition): Output of the solid. + description (str): Descripion of the solid. + + Returns: + SolidDefinition: + + Example: + + .. code-block:: python + + single_output_transform( + 'add_one', + inputs=InputDefinition('num', types.Int), + output=OutputDefinition(types.Int), + transform_fn=lambda context, inputs: inputs['num'] + 1 + ) + + ''' + + def _new_transform_fn(info, inputs): + value = transform_fn(info.context, inputs) + if isinstance(value, Result): + raise DagsterInvariantViolationError( + '''Single output transform Solid {name} returned a Result. Just return + value directly without wrapping it in Result''' + ) + yield Result(output_name=DEFAULT_OUTPUT, value=value) + + return SolidDefinition( + name=name, + inputs=inputs, + transform_fn=_new_transform_fn, + outputs=[output], + description=description, + ) \ No newline at end of file diff --git a/python_modules/dagster/dagster/pandas/__init__.py b/python_modules/dagster/dagster/pandas/__init__.py index b0fbe327d8fd1..b85cf459540ef 100644 --- a/python_modules/dagster/dagster/pandas/__init__.py +++ b/python_modules/dagster/dagster/pandas/__init__.py @@ -31,8 +31,8 @@ def _create_dataframe_type(): def load_csv_solid(name): check.str_param(name, 'name') - def _t_fn(_context, conf, _inputs): - yield Result(pd.read_csv(conf['path'])) + def _t_fn(info, _inputs): + yield Result(pd.read_csv(info.config['path'])) return SolidDefinition( name=name, @@ -46,8 +46,8 @@ def _t_fn(_context, conf, _inputs): def to_csv_solid(name): - def _t_fn(_context, conf, inputs): - inputs['df'].to_csv(conf['path'], index=False) + def _t_fn(info, inputs): + inputs['df'].to_csv(info.config['path'], index=False) return SolidDefinition( name=name, @@ -61,8 +61,8 @@ def _t_fn(_context, conf, inputs): def to_parquet_solid(name): - def _t_fn(_context, conf, inputs): - inputs['df'].to_parquet(conf['path']) + def _t_fn(info, inputs): + inputs['df'].to_parquet(info.config['path']) return SolidDefinition( name=name, diff --git a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_library_slide.py b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_library_slide.py index 3374ce51a55b5..1987e6a44eb9b 100644 --- a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_library_slide.py +++ b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_library_slide.py @@ -11,6 +11,8 @@ lambda_solid, ) +from dagster.core.test_utils import single_output_transform + from dagster.utils import script_relative_path from dagster.utils.test import get_temp_file_name @@ -112,7 +114,7 @@ def transform_fn(_context, inputs): return num_csv # supports CSV and PARQUET by default - hello_world = SolidDefinition.single_output_transform( + hello_world = single_output_transform( name='hello_world', inputs=[table_input], transform_fn=transform_fn, diff --git a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_no_library_slide.py b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_no_library_slide.py index e656f3df9601a..105efc8d4d5c5 100644 --- a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_no_library_slide.py +++ b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_hello_world_no_library_slide.py @@ -14,12 +14,14 @@ types, ) +from dagster.core.test_utils import single_output_transform + from dagster.utils import script_relative_path def define_read_csv_solid(name): - def _t_fn(_context, conf, _inputs): - yield Result(pd.read_csv(conf['path'])) + def _t_fn(info, _inputs): + yield Result(pd.read_csv(info.config['path'])) return SolidDefinition( name=name, @@ -33,8 +35,8 @@ def _t_fn(_context, conf, _inputs): def define_to_csv_solid(name): - def _t_fn(_context, conf, inputs): - inputs['df'].to_csv(conf['path'], index=False) + def _t_fn(info, inputs): + inputs['df'].to_csv(info.config['path'], index=False) return SolidDefinition( name=name, @@ -55,7 +57,7 @@ def hello_world_transform_fn(_context, inputs): read_csv_solid = define_read_csv_solid('read_csv_solid') - hello_world = SolidDefinition.single_output_transform( + hello_world = single_output_transform( name='hello_world', inputs=[InputDefinition('num_df')], transform_fn=hello_world_transform_fn, @@ -99,7 +101,7 @@ def transform_fn(_context, inputs): num_df['sum'] = num_df['num1'] + num_df['num2'] return num_df - hello_world = SolidDefinition.single_output_transform( + hello_world = single_output_transform( name='hello_world', inputs=[InputDefinition('num_df')], transform_fn=transform_fn, @@ -147,7 +149,7 @@ def solid_one_transform(_context, inputs): num_df['sum'] = num_df['num1'] + num_df['num2'] return num_df - solid_one = SolidDefinition.single_output_transform( + solid_one = single_output_transform( name='solid_one', inputs=[InputDefinition(name='num_df')], transform_fn=solid_one_transform, @@ -159,7 +161,7 @@ def solid_two_transform(_context, inputs): sum_df['sum_sq'] = sum_df['sum'] * sum_df['sum'] return sum_df - solid_two = SolidDefinition.single_output_transform( + solid_two = single_output_transform( name='solid_two', inputs=[InputDefinition(name='sum_df')], transform_fn=solid_two_transform, diff --git a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py index f1e2314bbfdc0..a495a31a8c2ae 100644 --- a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py +++ b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_solids.py @@ -21,6 +21,8 @@ import dagster.pandas as dagster_pd +from dagster.core.test_utils import single_output_transform + from dagster.utils import script_relative_path from dagster.utils.test import ( @@ -30,7 +32,7 @@ def _dataframe_solid(name, inputs, transform_fn): - return SolidDefinition.single_output_transform( + return single_output_transform( name=name, inputs=inputs, transform_fn=transform_fn, @@ -75,7 +77,7 @@ def transform(_context, inputs): num_csv['sum'] = num_csv['num1'] + num_csv['num2'] return num_csv - single_solid = SolidDefinition.single_output_transform( + single_solid = single_output_transform( name='sum_table', inputs=[csv_input], transform_fn=transform, @@ -113,7 +115,7 @@ def transform(context, inputs): num_csv['sum'] = num_csv['num1'] + num_csv['num2'] return num_csv - solid_def = SolidDefinition.single_output_transform( + solid_def = single_output_transform( name='sum_table', inputs=[csv_input], transform_fn=transform, diff --git a/python_modules/dagster/dagster/sqlalchemy/subquery_builder_experimental.py b/python_modules/dagster/dagster/sqlalchemy/subquery_builder_experimental.py index 144e6abe34f1e..c9523b226a07a 100644 --- a/python_modules/dagster/dagster/sqlalchemy/subquery_builder_experimental.py +++ b/python_modules/dagster/dagster/sqlalchemy/subquery_builder_experimental.py @@ -10,6 +10,8 @@ types, ) +from dagster.core.test_utils import single_output_transform + from dagster.sqlalchemy import execute_sql_text_on_context @@ -48,14 +50,14 @@ def from_target(self): def define_create_table_solid(name): - def _materialization_fn(context, conf, inputs): + def _materialization_fn(info, inputs): sql_expr = inputs['expr'] check.inst(sql_expr, DagsterSqlExpression) - output_table_name = check.str_elem(conf, 'table_name') + output_table_name = check.str_elem(info.config, 'table_name') total_sql = '''CREATE TABLE {output_table_name} AS {query_text}'''.format( output_table_name=output_table_name, query_text=sql_expr.query_text ) - context.resources.sa.engine.connect().execute(total_sql) + info.context.resources.sa.engine.connect().execute(total_sql) return SolidDefinition( name=name, @@ -92,7 +94,7 @@ def create_sql_solid(name, inputs, sql_text): check.list_param(inputs, 'inputs', of_type=InputDefinition) check.str_param(sql_text, 'sql_text') - return SolidDefinition.single_output_transform( + return single_output_transform( name, inputs=inputs, transform_fn=create_sql_transform(sql_text), @@ -117,7 +119,7 @@ def create_sql_statement_solid(name, sql_text, inputs=None): if inputs is None: inputs = [] - return SolidDefinition.single_output_transform( + return single_output_transform( name=name, transform_fn=_create_sql_alchemy_transform_fn(sql_text), inputs=inputs, diff --git a/python_modules/dagster/dagster/sqlalchemy/templated.py b/python_modules/dagster/dagster/sqlalchemy/templated.py index 5635823aa45c4..fabf19da76ede 100644 --- a/python_modules/dagster/dagster/sqlalchemy/templated.py +++ b/python_modules/dagster/dagster/sqlalchemy/templated.py @@ -42,9 +42,9 @@ def _render_template_string(template_text, config_dict): def _create_templated_sql_transform_with_output(sql): - def do_transform(context, conf, _inputs): - rendered_sql = _render_template_string(sql, conf) - execute_sql_text_on_context(context, rendered_sql) - yield Result(conf) + def do_transform(info, _inputs): + rendered_sql = _render_template_string(sql, info.config) + execute_sql_text_on_context(info.context, rendered_sql) + yield Result(info.config) return do_transform diff --git a/python_modules/dagster/dagster/version.py b/python_modules/dagster/dagster/version.py index 312d10d3004c4..fe9151c1a24c2 100644 --- a/python_modules/dagster/dagster/version.py +++ b/python_modules/dagster/dagster/version.py @@ -1 +1 @@ -__version__ = '0.2.0.dev10' +__version__ = '0.2.0.dev11' diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/test_intro_tutorial_part_eight.py b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/test_intro_tutorial_part_eight.py index 04a0d5d22ace1..3ad9aeb9ccb90 100644 --- a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/test_intro_tutorial_part_eight.py +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eight/test_intro_tutorial_part_eight.py @@ -8,8 +8,8 @@ @solid(config_def=ConfigDefinition(types.ConfigDictionary({'word': Field(types.String)}))) -def double_the_word_with_typed_config(_context, conf): - return conf['word'] * 2 +def double_the_word_with_typed_config(info): + return info.config['word'] * 2 @solid( @@ -18,8 +18,8 @@ def double_the_word_with_typed_config(_context, conf): })), outputs=[OutputDefinition(types.String)], ) -def typed_double_word(_context, conf): - return conf['word'] * 2 +def typed_double_word(info): + return info.config['word'] * 2 @solid( @@ -28,8 +28,8 @@ def typed_double_word(_context, conf): })), outputs=[OutputDefinition(types.Int)], ) -def typed_double_word_mismatch(_context, conf): - return conf['word'] * 2 +def typed_double_word_mismatch(info): + return info.config['word'] * 2 @lambda_solid(inputs=[InputDefinition('word')]) diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_five.py b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_five.py index 1084d04184b17..405d6874ca573 100644 --- a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_five.py +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_five.py @@ -3,13 +3,13 @@ @solid -def solid_one(context, _conf): - context.info('Something you should know about occurred.') +def solid_one(info): + info.context.info('Something you should know about occurred.') @solid -def solid_two(context, _conf): - context.error('An error occurred.') +def solid_two(info): + info.context.error('An error occurred.') def test_tutorial_part_five_sample_one(): diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_four.py b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_four.py index 4eb39e951b3b2..34d62e6558308 100644 --- a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_four.py +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_four.py @@ -3,9 +3,9 @@ @solid -def hello_world(_context, conf): - print(conf) - return conf +def hello_world(info): + print(info.config) + return info.config def test_tutorial_part_four(): diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/test_intro_tutorial_part_nine.py b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/test_intro_tutorial_part_nine.py index 1bc48819b52f5..7b702795895e6 100644 --- a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/test_intro_tutorial_part_nine.py +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/test_intro_tutorial_part_nine.py @@ -43,33 +43,33 @@ def define_contextless_solids(): config_def=ConfigDefinition(types.Int), outputs=[OutputDefinition(types.Int)], ) - def injest_a(_context, conf): - return conf + def injest_a(info): + return info.config @solid( config_def=ConfigDefinition(types.Int), outputs=[OutputDefinition(types.Int)], ) - def injest_b(_context, conf): - return conf + def injest_b(info): + return info.config - @solid( + @lambda_solid( inputs=[InputDefinition('num_one', types.Int), InputDefinition('num_two', types.Int)], - outputs=[OutputDefinition(types.Int)], + output=OutputDefinition(types.Int), ) - def add_ints(_context, _conf, num_one, num_two): + def add_ints(num_one, num_two): return num_one + num_two - @solid( + @lambda_solid( inputs=[InputDefinition('num_one', types.Int), InputDefinition('num_two', types.Int)], - outputs=[OutputDefinition(types.Int)], + output=OutputDefinition(types.Int), ) - def mult_ints(_context, _conf, num_one, num_two): + def mult_ints(num_one, num_two): return num_one * num_two return [injest_a, injest_b, add_ints, mult_ints] @@ -79,18 +79,17 @@ def define_contextful_solids(): config_def=ConfigDefinition(types.Int), outputs=[OutputDefinition(types.Int)], ) - def injest_a(context, conf): - context.resources.store.record_value(context, 'a', conf) - return conf - + def injest_a(info): + info.context.resources.store.record_value(info.context, 'a', info.config) + return info.config @solid( config_def=ConfigDefinition(types.Int), outputs=[OutputDefinition(types.Int)], ) - def injest_b(context, conf): - context.resources.store.record_value(context, 'b', conf) - return conf + def injest_b(info): + info.context.resources.store.record_value(info.context, 'b', info.config) + return info.config @solid( @@ -98,9 +97,9 @@ def injest_b(context, conf): InputDefinition('num_two', types.Int)], outputs=[OutputDefinition(types.Int)], ) - def add_ints(context, _conf, num_one, num_two): + def add_ints(info, num_one, num_two): result = num_one + num_two - context.resources.store.record_value(context, 'add', result) + info.context.resources.store.record_value(info.context, 'add', result) return result @@ -109,9 +108,9 @@ def add_ints(context, _conf, num_one, num_two): InputDefinition('num_two', types.Int)], outputs=[OutputDefinition(types.Int)], ) - def mult_ints(context, _conf, num_one, num_two): + def mult_ints(info, num_one, num_two): result = num_one * num_two - context.resources.store.record_value(context, 'mult', result) + info.context.resources.store.record_value(info.context, 'mult', result) return result return [injest_a, injest_b, add_ints, mult_ints] diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_seven/test_intro_tutorial_part_seven.py b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_seven/test_intro_tutorial_part_seven.py index fa428d629c36c..e6009c86cf5d1 100644 --- a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_seven/test_intro_tutorial_part_seven.py +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_seven/test_intro_tutorial_part_seven.py @@ -5,8 +5,8 @@ @solid -def double_the_word(_context, conf): - return conf['word'] * 2 +def double_the_word(info): + return info.config['word'] * 2 @lambda_solid(inputs=[InputDefinition('word')]) diff --git a/python_modules/dagster/docs/apidocs/execution.rst b/python_modules/dagster/docs/apidocs/execution.rst index 3c6f9e6330c44..d1d01a794649e 100644 --- a/python_modules/dagster/docs/apidocs/execution.rst +++ b/python_modules/dagster/docs/apidocs/execution.rst @@ -17,3 +17,6 @@ Executing pipelines and solids. .. autoclass:: SolidExecutionResult :members: + +.. autoclass:: TransformExecutionInfo + :members: \ No newline at end of file diff --git a/python_modules/dagster/docs/intro_tutorial/part_eight.rst b/python_modules/dagster/docs/intro_tutorial/part_eight.rst index 2e7aa4c2a2209..dfe62a2324beb 100644 --- a/python_modules/dagster/docs/intro_tutorial/part_eight.rst +++ b/python_modules/dagster/docs/intro_tutorial/part_eight.rst @@ -18,8 +18,8 @@ Before we had this: from dagster import * @solid - def double_the_word(_context, conf): - return conf['word'] * 2 + def double_the_word(info): + return info.config['word'] * 2 We are going to make the configuration of this strongly typed prevent errors and improve documentation. @@ -29,8 +29,8 @@ documentation. from dagster import * @solid(config_def=ConfigDefinition(types.ConfigDictionary({'word': Field(types.String)}))) - def double_the_word_with_typed_config(_context, conf): - return conf['word'] * 2 + def double_the_word_with_typed_config(info): + return info.config['word'] * 2 The previous env.yml file works as before: @@ -98,8 +98,8 @@ specified, it defaults to the Any type. })), outputs=[OutputDefinition(types.String)], ) - def typed_double_word(_context, conf): - return conf['word'] * 2 + def typed_double_word(info): + return info.config['word'] * 2 You'll see here that now the output is annotated with a type. This both ensures that the runtime value conforms requirements specified by the type (in this case @@ -117,8 +117,8 @@ So imagine we made a coding error (mistyped the output) such as: })), outputs=[OutputDefinition(types.Int)], ) - def typed_double_word(_context, conf): - return conf['word'] * 2 + def typed_double_word(info): + return info.config['word'] * 2 When we run it, it errors: diff --git a/python_modules/dagster/docs/intro_tutorial/part_five.rst b/python_modules/dagster/docs/intro_tutorial/part_five.rst index e7ea97f113fc2..2b24f0507a68a 100644 --- a/python_modules/dagster/docs/intro_tutorial/part_five.rst +++ b/python_modules/dagster/docs/intro_tutorial/part_five.rst @@ -1,9 +1,8 @@ Execution Context ----------------- -In addition to its configuration and its inputs, a solid also receives a ``context`` -argument. In this tutorial part, we'll go over this ``context`` variable which was -ignored in the previous step of the tutorial. +In this tutorial part, we'll go over the other property of the ``info`` parameter, +``context``. The context is an object of type :py:class:`ExecutionContext`. For every execution of a particular pipeline, one instance of this context is created, no matter how @@ -18,13 +17,13 @@ Let's use the context for one of its core capabilities: logging. from dagster import * @solid - def solid_one(context, _conf): - context.info('Something you should know about occurred.') + def solid_one(info): + info.context.info('Something you should know about occurred.') @solid - def solid_two(context, _conf): - context.error('An error occurred.') + def solid_two(info): + info.context.error('An error occurred.') if __name__ == '__main__': diff --git a/python_modules/dagster/docs/intro_tutorial/part_four.rst b/python_modules/dagster/docs/intro_tutorial/part_four.rst index 51ea403d00d52..7827cc7e1e87f 100644 --- a/python_modules/dagster/docs/intro_tutorial/part_four.rst +++ b/python_modules/dagster/docs/intro_tutorial/part_four.rst @@ -21,8 +21,8 @@ minimal API. ``solid`` is more complicated, but has more capabilities: from dagster import * @solid - def hello_world(_context, conf): - print(conf) + def hello_world(info): + print(info.config) return conf @@ -36,11 +36,10 @@ minimal API. ``solid`` is more complicated, but has more capabilities: You'll notice a new API, ``solid``. We will be exploring this API in much more detail as this tutorial proceeds. For now, the only is that the function annotated by solid now -takes two parameters where before it took zero (if it accepted no inputs). Ignore -the first parameter, ``_context``, for now. We will cover that in a later section -of the tutorial. The other parameter is ``conf``, which is short for config (which -we cannot use because of the name collision with the config namespace of dagster). -It represents the configuration for that solid. +takes one parameter where before it took zero (if it accepted no inputs). This +takes an info parameter, which is of type :py:class:`TransformExecutionInfo`. It +has a property config, which is the configuration that is passed into this +particular solid. We must provide that configuration. And thusly turn your attention to the second argument of execute_pipeline, which must be an instance of ``config.Environment``. An ``Environment`` diff --git a/python_modules/dagster/docs/intro_tutorial/part_seven.rst b/python_modules/dagster/docs/intro_tutorial/part_seven.rst index d41c10e067aa0..b2931d2d84ceb 100644 --- a/python_modules/dagster/docs/intro_tutorial/part_seven.rst +++ b/python_modules/dagster/docs/intro_tutorial/part_seven.rst @@ -12,8 +12,8 @@ and a yaml file so that the CLI tool can know about the repository. from dagster import * @solid - def double_the_word(_context, conf): - return conf['word'] * 2 + def double_the_word(info): + return info.config['word'] * 2 @lambda_solid(inputs=[InputDefinition('word')]) def count_letters(word): diff --git a/python_modules/dagster/setup.py b/python_modules/dagster/setup.py index 5b220646cc237..c27738edb16f0 100644 --- a/python_modules/dagster/setup.py +++ b/python_modules/dagster/setup.py @@ -18,7 +18,7 @@ def long_description(): version = {} with open("dagster/version.py") as fp: - exec(fp.read(), version) + exec(fp.read(), version) # pylint: disable=W0122 setup( name='dagster', From 976aee77d806ec6a6b130a60d3dc94d1da832a6e Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Thu, 13 Sep 2018 07:21:56 -0700 Subject: [PATCH 088/103] minor cleanup --- python_modules/dagster/dagster/core/test_utils.py | 2 +- python_modules/dagster/dagster/sqlalchemy/common.py | 8 ++++++-- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/python_modules/dagster/dagster/core/test_utils.py b/python_modules/dagster/dagster/core/test_utils.py index e4a1afbdc8840..911345c494c4b 100644 --- a/python_modules/dagster/dagster/core/test_utils.py +++ b/python_modules/dagster/dagster/core/test_utils.py @@ -79,7 +79,7 @@ def _new_transform_fn(info, inputs): '''Single output transform Solid {name} returned a Result. Just return value directly without wrapping it in Result''' ) - yield Result(output_name=DEFAULT_OUTPUT, value=value) + yield Result(value=value) return SolidDefinition( name=name, diff --git a/python_modules/dagster/dagster/sqlalchemy/common.py b/python_modules/dagster/dagster/sqlalchemy/common.py index e59eb558c53f0..6b95032584127 100644 --- a/python_modules/dagster/dagster/sqlalchemy/common.py +++ b/python_modules/dagster/dagster/sqlalchemy/common.py @@ -1,7 +1,11 @@ import sqlalchemy -from dagster import check -from dagster.core.execution import ExecutionContext +from dagster import ( + ExecutionContext, + Result, + SolidDefinition, + check, +) class SqlAlchemyResource(object): From 336d55d3e5d383e697fdddc1073060a10ba7e725 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Thu, 13 Sep 2018 09:21:47 -0700 Subject: [PATCH 089/103] Allow execution of empty pipelines --- .../dagster/core/core_tests/test_pipeline_execution.py | 6 ++++++ python_modules/dagster/dagster/core/execution.py | 8 +++++++- 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py index b9bb4f00fc425..162ca92a3ff8d 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py @@ -313,3 +313,9 @@ def _do_test(do_execute_pipeline_iter): transform_called('C'), transform_called('D'), ] + + +def test_empty_pipeline_execution(): + result = execute_pipeline(PipelineDefinition(solids=[])) + + assert result.success diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index 3fd184d7e69c9..e745efe52d440 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -286,7 +286,13 @@ def _execute_graph_iterator(context, execution_graph, environment): cn_nodes = list(cn_graph.topological_nodes()) - check.invariant(len(cn_nodes) > 0, 'No compute nodes!') + if not cn_nodes: + context.debug( + 'Pipeline {pipeline} has no nodes and no execution will happen'.format( + pipeline=execution_graph.pipeline.display_name + ) + ) + return context.debug( 'About to execute the compute node graph in the following order {order}'.format( From 0b584b02f99a36a8aa58582bed30432d3da14960 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Thu, 13 Sep 2018 09:33:45 -0700 Subject: [PATCH 090/103] Fix docs in part eight and nine --- .../docs/intro_tutorial/part_eight.rst | 18 +++++---- .../dagster/docs/intro_tutorial/part_nine.rst | 40 +++++++++---------- 2 files changed, 31 insertions(+), 27 deletions(-) diff --git a/python_modules/dagster/docs/intro_tutorial/part_eight.rst b/python_modules/dagster/docs/intro_tutorial/part_eight.rst index dfe62a2324beb..e707bf97cf8a7 100644 --- a/python_modules/dagster/docs/intro_tutorial/part_eight.rst +++ b/python_modules/dagster/docs/intro_tutorial/part_eight.rst @@ -28,7 +28,11 @@ documentation. from dagster import * - @solid(config_def=ConfigDefinition(types.ConfigDictionary({'word': Field(types.String)}))) + @solid( + config_def=ConfigDefinition( + types.ConfigDictionary({'word': Field(types.String)}) + ) + ) def double_the_word_with_typed_config(info): return info.config['word'] * 2 @@ -93,9 +97,9 @@ specified, it defaults to the Any type. .. code-block:: python @solid( - config_def=ConfigDefinition(types.ConfigDictionary({ - 'word': Field(types.String) - })), + config_def=ConfigDefinition( + types.ConfigDictionary({'word': Field(types.String)}) + ), outputs=[OutputDefinition(types.String)], ) def typed_double_word(info): @@ -112,9 +116,9 @@ So imagine we made a coding error (mistyped the output) such as: .. code-block:: python @solid( - config_def=ConfigDefinition(types.ConfigDictionary({ - 'word': Field(types.String) - })), + config_def=ConfigDefinition( + types.ConfigDictionary({'word': Field(types.String)}) + ), outputs=[OutputDefinition(types.Int)], ) def typed_double_word(info): diff --git a/python_modules/dagster/docs/intro_tutorial/part_nine.rst b/python_modules/dagster/docs/intro_tutorial/part_nine.rst index d48866312cd33..445517be956e6 100644 --- a/python_modules/dagster/docs/intro_tutorial/part_nine.rst +++ b/python_modules/dagster/docs/intro_tutorial/part_nine.rst @@ -30,23 +30,23 @@ and then two downstream solids add and multiple those numbers, respectively. config_def=ConfigDefinition(types.Int), outputs=[OutputDefinition(types.Int)], ) - def injest_a(context, conf): - return conf + def injest_a(info): + return info.config @solid( config_def=ConfigDefinition(types.Int), outputs=[OutputDefinition(types.Int)], ) - def injest_b(context, conf): - return conf + def injest_b(info): + return info.config @solid( inputs=[InputDefinition('num_one', types.Int), InputDefinition('num_two', types.Int)], outputs=[OutputDefinition(types.Int)], ) - def add_ints(_context, _conf, num_one, num_two): + def add_ints(_info, num_one, num_two): return num_one + num_two @@ -55,7 +55,7 @@ and then two downstream solids add and multiple those numbers, respectively. InputDefinition('num_two', types.Int)], outputs=[OutputDefinition(types.Int)], ) - def mult_ints(_context, _conf, num_one, num_two): + def mult_ints(_info, num_one, num_two): return num_one * num_two @@ -125,10 +125,10 @@ Naively let's add this to one of our transforms: config_def=ConfigDefinition(types.Int), outputs=[OutputDefinition(types.Int)], ) - def injest_a(_context, conf): + def injest_a(info): conn = PublicCloudConn('some_user', 'some_pwd') - set_value_in_cloud_store(conn, 'a', conf) - return conf + set_value_in_cloud_store(conn, 'a', info.config) + return info.config As coded above this is a bad idea on any number of dimensions. One the username/password combo is hard coded. We could pass it in as a configuration of the solid. However that @@ -154,11 +154,11 @@ this: config_def=ConfigDefinition(types.Int), outputs=[OutputDefinition(types.Int)], ) - def injest_a(context, conf): + def injest_a(info): # The store should be an interface to the cloud store # We will explain the ``resources`` property later. - context.resources.store.record_value(context, 'a', conf) - return conf + info.context.resources.store.record_value(info.context, 'a', conf) + return info.config The user will be able have complete control the creation of the ``store`` object attached to the ``resources`` object, which allows a pipeline designer to insert seams of testability. @@ -240,16 +240,16 @@ which is attached the resources property of the context. config_def=ConfigDefinition(types.Int), outputs=[OutputDefinition(types.Int)], ) - def injest_a(context, conf): - context.resources.store.record_value(context, 'a', conf) + def injest_a(info): + info.context.resources.store.record_value(info.context, 'a', info.config) return conf @solid( config_def=ConfigDefinition(types.Int), outputs=[OutputDefinition(types.Int)], ) - def injest_b(context, conf): - context.resources.store.record_value(context, 'b', conf) + def injest_b(info): + info.context.resources.store.record_value(info.context, 'b', info.config) return conf @@ -258,9 +258,9 @@ which is attached the resources property of the context. InputDefinition('num_two', types.Int)], outputs=[OutputDefinition(types.Int)], ) - def add_ints(context, _conf, num_one, num_two): + def add_ints(info, num_one, num_two): result = num_one + num_two - context.resources.store.record_value(context, 'add', result) + info.context.resources.store.record_value(info.context, 'add', result) return result @@ -269,9 +269,9 @@ which is attached the resources property of the context. InputDefinition('num_two', types.Int)], outputs=[OutputDefinition(types.Int)], ) - def mult_ints(context, _conf, num_one, num_two): + def mult_ints(info, num_one, num_two): result = num_one * num_two - context.resources.store.record_value(context, 'mult', result) + info.context.resources.store.record_value(info.context, 'mult', result) return result Now we need to declare the pipeline to use this PipelineContextDefinition. From 0faaa2d0ebdf14c9103b0d75c08aec280bceb218 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Thu, 13 Sep 2018 23:38:15 -0700 Subject: [PATCH 091/103] fix code-block --- python_modules/dagster/docs/intro_tutorial/part_nine.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python_modules/dagster/docs/intro_tutorial/part_nine.rst b/python_modules/dagster/docs/intro_tutorial/part_nine.rst index 445517be956e6..00f80d3d521a6 100644 --- a/python_modules/dagster/docs/intro_tutorial/part_nine.rst +++ b/python_modules/dagster/docs/intro_tutorial/part_nine.rst @@ -212,7 +212,7 @@ the entire execution. First let's create the context suitable for local testing: -..code-block:: python +.. code-block:: python PartNineResources = namedtuple('PartNineResources', 'store') From 098f9f0a656432712dcc7d39beda3545e4fcefc3 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Thu, 13 Sep 2018 09:39:32 -0700 Subject: [PATCH 092/103] Use info object contexts and expectations --- .../dagster-ge/dagster_ge/__init__.py | 4 ++-- python_modules/dagster/dagster/__init__.py | 9 ++++++- .../dagster/dagster/core/compute_nodes.py | 7 ++---- .../core/core_tests/test_compute_nodes.py | 6 +---- .../core/core_tests/test_custom_context.py | 12 +++++----- .../core/core_tests/test_pipeline_errors.py | 6 +---- .../dagster/dagster/core/definitions.py | 24 +++++++++++++++---- .../dagster/dagster/core/execution.py | 23 ++++++++++++------ .../dagster/dagster/core/execution_context.py | 2 +- .../dagster/dagster/core/test_utils.py | 2 -- .../test_sql_project_pipeline.py | 6 ++--- .../sqlalchemy_tests/test_basic_solid.py | 3 +-- .../test_isolated_sql_tests.py | 3 +-- .../test_isolated_templated_sql_tests.py | 3 +-- python_modules/dagster/dagster/version.py | 2 +- .../test_intro_tutorial_part_nine.py | 8 +++---- .../dagster/docs/intro_tutorial/part_nine.rst | 10 ++++---- 17 files changed, 72 insertions(+), 58 deletions(-) diff --git a/python_modules/dagster-ge/dagster_ge/__init__.py b/python_modules/dagster-ge/dagster_ge/__init__.py index b44f5519509c9..2dcf583124594 100644 --- a/python_modules/dagster-ge/dagster_ge/__init__.py +++ b/python_modules/dagster-ge/dagster_ge/__init__.py @@ -12,7 +12,7 @@ # dagster_ge.json_config_expectation('num_expectations', 'num_expectations.json') # ] def json_config_expectation(name, file_path): - def _file_passes(_context, _info, df): + def _file_passes(_info, df): with open(file_path) as ff: expt_config = json.load(ff) # This is necessary because ge ends up coercing a type change @@ -38,7 +38,7 @@ def _file_passes(_context, _info, df): # dagster_ge.ge_expectation(name, lambda ge_df: ge_df.expect_column_to_exist(col_name)), # ] def ge_expectation(name, ge_callback): - def _do_expectation(_context, _info, df): + def _do_expectation(_info, df): ge_df = ge.from_pandas(df) ge_result = ge_callback(ge_df) check.invariant('success' in ge_result) diff --git a/python_modules/dagster/dagster/__init__.py b/python_modules/dagster/dagster/__init__.py index 658ef5706af42..4af5024bccf72 100644 --- a/python_modules/dagster/dagster/__init__.py +++ b/python_modules/dagster/dagster/__init__.py @@ -12,16 +12,18 @@ from dagster.core.definitions import ( ConfigDefinition, + ContextCreationExecutionInfo, DependencyDefinition, ExpectationDefinition, + ExpectationExecutionInfo, ExpectationResult, Field, InputDefinition, OutputDefinition, PipelineContextDefinition, PipelineDefinition, - Result, RepositoryDefinition, + Result, SolidDefinition, ) @@ -60,6 +62,11 @@ 'SolidDefinition', 'Result', + # Infos + 'ContextCreationExecutionInfo', + 'ExpectationExecutionInfo', + 'TransformExecutionInfo', + # Decorators 'lambda_solid', 'solid', diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index 7086a0a7a6bbb..50e7b453ce514 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -24,6 +24,7 @@ from .definitions import ( ExecutionGraph, ExpectationDefinition, + ExpectationExecutionInfo, InputDefinition, OutputDefinition, Result, @@ -755,9 +756,6 @@ def _create_join_node(solid, prev_nodes, prev_output_name): ) -ExpectationExecutionInfo = namedtuple('ExpectationExecutionInfo', 'solid expectation_def') - - def _create_join_lambda(_context, _compute_node, inputs): yield Result(output_name=JOIN_OUTPUT, value=list(inputs.values())[0]) @@ -769,8 +767,7 @@ def _create_expectation_lambda(solid, expectation_def, output_name): def _do_expectation(context, _compute_node, inputs): expt_result = expectation_def.expectation_fn( - context, - ExpectationExecutionInfo(solid, expectation_def), + ExpectationExecutionInfo(context, solid, expectation_def), inputs[EXPECTATION_INPUT], ) if expt_result.success: diff --git a/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py b/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py index ad1640b7c682a..1ada1668335fc 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py +++ b/python_modules/dagster/dagster/core/core_tests/test_compute_nodes.py @@ -16,11 +16,7 @@ def silencing_default_context(): - return { - 'default': PipelineContextDefinition( - context_fn=lambda _pipeline, _config: ExecutionContext(), - ) - } + return {'default': PipelineContextDefinition(context_fn=lambda *_args: ExecutionContext(), )} @lambda_solid diff --git a/python_modules/dagster/dagster/core/core_tests/test_custom_context.py b/python_modules/dagster/dagster/core/core_tests/test_custom_context.py index 295935c25d1c4..eb3e363baac70 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_custom_context.py +++ b/python_modules/dagster/dagster/core/core_tests/test_custom_context.py @@ -77,7 +77,7 @@ def config_test(info): ) } ), - context_fn=lambda _pipeline, config_value: ExecutionContext(resources=config_value), + context_fn=lambda info: ExecutionContext(resources=info.config), ), } ) @@ -102,7 +102,7 @@ def custom_context_transform(info): 'field_one': Field(dagster_type=types.String) } ), - context_fn=lambda _pipeline, config_value: ExecutionContext(resources=config_value), + context_fn=lambda info: ExecutionContext(resources=info.config), ), 'custom_two': PipelineContextDefinition( @@ -111,7 +111,7 @@ def custom_context_transform(info): 'field_one': Field(dagster_type=types.String) } ), - context_fn=lambda _pipeline, config_value: ExecutionContext(resources=config_value), + context_fn=lambda info: ExecutionContext(resources=info.config), ) }, ) @@ -138,9 +138,9 @@ def custom_context_transform(info): assert info.context._context_dict['foo'] == 'bar' # pylint: disable=W0212 events.append('during') - def _yield_context(_pipeline, config_value): + def _yield_context(info): events.append('before') - context = ExecutionContext(resources=config_value) + context = ExecutionContext(resources=info.config) with context.value('foo', 'bar'): yield context events.append('after') @@ -206,7 +206,7 @@ def never_transform(): config_def=ConfigDefinition.config_dict({ 'string_field': Field(types.String) }), - context_fn=lambda _pipeline, _config_value: _config_value + context_fn=lambda info: info.config, ) } ) diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py index 1f3539fd979e8..10f8ffc2fdddd 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py @@ -23,11 +23,7 @@ def silencing_default_context(): - return { - 'default': PipelineContextDefinition( - context_fn=lambda _pipeline, _args: ExecutionContext(), - ) - } + return {'default': PipelineContextDefinition(context_fn=lambda *_args: ExecutionContext())} def silencing_pipeline(solids, dependencies=None): diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index 46cd2b5be5aaf..33a2be6de6d7f 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -97,7 +97,7 @@ def passthrough_context_definition(context): ''' check.inst_param(context, 'context', ExecutionContext) - context_definition = PipelineContextDefinition(context_fn=lambda _pipeline, _args: context) + context_definition = PipelineContextDefinition(context_fn=lambda *_args: context) return {DEFAULT_CONTEXT_NAME: context_definition} def __init__(self, context_fn, config_def=None, description=None): @@ -134,8 +134,8 @@ def __init__(self, context_fn, config_def=None, description=None): def _default_pipeline_context_definitions(): - def _default_context_fn(_pipeline, config_value): - log_level = level_from_string(config_value['log_level']) + def _default_context_fn(info): + log_level = level_from_string(info.config['log_level']) context = ExecutionContext( loggers=[define_colored_console_logger('dagster', level=log_level)] ) @@ -606,7 +606,6 @@ class ExpectationDefinition(object): ExpectationDefinition( name='is_positive', expectation_fn=lambda( - _context, _info, value, ): ExpectationResult(success=value > 0), @@ -1206,3 +1205,20 @@ def get_all_pipelines(self): ''' return list(self.iterate_over_pipelines()) + + +class ContextCreationExecutionInfo( + namedtuple('_ContextCreationExecutionInfo', 'config pipeline_def') +): + def __new__(cls, config, pipeline_def): + return super(ContextCreationExecutionInfo, cls).__new__( + cls, + config, + check.inst_param(pipeline_def, 'pipeline_def', PipelineDefinition), + ) + + +ExpectationExecutionInfo = namedtuple( + 'ExpectationExecutionInfo', + 'context solid_def expectation_def', +) diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index e745efe52d440..1f5b7793cd2cd 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -27,6 +27,7 @@ from .definitions import ( DEFAULT_OUTPUT, + ContextCreationExecutionInfo, ExecutionGraph, PipelineDefinition, SolidDefinition, @@ -233,6 +234,15 @@ def _validate_environment(environment, pipeline): ) +def _create_config_value(config_type, config_input): + try: + return config_type.evaluate_value(config_input) + except DagsterEvaluateValueError as e: + raise DagsterTypeError( + 'Invalid config value: {error_msg}'.format(error_msg=','.join(e.args)) + ) + + @contextmanager def yield_context(pipeline, environment): check.inst_param(pipeline, 'pipeline', PipelineDefinition) @@ -244,14 +254,13 @@ def yield_context(pipeline, environment): context_definition = pipeline.context_definitions[context_name] config_type = context_definition.config_def.config_type - try: - evaluation_result = config_type.evaluate_value(environment.context.config) - except DagsterEvaluateValueError as e: - raise DagsterTypeError( - 'Invalid config value: {error_msg}'.format(error_msg=','.join(e.args)) + config_value = _create_config_value(config_type, environment.context.config) + context_or_generator = context_definition.context_fn( + ContextCreationExecutionInfo( + config=config_value, + pipeline_def=pipeline, ) - - context_or_generator = context_definition.context_fn(pipeline, evaluation_result) + ) return _wrap_in_yield(context_or_generator) diff --git a/python_modules/dagster/dagster/core/execution_context.py b/python_modules/dagster/dagster/core/execution_context.py index d28c4cda4b5a5..272f4fff70c83 100644 --- a/python_modules/dagster/dagster/core/execution_context.py +++ b/python_modules/dagster/dagster/core/execution_context.py @@ -191,4 +191,4 @@ def __new__(cls, context, config): cls, check.inst_param(context, 'context', ExecutionContext), config, - ) \ No newline at end of file + ) diff --git a/python_modules/dagster/dagster/core/test_utils.py b/python_modules/dagster/dagster/core/test_utils.py index 911345c494c4b..b2b842206bc2c 100644 --- a/python_modules/dagster/dagster/core/test_utils.py +++ b/python_modules/dagster/dagster/core/test_utils.py @@ -10,8 +10,6 @@ execute_pipeline, ) -from dagster.core.definitions import DEFAULT_OUTPUT - def execute_single_solid(context, solid, environment=None, throw_on_error=True): check.inst_param(context, 'context', ExecutionContext) diff --git a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py index 10a847fd7f701..c07a1d3194d22 100644 --- a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py +++ b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/sql_project_example/test_sql_project_pipeline.py @@ -34,10 +34,8 @@ def create_persisted_context(): def create_mem_sql_pipeline_context_tuple(solids, dependencies=None): - default_def = PipelineContextDefinition(context_fn=lambda _pipeline, _args: in_mem_context(), ) - persisted_def = PipelineContextDefinition( - context_fn=lambda _pipeline, _args: create_persisted_context(), - ) + default_def = PipelineContextDefinition(context_fn=lambda _info: in_mem_context(), ) + persisted_def = PipelineContextDefinition(context_fn=lambda _info: create_persisted_context(), ) return PipelineDefinition( solids=solids, dependencies=dependencies, diff --git a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_basic_solid.py b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_basic_solid.py index 54564616b6472..d2050f1163b5e 100644 --- a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_basic_solid.py +++ b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_basic_solid.py @@ -24,8 +24,7 @@ def pipeline_test_def(solids, context, dependencies): return PipelineDefinition( solids=solids, context_definitions={ - 'default': PipelineContextDefinition(context_fn=lambda _pipeline, _args: context, - ), + 'default': PipelineContextDefinition(context_fn=lambda *_args: context), }, dependencies=dependencies, ) diff --git a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_sql_tests.py b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_sql_tests.py index 56c11d5804a41..17f056adbc6aa 100644 --- a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_sql_tests.py +++ b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_sql_tests.py @@ -20,8 +20,7 @@ def pipeline_test_def(solids, context, dependencies=None): return PipelineDefinition( solids=solids, context_definitions={ - 'default': PipelineContextDefinition(context_fn=lambda _pipeline, _args: context, - ), + 'default': PipelineContextDefinition(context_fn=lambda *_args: context), }, dependencies=dependencies, ) diff --git a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_templated_sql_tests.py b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_templated_sql_tests.py index 2d649f769b7f0..6033c21d14c29 100644 --- a/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_templated_sql_tests.py +++ b/python_modules/dagster/dagster/sqlalchemy/sqlalchemy_tests/test_isolated_templated_sql_tests.py @@ -27,8 +27,7 @@ def pipeline_test_def(solids, context, dependencies=None): return PipelineDefinition( solids=solids, context_definitions={ - 'default': PipelineContextDefinition(context_fn=lambda _pipeline, _args: context, - ), + 'default': PipelineContextDefinition(context_fn=lambda info: context), }, dependencies=dependencies, ) diff --git a/python_modules/dagster/dagster/version.py b/python_modules/dagster/dagster/version.py index fe9151c1a24c2..abf15875cd011 100644 --- a/python_modules/dagster/dagster/version.py +++ b/python_modules/dagster/dagster/version.py @@ -1 +1 @@ -__version__ = '0.2.0.dev11' +__version__ = '0.2.0.dev12' diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/test_intro_tutorial_part_nine.py b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/test_intro_tutorial_part_nine.py index 7b702795895e6..6fa60b272fbc5 100644 --- a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/test_intro_tutorial_part_nine.py +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_nine/test_intro_tutorial_part_nine.py @@ -151,7 +151,7 @@ def define_part_nine_step_two(): context_definitions={ 'local': PipelineContextDefinition( - context_fn=lambda _pipeline_def, _conf: + context_fn=lambda *_args: ExecutionContext.console_logging( log_level=DEBUG, resources=PartNineResources(InMemoryStore()) @@ -178,15 +178,15 @@ def define_part_nine_final(): context_definitions={ 'local': PipelineContextDefinition( - context_fn=lambda _pipeline_def, _conf: ExecutionContext.console_logging( + context_fn=lambda *_args: ExecutionContext.console_logging( log_level=DEBUG, resources=PartNineResources(InMemoryStore()) ) ), 'cloud': PipelineContextDefinition( - context_fn=lambda _pipeline_def, conf: ExecutionContext.console_logging( - resources=PartNineResources(PublicCloudStore(conf['credentials'])) + context_fn=lambda info: ExecutionContext.console_logging( + resources=PartNineResources(PublicCloudStore(info.config['credentials'])) ), config_def=ConfigDefinition(config_type=types.ConfigDictionary({ 'credentials': Field(types.ConfigDictionary({ diff --git a/python_modules/dagster/docs/intro_tutorial/part_nine.rst b/python_modules/dagster/docs/intro_tutorial/part_nine.rst index 00f80d3d521a6..e255b05d9e729 100644 --- a/python_modules/dagster/docs/intro_tutorial/part_nine.rst +++ b/python_modules/dagster/docs/intro_tutorial/part_nine.rst @@ -217,7 +217,7 @@ First let's create the context suitable for local testing: PartNineResources = namedtuple('PartNineResources', 'store') PipelineContextDefinition( - context_fn=lambda _pipeline_def, _conf: + context_fn=lambda _info: ExecutionContext.console_logging( log_level=DEBUG, resources=PartNineResources(InMemoryStore()) @@ -294,7 +294,7 @@ We do so with the following: }, context_definitions={ 'local': PipelineContextDefinition( - context_fn=lambda _pipeline_def, _conf: + context_fn=lambda _info: ExecutionContext.console_logging( log_level=DEBUG, resources=PartNineResources(InMemoryStore()) @@ -340,17 +340,17 @@ version of that store. }, context_definitions={ 'local': PipelineContextDefinition( - context_fn=lambda _pipeline_def, _conf: + context_fn=lambda _info: ExecutionContext.console_logging( log_level=DEBUG, resources=PartNineResources(InMemoryStore()) ) ), 'cloud': PipelineContextDefinition( - context_fn=lambda _pipeline_def, conf: + context_fn=lambda info: ExecutionContext.console_logging( resources=PartNineResources( - PublicCloudStore(conf['credentials'], + PublicCloudStore(info.config['credentials'], ) ) ), From f3a0566374496e7e6272234ed15a924a80538318 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Fri, 14 Sep 2018 10:22:10 -0700 Subject: [PATCH 093/103] Fix bug in cli printing for execute command Triggered when pipeline fails. Put use case under test. --- .../dagster/dagster/cli/pipeline.py | 8 +----- .../test_pandas_hello_world.py | 27 ++++++++++++++++++- .../pandas_hello_world/pipeline.py | 2 +- python_modules/dagster/dagster/version.py | 2 +- 4 files changed, 29 insertions(+), 10 deletions(-) diff --git a/python_modules/dagster/dagster/cli/pipeline.py b/python_modules/dagster/dagster/cli/pipeline.py index 44801bb48498a..624348754d18c 100644 --- a/python_modules/dagster/dagster/cli/pipeline.py +++ b/python_modules/dagster/dagster/cli/pipeline.py @@ -227,13 +227,7 @@ def process_results_for_console(pipeline_iter): for result in pipeline_iter: if not result.success: - if result.reason == DagsterExecutionFailureReason.USER_CODE_ERROR: - raise result.user_exception - elif result.reason == DagsterExecutionFailureReason.EXPECTATION_FAILURE: - for expectation_result in result.failed_expectation_results: - result.context.error(expectation_result.message, solid=result.solid.name) - click_context = click.get_current_context() - click_context.exit(1) + result.reraise_user_error() results.append(result) return results diff --git a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/pandas_hello_world/test_pandas_hello_world.py b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/pandas_hello_world/test_pandas_hello_world.py index a190a4fee465c..5ae2bf58c818f 100644 --- a/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/pandas_hello_world/test_pandas_hello_world.py +++ b/python_modules/dagster/dagster/dagster_examples/dagster_examples_tests/pandas_hello_world/test_pandas_hello_world.py @@ -1,5 +1,7 @@ import os +import pytest + from dagster import config from dagster.core.execution import execute_pipeline from dagster.utils import script_relative_path @@ -8,7 +10,10 @@ print_pipeline, ) -from dagster.dagster_examples.pandas_hello_world.pipeline import define_success_pipeline +from dagster.dagster_examples.pandas_hello_world.pipeline import ( + define_success_pipeline, + define_failure_pipeline, +) def test_pipeline_include(): @@ -60,6 +65,26 @@ def test_cli_execute(): os.chdir(cwd) +def test_cli_execute_failure(): + + # currently paths in env files have to be relative to where the + # script has launched so we have to simulate that + with pytest.raises(Exception, match='I am a programmer and I make error'): + cwd = os.getcwd() + try: + + os.chdir(script_relative_path('../..')) + + do_execute_command( + define_failure_pipeline(), + script_relative_path('../../pandas_hello_world/env.yml'), + lambda *_args, **_kwargs: None, + ) + finally: + # restore cwd + os.chdir(cwd) + + def test_cli_print(): print_pipeline(define_success_pipeline(), full=False, print_fn=lambda *_args, **_kwargs: None) print_pipeline(define_success_pipeline(), full=True, print_fn=lambda *_args, **_kwargs: None) diff --git a/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py b/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py index a2cfc1465ed57..bb5417f2f775d 100644 --- a/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py +++ b/python_modules/dagster/dagster/dagster_examples/pandas_hello_world/pipeline.py @@ -37,7 +37,7 @@ def always_fails_solid(**_kwargs): raise Exception('I am a programmer and I make error') -def define_pipeline(): +def define_failure_pipeline(): return dagster.PipelineDefinition( name='pandas_hello_world_fails', solids=[ diff --git a/python_modules/dagster/dagster/version.py b/python_modules/dagster/dagster/version.py index abf15875cd011..8db444e674a2c 100644 --- a/python_modules/dagster/dagster/version.py +++ b/python_modules/dagster/dagster/version.py @@ -1 +1 @@ -__version__ = '0.2.0.dev12' +__version__ = '0.2.0.dev13' From 04fceafb6746a62791df3b609d7f69df49158cc1 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Fri, 14 Sep 2018 12:58:33 -0700 Subject: [PATCH 094/103] Expectations tutorial and API evolution Add a tutorial for expectations and improved it along the way --- .../dagster/dagster/core/compute_nodes.py | 65 ++++-- .../dagster/dagster/core/definitions.py | 12 +- python_modules/dagster/dagster/core/errors.py | 24 ++- .../dagster/dagster/core/execution.py | 11 +- .../test_intro_tutorial_part_ten.py | 121 +++++++++++ python_modules/dagster/docs/index.rst | 1 + .../dagster/docs/intro_tutorial/part_ten.rst | 196 ++++++++++++++++++ 7 files changed, 399 insertions(+), 31 deletions(-) create mode 100644 python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_ten/test_intro_tutorial_part_ten.py create mode 100644 python_modules/dagster/docs/intro_tutorial/part_ten.rst diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index 50e7b453ce514..7ac16f82b061f 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -38,6 +38,7 @@ ) from .errors import ( + DagsterError, DagsterExpectationFailedError, DagsterInvariantViolationError, DagsterTypeError, @@ -116,7 +117,7 @@ def __new__(cls, dagster_user_exception): dagster_user_exception=check.inst_param( dagster_user_exception, 'dagster_user_exception', - DagsterUserCodeExecutionError, + DagsterError, ), ) @@ -167,6 +168,10 @@ def _user_code_error_boundary(context, msg, **kwargs): try: yield + except DagsterError as de: + stack_trace = get_formatted_stack_trace(de) + context.error(str(de), stack_trace=stack_trace) + raise de except Exception as e: stack_trace = get_formatted_stack_trace(e) context.error(str(e), stack_trace=stack_trace) @@ -360,7 +365,9 @@ def execute(self, context, inputs): ) ) - error_str = 'TODO error string' + error_str = 'Error occured during compute node {friendly_name}'.format( + friendly_name=self.friendly_name, + ) seen_outputs = set() @@ -369,11 +376,11 @@ def execute(self, context, inputs): with _user_code_error_boundary(context, error_str): gen = self.compute_fn(context, self, inputs) - if gen is None: - check.invariant(not self.node_outputs) - return + if gen is None: + check.invariant(not self.node_outputs) + return - results = list(gen) + results = list(gen) for result in results: if not self.has_node(result.output_name): @@ -404,6 +411,15 @@ def execute(self, context, inputs): ), ) return + except DagsterError as dagster_error: + yield ComputeNodeResult.failure_result( + compute_node=self, + tag=self.tag, + failure_data=ComputeNodeFailureData( + dagster_user_exception=dagster_error, + ), + ) + return def output_named(self, name): check.str_param(name, 'name') @@ -526,6 +542,7 @@ def create_expectation_cn( arg_dict={}, compute_fn=_create_expectation_lambda( solid, + inout_def, expectation_def, EXPECTATION_VALUE_OUTPUT, ), @@ -760,20 +777,32 @@ def _create_join_lambda(_context, _compute_node, inputs): yield Result(output_name=JOIN_OUTPUT, value=list(inputs.values())[0]) -def _create_expectation_lambda(solid, expectation_def, output_name): +def _create_expectation_lambda(solid, inout_def, expectation_def, internal_output_name): check.inst_param(solid, 'solid', SolidDefinition) + check.inst_param(inout_def, 'inout_def', (InputDefinition, OutputDefinition)) check.inst_param(expectation_def, 'expectations_def', ExpectationDefinition) - check.str_param(output_name, 'output_name') - - def _do_expectation(context, _compute_node, inputs): - expt_result = expectation_def.expectation_fn( - ExpectationExecutionInfo(context, solid, expectation_def), - inputs[EXPECTATION_INPUT], - ) - if expt_result.success: - yield Result(output_name=output_name, value=inputs[EXPECTATION_INPUT]) - else: - raise DagsterExpectationFailedError(None) # for now + check.str_param(internal_output_name, 'internal_output_name') + + def _do_expectation(context, compute_node, inputs): + with context.value('solid', solid.name),\ + context.value(inout_def.descriptive_key, inout_def.name),\ + context.value('expectation', expectation_def.name): + + value = inputs[EXPECTATION_INPUT] + info = ExpectationExecutionInfo(context, inout_def, solid, expectation_def) + expt_result = expectation_def.expectation_fn(info, value) + if expt_result.success: + context.debug('Expectation {friendly_name} succeeded on {value}.'.format( + friendly_name=compute_node.friendly_name, + value=value, + )) + yield Result(output_name=internal_output_name, value=inputs[EXPECTATION_INPUT]) + else: + context.debug('Expectation {friendly_name} failed on {value}.'.format( + friendly_name=compute_node.friendly_name, + value=value, + )) + raise DagsterExpectationFailedError(info, value) return _do_expectation diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index 33a2be6de6d7f..2c086a18811eb 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -642,6 +642,10 @@ def __init__(self, name, dagster_type=types.Any, expectations=None, description= ) self.description = check.opt_str_param(description, 'description') + @property + def descriptive_key(self): + return 'output' + class OutputDefinition(object): '''An OutputDefinition represents an output from a solid. Solids can have multiple @@ -668,6 +672,10 @@ def __init__(self, dagster_type=None, name=None, expectations=None, description= ) self.description = check.opt_str_param(description, 'description') + @property + def descriptive_key(self): + return 'output' + class SolidInputHandle(namedtuple('_SolidInputHandle', 'solid input_def')): def __new__(cls, solid, input_def): @@ -1220,5 +1228,5 @@ def __new__(cls, config, pipeline_def): ExpectationExecutionInfo = namedtuple( 'ExpectationExecutionInfo', - 'context solid_def expectation_def', -) + 'context inout_def solid_def expectation_def', +) \ No newline at end of file diff --git a/python_modules/dagster/dagster/core/errors.py b/python_modules/dagster/dagster/core/errors.py index ac0fcc1ab1199..481e9df2e5a36 100644 --- a/python_modules/dagster/dagster/core/errors.py +++ b/python_modules/dagster/dagster/core/errors.py @@ -58,14 +58,20 @@ def __init__(self, *args, **kwargs): class DagsterExpectationFailedError(DagsterError): '''Thrown with pipeline configured to throw on expectation failure''' - def __init__(self, execution_result, *args, **kwargs): + def __init__(self, info, value, *args, **kwargs): super(DagsterExpectationFailedError, self).__init__(*args, **kwargs) - # FIXME: need to reorganize to fix this circular dep - # Probable fix is to move all "execution result" objects - # to definitions - import dagster.core.execution - self.execution_result = check.opt_inst_param( - execution_result, - 'execution_result', - dagster.core.execution.SolidExecutionResult, + self.info = info + self.value = value + + def __repr__(self): + inout_def = self.info.inout_def + return ( + 'DagsterExpectationFailedError(' + + 'solid={name}, '.format(name=self.info.solid_def.name) + + '{key}={name}, '.format(key=inout_def.descriptive_key, name=inout_def.name) + + 'expectation={name}'.format(name=self.info.expectation_def.name + ) + 'value={value}'.format(value=repr(self.value)) + ')' ) + + def __str__(self): + return self.__repr__() diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index 1f5b7793cd2cd..3f23323ed6f3b 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -34,9 +34,10 @@ ) from .errors import ( + DagsterEvaluateValueError, DagsterInvariantViolationError, DagsterTypeError, - DagsterEvaluateValueError, + DagsterUserCodeExecutionError, ) from .compute_nodes import ( @@ -179,7 +180,13 @@ def reraise_user_error(self): self.input_expectations, self.output_expectations, self.transforms ): if not result.success: - six.reraise(*result.failure_data.dagster_user_exception.original_exc_info) + if isinstance( + result.failure_data.dagster_user_exception, + DagsterUserCodeExecutionError, + ): + six.reraise(*result.failure_data.dagster_user_exception.original_exc_info) + else: + raise result.failure_data.dagster_user_exception @property def dagster_user_exception(self): diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_ten/test_intro_tutorial_part_ten.py b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_ten/test_intro_tutorial_part_ten.py new file mode 100644 index 0000000000000..1c7f4c8543413 --- /dev/null +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_ten/test_intro_tutorial_part_ten.py @@ -0,0 +1,121 @@ +# pylint: disable=W0622,W0614,W0401 +from logging import DEBUG + +import pytest + +from dagster import * + + +@solid( + config_def=ConfigDefinition(types.Int), + outputs=[ + OutputDefinition( + types.Int, + expectations=[ + ExpectationDefinition( + name="check_positive", + expectation_fn=lambda _info, value: ExpectationResult(success=value > 0) + ) + ] + ) + ], +) +def injest_a(info): + return info.config + + +@solid( + config_def=ConfigDefinition(types.Int), + outputs=[OutputDefinition(types.Int)], +) +def injest_b(info): + return info.config + + +@lambda_solid( + inputs=[InputDefinition('num_one', types.Int), + InputDefinition('num_two', types.Int)], + output=OutputDefinition(types.Int), +) +def add_ints(num_one, num_two): + return num_one + num_two + + +def define_part_ten_step_one(): + return PipelineDefinition( + name='part_ten_step_one', + solids=[injest_a, injest_b, add_ints], + dependencies={ + 'add_ints': { + 'num_one': DependencyDefinition('injest_a'), + 'num_two': DependencyDefinition('injest_b'), + }, + }, + ) + + +def test_intro_tutorial_part_ten_step_one(): + result = execute_pipeline( + define_part_ten_step_one(), + config.Environment( + context=config.Context(config={ + 'log_level': 'DEBUG', + }), + solids={ + 'injest_a': config.Solid(2), + 'injest_b': config.Solid(3), + } + ), + ) + + assert result.success + + +def test_intro_tutorial_part_ten_step_two_fails_hard(): + with pytest.raises(DagsterExpectationFailedError): + execute_pipeline( + define_part_ten_step_one(), + config.Environment( + context=config.Context(config={ + 'log_level': 'DEBUG', + }), + solids={ + 'injest_a': config.Solid(-2), + 'injest_b': config.Solid(3), + } + ), + ) + + +def test_intro_tutorial_part_ten_step_two_fails_soft(): + result = execute_pipeline( + define_part_ten_step_one(), + config.Environment( + context=config.Context(config={ + 'log_level': 'DEBUG', + }), + solids={ + 'injest_a': config.Solid(-2), + 'injest_b': config.Solid(3), + } + ), + throw_on_error=False, + ) + + assert not result.success + + +if __name__ == '__main__': + execute_pipeline( + define_part_ten_step_one(), + config.Environment( + context=config.Context(config={ + 'log_level': 'DEBUG', + }), + solids={ + 'injest_a': config.Solid(-2), + 'injest_b': config.Solid(3), + }, + expectations=config.Expectations(evaluate=False), + ), + ) diff --git a/python_modules/dagster/docs/index.rst b/python_modules/dagster/docs/index.rst index fb9726739f747..f159b77b04438 100644 --- a/python_modules/dagster/docs/index.rst +++ b/python_modules/dagster/docs/index.rst @@ -30,6 +30,7 @@ Intro Tutorial intro_tutorial/part_seven intro_tutorial/part_eight intro_tutorial/part_nine + intro_tutorial/part_ten API Reference diff --git a/python_modules/dagster/docs/intro_tutorial/part_ten.rst b/python_modules/dagster/docs/intro_tutorial/part_ten.rst new file mode 100644 index 0000000000000..eab7eda2d3384 --- /dev/null +++ b/python_modules/dagster/docs/intro_tutorial/part_ten.rst @@ -0,0 +1,196 @@ +Expectations +============ + +Dagster has a first-class concept to capture data quality tests. We call these +data quality tests expectations. + +Data pipelines have the property that they typically do not control +what data they injest. Unlike a traditional application where you can +prevent users from entering malformed data, data pipelines do not have +that option. When unexpected data enters a pipeline and causes a software +error, typically the only recourse is to update your code. + +Lying within the code of data pipelines are a whole host of implicit +assumptions about the nature of the data. One way to frame the goal of +expectations is to say that they make those implict assumption explicit. +And by making these a first class concept they can be described with metadata, +inspected, and configured to run in different ways. + +Let us return to a slightly simplified version of the data pipeline from part nine. + +.. code-block:: python + + @solid( + config_def=ConfigDefinition(types.Int), + outputs=[OutputDefinition(types.Int)], + ) + def injest_a(info): + return info.config + + + @solid( + config_def=ConfigDefinition(types.Int), + outputs=[OutputDefinition(types.Int)], + ) + def injest_b(info): + return info.config + + @solid( + inputs=[InputDefinition('num_one', types.Int), + InputDefinition('num_two', types.Int)], + outputs=[OutputDefinition(types.Int)], + ) + def add_ints(_info, num_one, num_two): + return num_one + num_two + + + def define_part_ten_step_one(): + return PipelineDefinition( + name='part_ten_step_one', + solids=[injest_a, injest_b, add_ints], + dependencies={ + 'add_ints': { + 'num_one': DependencyDefinition('injest_a'), + 'num_two': DependencyDefinition('injest_b'), + }, + }, + ) + +Imagine that we had assumptions baked into the code of this pipeline such that the code only +worked on positive numbers, and we wanted to communicate that requirement to the user +in clear terms. We'll add an expectation in order to do this. + + +.. code-block:: python + + @solid( + config_def=ConfigDefinition(types.Int), + outputs=[ + OutputDefinition( + types.Int, + expectations=[ + ExpectationDefinition( + name="check_positive", + expectation_fn=lambda _info, value: ExpectationResult(success=value > 0) + ), + ], + ), + ], + ) + def injest_a(info): + return info.config + + +You'll notice that we added an ExpectationDefinition to the output of injest_a. Expectations +can be attached to inputs or outputs and operate on the value of that input or output. + +Expectations perform arbitrary computation on that value and then return an ExpectationResult. +The user communicates whether or not the expectation succeeded via this return value. + +If you run this pipeline, you'll notice some logging that indicates that the expectation +was processed: + +.. code-block:: python + + execute_pipeline( + define_part_ten_step_one(), + config.Environment( + context=config.Context(config={ + 'log_level': 'DEBUG', + }), + solids={ + 'injest_a': config.Solid(2), + 'injest_b': config.Solid(3), + } + ), + ) + +And run it... + +.. code-block:: sh + + $ python part_ten.py + ... log spew + 2018-09-14 13:13:13 - dagster - DEBUG - orig_message="Expectation injest_a.result.expectation.check_positive succeeded on 2." log_message_id="938ab7fa-c955-408a-9f44-66b0b6ecdcad" pipeline="part_ten_step_one" solid="injest_a" output="result" expectation="check_positive" + ... more log spew + +Now let's make this fail. Currently the default behavior is to throw an error and halt execution +when an expectation fails. So: + +.. code-block:: python + + execute_pipeline( + define_part_ten_step_one(), + config.Environment( + context=config.Context(config={ + 'log_level': 'DEBUG', + }), + solids={ + 'injest_a': config.Solid(-2), # oh noes! + 'injest_b': config.Solid(3), + } + ), + ) + +And then: + +.. code-block:: sh + + $ python part_ten.py + ... bunch of log spew + dagster.core.errors.DagsterExpectationFailedError: DagsterExpectationFailedError(solid=injest_a, output=result, expectation=check_positivevalue=-2) + +We can also tell execute_pipeline to not throw on error: + +.. code-block:: python + + execute_pipeline( + define_part_ten_step_one(), + config.Environment( + context=config.Context(config={ + 'log_level': 'DEBUG', + }), + solids={ + 'injest_a': config.Solid(-2), # oh noes! + 'injest_b': config.Solid(3), + } + ), + throw_on_error=False, + ) + +.. code-block:: sh + + $ python part_ten.py + ... log spew + 2018-09-14 13:31:09 - dagster - DEBUG - orig_message="Expectation injest_a.result.expectation.check_positive failed on -2." log_message_id="24bbaa2a-34a2-4817-b364-199a6d9f6066" pipeline="part_ten_step_one" solid="injest_a" output="result" expectation="check_positive" + +Because the system is explictly aware of these expectations they are viewable in tools like dagit. +It can also configure the execution of these expectations. The capabilities of this aspect of the +system are currently quite immature, but we expect to develop these more in the future. The only +feature right now is the ability to skip expectations entirely. This is useful in a case where +expectations are expensive and you have a time-critical job you must. In that case you can +configure the pipeline to skip expectations entirely. + + +.. code-block:: python + + execute_pipeline( + define_part_ten_step_one(), + config.Environment( + context=config.Context(config={ + 'log_level': 'DEBUG', + }), + solids={ + 'injest_a': config.Solid(-2), # oh noes! + 'injest_b': config.Solid(3), + }, + expectations=config.Expectations(evaluate=True), + ), + ) + +.. code-block:: sh + + $ python part_ten.py + ... expectations will not in the log spew + +We plan on adding more sophisticated capabilties to this in the future. From 5e3192ea837c411bbeb0e1153b31dbdd3e06d052 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Fri, 14 Sep 2018 13:55:08 -0700 Subject: [PATCH 095/103] Add context.values This is nicer for popping on more than one value on the context stack at once --- .../dagster/dagster/core/compute_nodes.py | 9 +++++---- .../core/core_tests/test_context_logging.py | 16 ++++++++++++++++ .../dagster/dagster/core/execution_context.py | 14 +++++++++++--- 3 files changed, 32 insertions(+), 7 deletions(-) diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index 7ac16f82b061f..4667de43079ba 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -784,10 +784,11 @@ def _create_expectation_lambda(solid, inout_def, expectation_def, internal_outpu check.str_param(internal_output_name, 'internal_output_name') def _do_expectation(context, compute_node, inputs): - with context.value('solid', solid.name),\ - context.value(inout_def.descriptive_key, inout_def.name),\ - context.value('expectation', expectation_def.name): - + with context.values({ + 'solid': solid.name, + inout_def.descriptive_key: inout_def.name, + 'expectation': expectation_def.name + }): value = inputs[EXPECTATION_INPUT] info = ExpectationExecutionInfo(context, inout_def, solid, expectation_def) expt_result = expectation_def.expectation_fn(info, value) diff --git a/python_modules/dagster/dagster/core/core_tests/test_context_logging.py b/python_modules/dagster/dagster/core/core_tests/test_context_logging.py index af6d624cbaa1c..17b5041a8534b 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_context_logging.py +++ b/python_modules/dagster/dagster/core/core_tests/test_context_logging.py @@ -127,3 +127,19 @@ def test_message_specific_logging(): assert set(message_one.extra.keys()) == set( ['key_one', 'key_two', 'log_message_id', 'orig_message'] ) + + +def test_multicontext_value(): + logger = LoggerForTest() + context = ExecutionContext(loggers=[logger]) + with context.values({ + 'key_one': 'value_one', + 'key_two': 'value_two', + }): + context.info('message one') + + message_two = logger.messages[0] + assert message_two.extra['key_one'] == 'value_one' + assert message_two.extra['key_two'] == 'value_two' + assert 'key_one="value_one"' in message_two.msg + assert 'key_two="value_two"' in message_two.msg diff --git a/python_modules/dagster/dagster/core/execution_context.py b/python_modules/dagster/dagster/core/execution_context.py index 272f4fff70c83..a2dcfc39a0f94 100644 --- a/python_modules/dagster/dagster/core/execution_context.py +++ b/python_modules/dagster/dagster/core/execution_context.py @@ -165,16 +165,24 @@ def value(self, key, value): context.info('msg without some_key context value') ''' + check.str_param(key, 'key') check.not_none_param(value, 'value') + with self.values({key: value}): + yield - check.invariant(not key in self._context_dict, 'Should not be in context') + @contextmanager + def values(self, ddict): + check.dict_param(ddict, 'ddict') - self._context_dict[key] = value + for key, value in ddict.items(): + check.invariant(not key in self._context_dict, 'Should not be in context') + self._context_dict[key] = value yield - self._context_dict.pop(key) + for key in ddict.keys(): + self._context_dict.pop(key) class TransformExecutionInfo(namedtuple('_TransformExecutionInfo', 'context config')): From d91fe9bc5518549f4a3db6ae8b655b28f2d5f9b2 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Sun, 16 Sep 2018 17:15:42 -0700 Subject: [PATCH 096/103] Tutorial part 11 Along with improved logging and error reporting that was surfaced by writing this tutorial. --- .../dagster/dagster/core/compute_nodes.py | 17 +- .../core/core_tests/test_custom_context.py | 2 +- .../core/core_tests/test_multiple_outputs.py | 32 +++ .../core_tests/test_pipeline_execution.py | 12 +- .../dagster/dagster/core/definitions.py | 5 +- .../dagster/dagster/core/execution.py | 69 ++++--- .../test_intro_tutorial_part_eleven.py | 152 ++++++++++++++ python_modules/dagster/docs/index.rst | 1 + .../docs/intro_tutorial/part_eleven.rst | 191 ++++++++++++++++++ 9 files changed, 446 insertions(+), 35 deletions(-) create mode 100644 python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eleven/test_intro_tutorial_part_eleven.py create mode 100644 python_modules/dagster/docs/intro_tutorial/part_eleven.rst diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index 4667de43079ba..ece89b021f7a3 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -224,7 +224,7 @@ def _yield_transform_results(context, compute_node, conf, inputs): def _collect_result_list(context, compute_node, conf, inputs): for result in _yield_transform_results(context, compute_node, conf, inputs): - context.debug( + context.info( 'Solid {solid} emitted output "{output}" value {value}'.format( solid=compute_node.solid.name, output=result.output_name, @@ -245,9 +245,11 @@ def _execute_core_transform(context, compute_node, conf, inputs): error_str = 'Error occured during core transform' - with context.value('solid', compute_node.solid.name): + solid = compute_node.solid + + with context.value('solid', solid.name): context.debug( - 'Executing core transform for solid {solid}.'.format(solid=compute_node.solid.name) + 'Executing core transform for solid {solid}.'.format(solid=solid.name) ) with time_execution_scope() as timer_result, \ @@ -255,6 +257,15 @@ def _execute_core_transform(context, compute_node, conf, inputs): all_results = list(_collect_result_list(context, compute_node, conf, inputs)) + if len(all_results) != len(solid.output_defs): + emitted_result_names = set([r.output_name for r in all_results]) + solid_output_names = set([output_def.name for output_def in solid.output_defs]) + omitted_outputs = solid_output_names.difference(emitted_result_names) + context.info('Solid {solid} did not fire outputs {outputs}'.format( + solid=solid.name, + outputs=repr(omitted_outputs), + )) + context.debug( 'Finished executing transform for solid {solid}. Time elapsed: {millis:.3f} ms'.format( solid=compute_node.solid.name, diff --git a/python_modules/dagster/dagster/core/core_tests/test_custom_context.py b/python_modules/dagster/dagster/core/core_tests/test_custom_context.py index eb3e363baac70..f6d69f3a941f9 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_custom_context.py +++ b/python_modules/dagster/dagster/core/core_tests/test_custom_context.py @@ -27,7 +27,7 @@ def test_default_context(): def default_context_transform(info): called['yes'] = True for logger in info.context._logger.loggers: - assert logger.level == ERROR + assert logger.level == INFO pipeline = PipelineDefinition(solids=[default_context_transform]) execute_pipeline(pipeline) diff --git a/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py b/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py index 3cab6d7ee4c06..1e621e2673610 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py +++ b/python_modules/dagster/dagster/core/core_tests/test_multiple_outputs.py @@ -39,6 +39,12 @@ def _t_fn(*_args): assert solid_result.transformed_value('output_one') == 'foo' assert solid_result.transformed_value('output_two') == 'bar' + with pytest.raises( + DagsterInvariantViolationError, + match='not_defined not defined in solid multiple_outputs', + ): + solid_result.transformed_value('not_defined') + def test_multiple_outputs_expectations(): called = {} @@ -179,3 +185,29 @@ def _transform_fn_two(*_args, **_kwargs): assert result.success assert called['one'] + solid_result = result.result_for_solid('multiple_outputs') + assert set(solid_result.transformed_values.keys()) == set(['output_one']) + + with pytest.raises( + DagsterInvariantViolationError, + match='not_defined not defined in solid multiple_outputs', + ): + solid_result.transformed_value('not_defined') + + with pytest.raises( + DagsterInvariantViolationError, + match='Did not find result output_two', + ): + solid_result.transformed_value('output_two') + + with pytest.raises( + DagsterInvariantViolationError, + match='Try to get result for solid not_present in <>. No such solid.', + ): + result.result_for_solid('not_present') + + with pytest.raises( + DagsterInvariantViolationError, + match='Did not find result for solid downstream_two in pipeline execution result', + ): + result.result_for_solid('downstream_two') diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py index 162ca92a3ff8d..60af8428922d1 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_execution.py @@ -3,7 +3,6 @@ InputDefinition, OutputDefinition, PipelineDefinition, - SolidDefinition, check, config, execute_pipeline, @@ -16,7 +15,10 @@ ) from dagster.core.execution import ( - execute_pipeline_iterator, ExecutionContext, SolidExecutionResult, PipelineExecutionResult + execute_pipeline_iterator, + ExecutionContext, + SolidExecutionResult, + PipelineExecutionResult, ) from dagster.core.test_utils import single_output_transform @@ -242,7 +244,7 @@ def assert_all_results_equivalent(expected_results, result_results): def test_pipeline_execution_graph_diamond(): pipeline = PipelineDefinition(solids=create_diamond_solids(), dependencies=diamond_deps()) environment = config.Environment() - return _do_test(lambda: execute_pipeline_iterator( + return _do_test(pipeline, lambda: execute_pipeline_iterator( pipeline, environment=environment, )) @@ -275,14 +277,14 @@ def test_create_single_solid_pipeline(): # )) -def _do_test(do_execute_pipeline_iter): +def _do_test(pipeline, do_execute_pipeline_iter): results = list() for result in do_execute_pipeline_iter(): results.append(result) - result = PipelineExecutionResult(ExecutionContext(), results) + result = PipelineExecutionResult(pipeline, ExecutionContext(), results) assert result.result_for_solid('A').transformed_value() == [ input_set('A_input'), transform_called('A') diff --git a/python_modules/dagster/dagster/core/definitions.py b/python_modules/dagster/dagster/core/definitions.py index 2c086a18811eb..264881475cd2e 100644 --- a/python_modules/dagster/dagster/core/definitions.py +++ b/python_modules/dagster/dagster/core/definitions.py @@ -147,7 +147,7 @@ def _default_context_fn(info): 'log_level': Field( dagster_type=types.String, is_optional=True, - default_value='ERROR', + default_value='INFO', ) } ), @@ -781,12 +781,13 @@ def config_dict(field_dict): ''' return ConfigDefinition(types.ConfigDictionary(field_dict)) - def __init__(self, config_type=types.Any): + def __init__(self, config_type=types.Any, description=None): '''Construct a ConfigDefinition Args: config_type (DagsterType): Type the determines shape and values of config''' self.config_type = check.inst_param(config_type, 'config_type', DagsterType) + self.description = check.opt_str_param(description, 'description') class SolidDefinition(object): diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index 3f23323ed6f3b..5fb387023d2f0 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -48,25 +48,25 @@ execute_compute_nodes, ) -from .execution_context import ( - ExecutionContext, - TransformExecutionInfo, -) +from .execution_context import ExecutionContext class PipelineExecutionResult(object): '''Result of execution of the whole pipeline. Returned eg by :py:function:`execute_pipeline`. Attributes: - context (ExecutionContext): ExecutionContext of that particular Pipeline run. - result_list (list[SolidExecutionResult]): List of results for each pipeline solid. + pipeline (PipelineDefinition): Pipeline that was executed + context (ExecutionContext): ExecutionContext of that particular Pipeline run. + result_list (list[SolidExecutionResult]): List of results for each pipeline solid. ''' def __init__( self, + pipeline, context, result_list, ): + self.pipeline = check.inst_param(pipeline, 'pipeline', PipelineDefinition) self.context = check.inst_param(context, 'context', ExecutionContext) self.result_list = check.list_param( result_list, 'result_list', of_type=SolidExecutionResult @@ -84,10 +84,20 @@ def result_for_solid(self, name): SolidExecutionResult ''' check.str_param(name, 'name') + + if not self.pipeline.has_solid(name): + raise DagsterInvariantViolationError( + 'Try to get result for solid {name} in {pipeline}. No such solid.'.format( + name=name, + pipeline=self.pipeline.display_name, + ) + ) + for result in self.result_list: if result.solid.name == name: return result - check.failed( + + raise DagsterInvariantViolationError( 'Did not find result for solid {name} in pipeline execution result'.format(name=name) ) @@ -163,11 +173,20 @@ def transformed_value(self, output_name=DEFAULT_OUTPUT): '''Returns transformed value either for DEFAULT_OUTPUT or for the output given as output_name. Returns None if execution result isn't a success''' check.str_param(output_name, 'output_name') + + if not self.solid.has_output(output_name): + raise DagsterInvariantViolationError( + '{output_name} not defined in solid {solid}'.format( + output_name=output_name, + solid=self.solid.name, + ) + ) + if self.success: for result in self.transforms: if result.success_data.output_name == output_name: return result.success_data.value - check.failed( + raise DagsterInvariantViolationError( 'Did not find result {output_name} in solid {self.solid.name} execution result'. format(output_name=output_name, self=self) ) @@ -198,14 +217,6 @@ def dagster_user_exception(self): return result.failure_data.dagster_user_exception -def _do_throw_on_error(execution_result): - check.inst_param(execution_result, 'execution_result', SolidExecutionResult) - if execution_result.success: - return - else: - execution_result.reraise_user_error() - - def _wrap_in_yield(context_or_generator): if isinstance(context_or_generator, ExecutionContext): @@ -382,19 +393,29 @@ def _execute_graph( check.inst_param(environment, 'environment', config.Environment) check.bool_param(throw_on_error, 'throw_on_error') + display_name = execution_graph.pipeline.display_name results = [] with yield_context(execution_graph.pipeline, environment) as context, \ context.value('pipeline', execution_graph.pipeline.display_name): - context.info( - 'Beginning execution of pipeline {pipeline}', - pipeline=execution_graph.pipeline.display_name, - ) + context.info('Beginning execution of pipeline {pipeline}'.format(pipeline=display_name)) for result in _execute_graph_iterator(context, execution_graph, environment): - if throw_on_error: - if not result.success: - _do_throw_on_error(result) + if throw_on_error and not result.success: + result.reraise_user_error() results.append(result) - return PipelineExecutionResult(context, results) + + pipeline_result = PipelineExecutionResult(execution_graph.pipeline, context, results) + if pipeline_result.success: + context.info( + 'Completing successful execution of pipeline {pipeline}'.format( + pipeline=display_name + ) + ) + else: + context.info( + 'Completing failing execution of pipeline {pipeline}'.format(pipeline=display_name) + ) + + return pipeline_result diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eleven/test_intro_tutorial_part_eleven.py b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eleven/test_intro_tutorial_part_eleven.py new file mode 100644 index 0000000000000..8c26ee07ef255 --- /dev/null +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eleven/test_intro_tutorial_part_eleven.py @@ -0,0 +1,152 @@ +# pylint: disable=W0622,W0614,W0401 +import pytest + +from dagster import * + + +@solid( + outputs=[ + OutputDefinition(dagster_type=types.Int, name='out_one'), + OutputDefinition(dagster_type=types.Int, name='out_two'), + ], +) +def yield_outputs(_info): + yield Result(23, 'out_one') + yield Result(45, 'out_two') + + +@solid( + outputs=[ + OutputDefinition(dagster_type=types.Int, name='out_one'), + OutputDefinition(dagster_type=types.Int, name='out_two'), + ], +) +def return_dict_results(_info): + return MultipleResults.from_dict({ + 'out_one': 23, + 'out_two': 45, + }) + + +@solid( + config_def=ConfigDefinition(types.String, description='Should be either out_one or out_two'), + outputs=[ + OutputDefinition(dagster_type=types.Int, name='out_one'), + OutputDefinition(dagster_type=types.Int, name='out_two'), + ], +) +def conditional(info): + if info.config == 'out_one': + yield Result(23, 'out_one') + elif info.config == 'out_two': + yield Result(45, 'out_two') + else: + raise Exception('invalid config') + + +@solid(inputs=[InputDefinition('num', dagster_type=types.Int)]) +def log_num(info, num): + info.context.info('num {num}'.format(num=num)) + return num + + +@solid(inputs=[InputDefinition('num', dagster_type=types.Int)]) +def log_num_squared(info, num): + info.context.info('num_squared {num_squared}'.format(num_squared=num * num)) + return num * num + + +def define_part_eleven_step_one(): + return PipelineDefinition( + name='part_eleven_step_one', + solids=[return_dict_results, log_num, log_num_squared], + dependencies={ + 'log_num': { + 'num': DependencyDefinition('return_dict_results', 'out_one') + }, + 'log_num_squared': { + 'num': DependencyDefinition('return_dict_results', 'out_two') + }, + }, + ) + + +def define_part_eleven_step_two(): + return PipelineDefinition( + name='part_eleven_step_two', + solids=[yield_outputs, log_num, log_num_squared], + dependencies={ + 'log_num': { + 'num': DependencyDefinition('yield_outputs', 'out_one') + }, + 'log_num_squared': { + 'num': DependencyDefinition('yield_outputs', 'out_two') + }, + }, + ) + + +def define_part_eleven_step_three(): + return PipelineDefinition( + name='part_eleven_step_three', + solids=[conditional, log_num, log_num_squared], + dependencies={ + 'log_num': { + 'num': DependencyDefinition('conditional', 'out_one') + }, + 'log_num_squared': { + 'num': DependencyDefinition('conditional', 'out_two') + }, + }, + ) + + +def test_intro_tutorial_part_eleven_step_one(): + result = execute_pipeline(define_part_eleven_step_one()) + + assert result.success + assert result.result_for_solid('return_dict_results').transformed_value('out_one') == 23 + assert result.result_for_solid('return_dict_results').transformed_value('out_two') == 45 + assert result.result_for_solid('log_num').transformed_value() == 23 + assert result.result_for_solid('log_num_squared').transformed_value() == 45 * 45 + + +def test_intro_tutorial_part_eleven_step_two(): + result = execute_pipeline(define_part_eleven_step_two()) + + assert result.success + assert result.result_for_solid('yield_outputs').transformed_value('out_one') == 23 + assert result.result_for_solid('yield_outputs').transformed_value('out_two') == 45 + assert result.result_for_solid('log_num').transformed_value() == 23 + assert result.result_for_solid('log_num_squared').transformed_value() == 45 * 45 + + +def test_intro_tutorial_part_eleven_step_three(): + result = execute_pipeline( + define_part_eleven_step_three(), + config.Environment(solids={'conditional': config.Solid('out_two')}) + ) + + # successful things + assert result.success + assert result.result_for_solid('conditional').transformed_value('out_two') == 45 + assert result.result_for_solid('log_num_squared').transformed_value() == 45 * 45 + + # unsuccessful things + with pytest.raises(DagsterInvariantViolationError): + assert result.result_for_solid('conditional').transformed_value('out_one') == 45 + + with pytest.raises(DagsterInvariantViolationError): + result.result_for_solid('unnamed') + + with pytest.raises(DagsterInvariantViolationError): + result.result_for_solid('log_num') + + +if __name__ == '__main__': + execute_pipeline( + define_part_eleven_step_three(), + config.Environment(solids={'conditional': config.Solid('out_two')}) + ) + + # execute_pipeline(define_part_eleven_step_two()) diff --git a/python_modules/dagster/docs/index.rst b/python_modules/dagster/docs/index.rst index f159b77b04438..ec66ac17ce48f 100644 --- a/python_modules/dagster/docs/index.rst +++ b/python_modules/dagster/docs/index.rst @@ -31,6 +31,7 @@ Intro Tutorial intro_tutorial/part_eight intro_tutorial/part_nine intro_tutorial/part_ten + intro_tutorial/part_eleven API Reference diff --git a/python_modules/dagster/docs/intro_tutorial/part_eleven.rst b/python_modules/dagster/docs/intro_tutorial/part_eleven.rst new file mode 100644 index 0000000000000..cba32e0f0ce82 --- /dev/null +++ b/python_modules/dagster/docs/intro_tutorial/part_eleven.rst @@ -0,0 +1,191 @@ +Multiple Outputs +---------------- + +So far all of the examples have been solids that have a single output. However +solids support an arbitrary number of outputs. This allows for downstream +solids to only tie their dependency to a single output. Additionally -- by +allowing for multiple outputs to conditionally fire -- this also ends up +supporting dynamic branching and conditional execution of pipelines. + + +.. code-block:: python + + @solid( + outputs=[ + OutputDefinition(dagster_type=types.Int, name='out_one'), + OutputDefinition(dagster_type=types.Int, name='out_two'), + ], + ) + def return_dict_results(_info): + return MultipleResults.from_dict({ + 'out_one': 23, + 'out_two': 45, + }) + + @solid(inputs=[InputDefinition('num', dagster_type=types.Int)]) + def log_num(info, num): + info.context.info('num {num}'.format(num=num)) + return num + + @solid(inputs=[InputDefinition('num', dagster_type=types.Int)]) + def log_num_squared(info, num): + info.context.info( + 'num_squared {num_squared}'.format(num_squared=num * num) + ) + return num * num + +Notice how ``return_dict_results`` has two outputs. For the first time +we have provided the name argument to an :py:class:`OutputDefinition`. (It +defaults to ``'result'``, as it does in a :py:class:`DependencyDefinition`) +These names must be unique and results returns by a solid transform function +must be named one of these inputs. (In all previous examples the value returned +by the transform had been wrapped in a :py:class:`Result` object with the name +``'result'``.) + +So from ``return_dict_results`` we used :py:class:`MultipleResults` to return +all outputs from this transform. + +Next let's examine the :py:class:`PipelineDefinition`: + +.. code-block:: python + + def define_part_eleven_step_one(): + return PipelineDefinition( + name='part_eleven_step_one', + solids=[return_dict_results, log_num, log_num_squared], + dependencies={ + 'log_num': { + 'num': DependencyDefinition( + 'return_dict_results', + 'out_one', + ), + }, + 'log_num_squared': { + 'num': DependencyDefinition( + 'return_dict_results', + 'out_two', + ), + }, + }, + ) + +Just like this tutorial is the first example of an :py:class:`OutputDefinition` with +a name, this is also the first time that a :py:class:`DependencyDefinition` has +specified name, because dependencies point to a particular **output** of a solid, +rather than to the solid itself. In previous examples the name of output has been +defaulted to ``'result'``. + +With this we can run the pipeline: + +.. code-block:: python + + execute_pipeline(define_part_eleven_step_one()) + +and run it: foobar + +.. code-block:: sh + + python step_eleven.py + ... log spew + 2018-09-16 17:08:09 - dagster - INFO - orig_message="Solid return_dict_results emitted output \"out_one\" value 23" log_message_id="76fe7e9b-f11c-43a3-ac17-8bc8616bd0bd" pipeline="part_eleven_step_one" solid="return_dict_results" + 2018-09-16 17:08:09 - dagster - INFO - orig_message="Solid return_dict_results emitted output \"out_two\" value 45" log_message_id="ef11a36a-da7b-4df1-9eeb-0f92c04d392a" pipeline="part_eleven_step_one" solid="return_dict_results" + ... more log spew + +The :py:class:`MultipleResults` class is not the only way to return multiple to +results from a solid transform function. You can also yield multiple instances +of the `Result` object. (Note: this is actually the core specification +of the transform function: all other forms are implemented in terms of +the iterator form.) + +.. code-block:: python + + @solid( + outputs=[ + OutputDefinition(dagster_type=types.Int, name='out_one'), + OutputDefinition(dagster_type=types.Int, name='out_two'), + ], + ) + def yield_outputs(_info): + yield Result(23, 'out_one') + yield Result(45, 'out_two') + + def define_part_eleven_step_two(): + return PipelineDefinition( + name='part_eleven_step_two', + solids=[yield_outputs, log_num, log_num_squared], + dependencies={ + 'log_num': { + 'num': DependencyDefinition('yield_outputs', 'out_one') + }, + 'log_num_squared': { + 'num': DependencyDefinition('yield_outputs', 'out_two') + }, + }, + ) + + if __name__ == '__main__': + execute_pipeline(define_part_eleven_step_two()) + +... and you'll see the same log spew around outputs in this version: + +.. code-block:: sh + $ python part_eleven.py + 2018-09-16 17:53:15 - dagster - INFO - orig_message="Solid yield_outputs emitted output \"out_one\" value 23" log_message_id="7313cb9c-85dc-4467-9f48-a724a75db63f" pipeline="part_eleven_step_two" solid="yield_outputs" + 2018-09-16 17:53:15 - dagster - INFO - orig_message="Solid yield_outputs emitted output \"out_two\" value 45" log_message_id="fed2866f-5f29-4bbd-b124-90bd9eda8690" pipeline="part_eleven_step_two" solid="yield_outputs" + +Conditional Outputs +^^^^^^^^^^^^^^^^^^^ + +Multiple outputs are the mechanism by which we implement branching or conditional execution. + +Let's modify the first solid above to conditionally emit one output or the other based on config +and then execute that pipeline. + +.. code-block:: python + + @solid( + config_def=ConfigDefinition(types.String, description='Should be either out_one or out_two'), + outputs=[ + OutputDefinition(dagster_type=types.Int, name='out_one'), + OutputDefinition(dagster_type=types.Int, name='out_two'), + ], + ) + def conditional(info): + if info.config == 'out_one': + yield Result(23, 'out_one') + elif info.config == 'out_two': + yield Result(45, 'out_two') + else: + raise Exception('invalid config') + + + def define_part_eleven_step_three(): + return PipelineDefinition( + name='part_eleven_step_three', + solids=[conditional, log_num, log_num_squared], + dependencies={ + 'log_num': { + 'num': DependencyDefinition('conditional', 'out_one') + }, + 'log_num_squared': { + 'num': DependencyDefinition('conditional', 'out_two') + }, + }, + ) + + if __name__ == '__main__': + execute_pipeline( + define_part_eleven_step_three(), + config.Environment(solids={'conditional': config.Solid('out_two')}) + ) + +Note that we are configuring this solid to *only* emit out_two which will end up +only triggering log_num_squared. log_num will never be executed. + +.. code-block:: sh + + $ python part_eleven.py + ... log spew + 2018-09-16 18:58:32 - dagster - INFO - orig_message="Solid conditional emitted output \"out_two\" value 45" log_message_id="f6fd78c5-c25e-40ea-95ef-6b80d12155de" pipeline="part_eleven_step_three" solid="conditional" + 2018-09-16 18:58:32 - dagster - INFO - orig_message="Solid conditional did not fire outputs {'out_one'}" log_message_id="d548ea66-cb10-42b8-b150-aed8162cc25c" pipeline="part_eleven_step_three" solid="conditional" + ... log spew From 919c5c37126ac5743e7a4d4834cd75375219e483 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Sun, 16 Sep 2018 17:51:07 -0700 Subject: [PATCH 097/103] Refactor things to improve custom typing Did a bunch of work to improve the custom typing. Included a substantial (but somewhat orthogonal) refactor of the core execution function in the compute engine. --- .../dagster/dagster/core/compute_nodes.py | 135 ++++++++------- .../core_tests/test_config_type_system.py | 44 +++++ .../core/core_tests/test_pipeline_errors.py | 4 +- .../dagster/core/core_tests/test_types.py | 19 ++- .../dagster/dagster/core/execution.py | 10 +- python_modules/dagster/dagster/core/types.py | 67 +++----- .../pandas_tests/test_pandas_user_error.py | 14 +- .../test_intro_tutorial_part_eleven.py | 0 .../test_intro_tutorial_part_ten.py | 0 .../test_intro_tutorial_part_twelve.py | 159 ++++++++++++++++++ .../docs/intro_tutorial/part_twelve.rst | 13 ++ 11 files changed, 340 insertions(+), 125 deletions(-) rename python_modules/dagster/dagster_tests/tutorials/{test_intro_tutorial_part_eleven => }/test_intro_tutorial_part_eleven.py (100%) rename python_modules/dagster/dagster_tests/tutorials/{test_intro_tutorial_part_ten => }/test_intro_tutorial_part_ten.py (100%) create mode 100644 python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_twelve.py create mode 100644 python_modules/dagster/docs/intro_tutorial/part_twelve.rst diff --git a/python_modules/dagster/dagster/core/compute_nodes.py b/python_modules/dagster/dagster/core/compute_nodes.py index ece89b021f7a3..f80a24e5d5c41 100644 --- a/python_modules/dagster/dagster/core/compute_nodes.py +++ b/python_modules/dagster/dagster/core/compute_nodes.py @@ -5,6 +5,8 @@ import uuid import sys +from future.utils import raise_from + import toposort from dagster import ( @@ -110,15 +112,11 @@ def __new__(cls, output_name, value): ) -class ComputeNodeFailureData(namedtuple('_ComputeNodeFailureData', 'dagster_user_exception')): - def __new__(cls, dagster_user_exception): +class ComputeNodeFailureData(namedtuple('_ComputeNodeFailureData', 'dagster_error')): + def __new__(cls, dagster_error): return super(ComputeNodeFailureData, cls).__new__( cls, - dagster_user_exception=check.inst_param( - dagster_user_exception, - 'dagster_user_exception', - DagsterError, - ), + dagster_error=check.inst_param(dagster_error, 'dagster_error', DagsterError), ) @@ -149,10 +147,6 @@ def failure_result(compute_node, tag, failure_data): ) -LOG_LEVEL = ERROR -logger = define_colored_console_logger('dagster-compute-nodes', LOG_LEVEL) - - @contextmanager def _user_code_error_boundary(context, msg, **kwargs): ''' @@ -172,11 +166,16 @@ def _user_code_error_boundary(context, msg, **kwargs): stack_trace = get_formatted_stack_trace(de) context.error(str(de), stack_trace=stack_trace) raise de - except Exception as e: + except Exception as e: # pylint: disable=W0703 stack_trace = get_formatted_stack_trace(e) context.error(str(e), stack_trace=stack_trace) - raise DagsterUserCodeExecutionError( - msg.format(**kwargs), e, user_exception=e, original_exc_info=sys.exc_info() + raise_from( + DagsterUserCodeExecutionError( + msg.format(**kwargs), + user_exception=e, + original_exc_info=sys.exc_info(), + ), + e, ) @@ -354,80 +353,88 @@ def _create_compute_node_result(self, result): ), ) - def execute(self, context, inputs): - check.inst_param(context, 'context', ExecutionContext) - check.dict_param(inputs, 'inputs', key_type=str) - - logger.debug('Entering execution for {self.friendly_name}'.format(self=self)) - - # do runtime type checks of inputs versus node inputs - for input_name, input_value in inputs.items(): - compute_node_input = self._node_input_dict[input_name] - if not compute_node_input.dagster_type.is_python_valid_value(input_value): - raise DagsterInvariantViolationError( - '''Solid {cn.solid.name} input {input_name} - received value {input_value} which does not match the type for Dagster type - {compute_node_input.dagster_type.name}. Compute node {cn.friendly_name}''' - .format( + def _get_evaluated_input(self, input_name, input_value): + compute_node_input = self._node_input_dict[input_name] + try: + return compute_node_input.dagster_type.evaluate_value(input_value) + except DagsterEvaluateValueError as evaluate_error: + raise_from( + DagsterTypeError(( + 'Solid {cn.solid.name} input {input_name} received value {input_value} ' + + 'which does not pass the typecheck for Dagster type ' + + '{compute_node_input.dagster_type.name}. Compute node {cn.friendly_name}' + ).format( cn=self, input_name=input_name, input_value=input_value, - compute_node_input=compute_node_input + compute_node_input=compute_node_input, ) - ) + ), + evaluate_error, + ) + def _compute_result_list(self, context, evaluated_inputs): error_str = 'Error occured during compute node {friendly_name}'.format( friendly_name=self.friendly_name, ) + with _user_code_error_boundary(context, error_str): + gen = self.compute_fn(context, self, evaluated_inputs) + + if gen is None: + check.invariant(not self.node_outputs) + return + + return list(gen) + + def _error_check_results(self, results): seen_outputs = set() + for result in results: + if not self.has_node(result.output_name): + output_names = list([output_def.name for output_def in self.solid.output_defs]) + raise DagsterInvariantViolationError( + '''Core transform for {cn.solid.name} returned an output + {result.output_name} that does not exist. The available + outputs are {output_names}''' + .format(cn=self, result=result, output_names=output_names) + ) - try: + if result.output_name in seen_outputs: + raise DagsterInvariantViolationError( + '''Core transform for {cn.solid.name} returned an output + {result.output_name} multiple times'''.format(cn=self, result=result) + ) + + seen_outputs.add(result.output_name) - with _user_code_error_boundary(context, error_str): - gen = self.compute_fn(context, self, inputs) - if gen is None: - check.invariant(not self.node_outputs) - return + def _execute_inner_compute_node_loop(self, context, inputs): + evaluated_inputs = {} + # do runtime type checks of inputs versus node inputs + for input_name, input_value in inputs.items(): + evaluated_inputs[input_name] = self._get_evaluated_input(input_name, input_value) - results = list(gen) + results = self._compute_result_list(context, evaluated_inputs) - for result in results: - if not self.has_node(result.output_name): - output_names = list([output_def.name for output_def in self.solid.output_defs]) - raise DagsterInvariantViolationError( - '''Core transform for {cn.solid.name} returned an output - {result.output_name} that does not exist. The available - outputs are {output_names}''' - .format(cn=self, result=result, output_names=output_names) - ) + self._error_check_results(results) - if result.output_name in seen_outputs: - raise DagsterInvariantViolationError( - '''Core transform for {cn.solid.name} returned an output - {result.output_name} multiple times'''.format(cn=self, result=result) - ) + return [self._create_compute_node_result(result) for result in results] - seen_outputs.add(result.output_name) + def execute(self, context, inputs): + check.inst_param(context, 'context', ExecutionContext) + check.dict_param(inputs, 'inputs', key_type=str) - yield self._create_compute_node_result(result) + try: + for compute_node_result in self._execute_inner_compute_node_loop(context, inputs): + yield compute_node_result - except DagsterUserCodeExecutionError as dagster_user_exception: - yield ComputeNodeResult.failure_result( - compute_node=self, - tag=self.tag, - failure_data=ComputeNodeFailureData( - dagster_user_exception=dagster_user_exception, - ), - ) - return except DagsterError as dagster_error: + context.error(str(dagster_error)) yield ComputeNodeResult.failure_result( compute_node=self, tag=self.tag, failure_data=ComputeNodeFailureData( - dagster_user_exception=dagster_error, + dagster_error=dagster_error, ), ) return diff --git a/python_modules/dagster/dagster/core/core_tests/test_config_type_system.py b/python_modules/dagster/dagster/core/core_tests/test_config_type_system.py index 744ade3ef4f12..f057f5f54aef7 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_config_type_system.py +++ b/python_modules/dagster/dagster/core/core_tests/test_config_type_system.py @@ -1,3 +1,5 @@ +from collections import namedtuple + import pytest from dagster import ( ConfigDefinition, @@ -313,3 +315,45 @@ def test_nested_optional_with_no_default(): } assert _validate(_nested_optional_config_with_no_default(), {'nested': {}}) == {'nested': {}} + + +CustomStructConfig = namedtuple('CustomStructConfig', 'foo bar') + + +class CustomStructConfigType(types.DagsterCompositeType): + def __init__(self): + super(CustomStructConfigType, self).__init__( + 'CustomStructConfigType', + { + 'foo': Field(types.String), + 'bar': Field(types.Int), + }, + lambda val: CustomStructConfig(foo=val['foo'], bar=val['bar']), + ) + + +def test_custom_composite_type(): + config_type = CustomStructConfigType() + + assert config_type.evaluate_value({ + 'foo': 'some_string', + 'bar': 2 + }) == CustomStructConfig( + foo='some_string', bar=2 + ) + + with pytest.raises(DagsterEvaluateValueError): + assert config_type.evaluate_value({ + 'foo': 'some_string', + }) + + with pytest.raises(DagsterEvaluateValueError): + assert config_type.evaluate_value({ + 'bar': 'some_string', + }) + + with pytest.raises(DagsterEvaluateValueError): + assert config_type.evaluate_value({ + 'foo': 'some_string', + 'bar': 'not_an_int', + }) diff --git a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py index 10f8ffc2fdddd..ca8beb302c5e9 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py +++ b/python_modules/dagster/dagster/core/core_tests/test_pipeline_errors.py @@ -70,7 +70,7 @@ def test_transform_failure_pipeline(): assert len(result_list) == 1 assert not result_list[0].success - assert result_list[0].dagster_user_exception + assert result_list[0].dagster_error def test_failure_midstream(): @@ -107,7 +107,7 @@ def transform_fn(_context, inputs): assert result_list[0].success assert result_list[1].success assert not result_list[2].success - assert isinstance(result_list[2].dagster_user_exception, DagsterUserCodeExecutionError) + assert isinstance(result_list[2].dagster_error, DagsterUserCodeExecutionError) def test_do_not_yield_result(): diff --git a/python_modules/dagster/dagster/core/core_tests/test_types.py b/python_modules/dagster/dagster/core/core_tests/test_types.py index c7a7eb97795fe..e80c68d81e2e5 100644 --- a/python_modules/dagster/dagster/core/core_tests/test_types.py +++ b/python_modules/dagster/dagster/core/core_tests/test_types.py @@ -1,4 +1,14 @@ -from dagster.core.types import (DagsterType, PythonObjectType) +from collections import namedtuple +import pytest +from dagster.core.types import ( + DagsterCompositeType, + DagsterEvaluateValueError, + DagsterType, + Field, + Int, + PythonObjectType, + String, +) def test_desc(): @@ -15,6 +25,7 @@ class Bar(object): assert type_bar.name == 'Bar' assert type_bar.description == 'A bar.' - assert type_bar.is_python_valid_value(Bar()) - assert type_bar.is_python_valid_value(None) # allow nulls - assert not type_bar.is_python_valid_value('not_a_bar') + assert type_bar.evaluate_value(Bar()) + assert type_bar.evaluate_value(None) is None # allow nulls + with pytest.raises(DagsterEvaluateValueError): + type_bar.evaluate_value('not_a_bar') diff --git a/python_modules/dagster/dagster/core/execution.py b/python_modules/dagster/dagster/core/execution.py index 5fb387023d2f0..ec59ef8754042 100644 --- a/python_modules/dagster/dagster/core/execution.py +++ b/python_modules/dagster/dagster/core/execution.py @@ -200,21 +200,21 @@ def reraise_user_error(self): ): if not result.success: if isinstance( - result.failure_data.dagster_user_exception, + result.failure_data.dagster_error, DagsterUserCodeExecutionError, ): - six.reraise(*result.failure_data.dagster_user_exception.original_exc_info) + six.reraise(*result.failure_data.dagster_error.original_exc_info) else: - raise result.failure_data.dagster_user_exception + raise result.failure_data.dagster_error @property - def dagster_user_exception(self): + def dagster_error(self): '''Returns exception that happened during this solid's execution, if any''' for result in itertools.chain( self.input_expectations, self.output_expectations, self.transforms ): if not result.success: - return result.failure_data.dagster_user_exception + return result.failure_data.dagster_error def _wrap_in_yield(context_or_generator): diff --git a/python_modules/dagster/dagster/core/types.py b/python_modules/dagster/dagster/core/types.py index 326e14f3fe491..0e433e30d53d2 100644 --- a/python_modules/dagster/dagster/core/types.py +++ b/python_modules/dagster/dagster/core/types.py @@ -1,4 +1,3 @@ -from collections import namedtuple from six import (string_types, integer_types) from dagster import check @@ -7,7 +6,8 @@ class DagsterType(object): - '''Base class for Dagster Type system. Should be inherited by a subclass. Subclass must implement `evaluate_value` + '''Base class for Dagster Type system. Should be inherited by a subclass. + Subclass must implement `evaluate_value` Attributes: name (str): Name of the type @@ -24,7 +24,11 @@ def __repr__(self): return 'DagsterType({name})'.format(name=self.name) def evaluate_value(self, _value): - '''Subclasses must implement this method. Check if the value is a valid one and return a processed version of it. If value is invalid, raise `DagsterEvaluateValueError`. + '''Subclasses can implement this method. Check if the value is a valid one + and return a processed version of it. If value is invalid, + raise `DagsterEvaluateValueError`. + + This class provides a default implementation of this method Args: value: The value to check @@ -35,33 +39,20 @@ def evaluate_value(self, _value): check.not_implemented('Must implement in subclass') -class DagsterScalarType(DagsterType): - '''Base class for dagster types that are scalar python values. - - Attributes: - name (str): Name of the type - - description (str): Description of the type +class UncoercedTypeMixin(object): + '''This is a helper mixin used when you only want to do a type check + against an in-memory value and then leave that value uncoerced. Only + is_python_valid_value must be implemented for these classes. + evaluate_value is implemented for you. ''' - def __init__(self, *args, **kwargs): - super(DagsterScalarType, self).__init__(*args, **kwargs) - - def process_value(self, value): - '''Modify the value before it's evaluated. Subclasses may override. - - Returns: - any: New value - ''' - return value - def is_python_valid_value(self, _value): '''Subclasses must implement this method. Check if the value and output a boolean. Returns: bool: Whether the value is valid. ''' - raise Exception('must implement') + check.failed('must implement') def evaluate_value(self, value): if not self.is_python_valid_value(value): @@ -73,7 +64,20 @@ def evaluate_value(self, value): return value -class _DagsterAnyType(DagsterType): +class DagsterScalarType(UncoercedTypeMixin, DagsterType): + '''Base class for dagster types that are scalar python values. + + Attributes: + name (str): Name of the type + + description (str): Description of the type + ''' + + def __init__(self, *args, **kwargs): + super(DagsterScalarType, self).__init__(*args, **kwargs) + + +class _DagsterAnyType(UncoercedTypeMixin, DagsterType): def __init__(self): super(_DagsterAnyType, self).__init__( name='Any', description='The type that allows any value, including no value.' @@ -82,18 +86,12 @@ def __init__(self): def is_python_valid_value(self, _value): return True - def process_value(self, value): - return value - - def evaluate_value(self, value): - return value - def nullable_isinstance(value, typez): return value is None or isinstance(value, typez) -class PythonObjectType(DagsterType): +class PythonObjectType(UncoercedTypeMixin, DagsterType): '''Dagster Type that checks if the value is an instance of some `python_type`''' def __init__( @@ -108,15 +106,6 @@ def __init__( def is_python_valid_value(self, value): return nullable_isinstance(value, self.python_type) - def evaluate_value(self, value): - if not self.is_python_valid_value(value): - raise DagsterEvaluateValueError( - 'Expected valid value for {type_name} but got {value}'.format( - type_name=self.name, value=repr(value) - ) - ) - return value - class _DagsterStringType(DagsterScalarType): def is_python_valid_value(self, value): diff --git a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_user_error.py b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_user_error.py index b4b20f3d872fe..86a8d890345f8 100644 --- a/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_user_error.py +++ b/python_modules/dagster/dagster/pandas/pandas_tests/test_pandas_user_error.py @@ -6,6 +6,8 @@ import dagster.pandas as dagster_pd from dagster import ( + DagsterInvariantViolationError, + DagsterTypeError, DependencyDefinition, InputDefinition, OutputDefinition, @@ -15,19 +17,9 @@ execute_pipeline, lambda_solid, ) -from dagster.core.errors import DagsterInvariantViolationError from dagster.core.utility_solids import define_stub_solid -def _dataframe_solid(name, inputs, transform_fn): - return SolidDefinition.single_output_transform( - name=name, - inputs=inputs, - transform_fn=transform_fn, - output=OutputDefinition(dagster_pd.DataFrame), - ) - - def test_wrong_output_value(): csv_input = InputDefinition('num_csv', dagster_pd.DataFrame) @@ -69,5 +61,5 @@ def df_solid(foo): }} ) - with pytest.raises(DagsterInvariantViolationError): + with pytest.raises(DagsterTypeError): execute_pipeline(pipeline) diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eleven/test_intro_tutorial_part_eleven.py b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eleven.py similarity index 100% rename from python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eleven/test_intro_tutorial_part_eleven.py rename to python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_eleven.py diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_ten/test_intro_tutorial_part_ten.py b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_ten.py similarity index 100% rename from python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_ten/test_intro_tutorial_part_ten.py rename to python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_ten.py diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_twelve.py b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_twelve.py new file mode 100644 index 0000000000000..ef97dc98377d9 --- /dev/null +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_twelve.py @@ -0,0 +1,159 @@ +# pylint: disable=W0622,W0614,W0401 +from collections import namedtuple +import re + +import pytest + +from dagster import * + +StringTuple = namedtuple('StringTuple', 'str_one str_two') + +StringTupleType = types.PythonObjectType( + 'StringTuple', + python_type=StringTuple, + description='A tuple of strings.', +) + +SSNString = namedtuple('SSNString', 'value') + + +class SSNStringTypeClass(types.DagsterType): + def __init__(self): + super(SSNStringTypeClass, self).__init__(name='SSNString') + + def evaluate_value(self, value): + if isinstance(value, SSNString): + return value + + if not isinstance(value, str): + raise DagsterEvaluateValueError( + '{value} is not a string. SSNStringType typecheck failed'.format(value=repr(value)) + ) + + if not re.match(r'^(\d\d\d)-(\d\d)-(\d\d\d\d)$', value): + raise DagsterEvaluateValueError( + '{value} did not match SSN regex'.format(value=repr(value)) + ) + + return SSNString(value) + + +SSNStringType = SSNStringTypeClass() + + +@lambda_solid +def produce_valid_value(): + return StringTuple(str_one='value_one', str_two='value_two') + + +@lambda_solid +def produce_invalid_value(): + return 'not_a_tuple' + + +@solid(inputs=[InputDefinition('string_tuple', StringTupleType)]) +def consume_string_tuple(info, string_tuple): + info.context.info('Logging value {string_tuple}'.format(string_tuple=string_tuple)) + + +@lambda_solid +def produce_valid_ssn_string(): + return '394-30-2032' + + +@lambda_solid +def produce_invalid_ssn_string(): + return '394-30-203239483' + + +@solid(inputs=[InputDefinition('ssn', SSNStringType)]) +def consume_ssn(info, ssn): + info.context.info('ssn: {ssn}'.format(ssn=ssn)) + + +def define_part_twelve_step_one(): + return PipelineDefinition( + name='part_twelve_step_one', + solids=[produce_valid_value, consume_string_tuple], + dependencies={ + 'consume_string_tuple': { + 'string_tuple': DependencyDefinition('produce_valid_value') + } + }, + ) + + +def define_part_twelve_step_two(): + return PipelineDefinition( + name='part_twelve_step_two', + solids=[produce_invalid_value, consume_string_tuple], + dependencies={ + 'consume_string_tuple': { + 'string_tuple': DependencyDefinition('produce_invalid_value') + } + }, + ) + + +def define_part_twelve_step_three(): + return PipelineDefinition( + name='part_twelve_step_three', + solids=[produce_valid_ssn_string, consume_ssn], + dependencies={'consume_ssn': { + 'ssn': DependencyDefinition('produce_valid_ssn_string') + }}, + ) + + +def define_part_twelve_step_four(): + return PipelineDefinition( + name='part_twelve_step_four', + solids=[produce_invalid_ssn_string, consume_ssn], + dependencies={'consume_ssn': { + 'ssn': DependencyDefinition('produce_invalid_ssn_string') + }}, + ) + + +def test_ssn_type(): + good_ssn_string = '123-43-4939' + good_ssn = SSNString(good_ssn_string) + assert SSNStringType.evaluate_value(good_ssn_string) == good_ssn + assert SSNStringType.evaluate_value(good_ssn) == good_ssn + + with pytest.raises(DagsterEvaluateValueError): + SSNStringType.evaluate_value(123) + + with pytest.raises(DagsterEvaluateValueError): + SSNStringType.evaluate_value(None) + + with pytest.raises(DagsterEvaluateValueError): + SSNStringType.evaluate_value('12932-9234892038-384') + + with pytest.raises(DagsterEvaluateValueError): + SSNStringType.evaluate_value('1292-34-383434') + + +def test_intro_tutorial_part_twelve_step_one(): + execute_pipeline(define_part_twelve_step_one()) + + +def test_intro_tutorial_part_twelve_step_two(): + with pytest.raises(DagsterTypeError): + execute_pipeline(define_part_twelve_step_two()) + + +def test_intro_tutorial_part_twelve_step_three(): + execute_pipeline(define_part_twelve_step_three()) + + +def test_intro_tutorial_part_twelve_step_four(): + with pytest.raises( + DagsterTypeError, + match='Solid consume_ssn input ssn received value 394-30-203239483 ', + ): + execute_pipeline(define_part_twelve_step_four()) + + +if __name__ == '__main__': + execute_pipeline(define_part_twelve_step_four(), throw_on_error=True) diff --git a/python_modules/dagster/docs/intro_tutorial/part_twelve.rst b/python_modules/dagster/docs/intro_tutorial/part_twelve.rst new file mode 100644 index 0000000000000..d7d02f0a6f234 --- /dev/null +++ b/python_modules/dagster/docs/intro_tutorial/part_twelve.rst @@ -0,0 +1,13 @@ +Custom Types +------------ + +So far we have only used the built-in types the come with dagster +to describe data flowing between different solids. However this +only gets one so far, and is typically only useful for toy +pipelines. You are going to want to define our own custom types +to describe your pipeline- and runtime-specific data structures. + +The most common operation will be to simply do a typecheck against +a particular python type. + +TODO: complete this tutorial \ No newline at end of file From c84e001c976b3e689d8d510d624383529c1c7b7b Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Mon, 17 Sep 2018 07:17:20 -0700 Subject: [PATCH 098/103] Rough draft of part 12 text --- .../test_intro_tutorial_part_twelve.py | 10 +- python_modules/dagster/docs/index.rst | 1 + .../docs/intro_tutorial/part_twelve.rst | 198 +++++++++++++++++- 3 files changed, 197 insertions(+), 12 deletions(-) diff --git a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_twelve.py b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_twelve.py index ef97dc98377d9..9f360e706800f 100644 --- a/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_twelve.py +++ b/python_modules/dagster/dagster_tests/tutorials/test_intro_tutorial_part_twelve.py @@ -14,7 +14,9 @@ description='A tuple of strings.', ) -SSNString = namedtuple('SSNString', 'value') + +class SSNString(str): + pass class SSNStringTypeClass(types.DagsterType): @@ -41,7 +43,7 @@ def evaluate_value(self, value): SSNStringType = SSNStringTypeClass() -@lambda_solid +@lambda_solid(output=OutputDefinition(StringTupleType)) def produce_valid_value(): return StringTuple(str_one='value_one', str_two='value_two') @@ -68,6 +70,8 @@ def produce_invalid_ssn_string(): @solid(inputs=[InputDefinition('ssn', SSNStringType)]) def consume_ssn(info, ssn): + if not isinstance(ssn, SSNString): + raise Exception('This should never be thrown') info.context.info('ssn: {ssn}'.format(ssn=ssn)) @@ -156,4 +160,4 @@ def test_intro_tutorial_part_twelve_step_four(): if __name__ == '__main__': - execute_pipeline(define_part_twelve_step_four(), throw_on_error=True) + execute_pipeline(define_part_twelve_step_three(), throw_on_error=True) diff --git a/python_modules/dagster/docs/index.rst b/python_modules/dagster/docs/index.rst index ec66ac17ce48f..4034ab39591dd 100644 --- a/python_modules/dagster/docs/index.rst +++ b/python_modules/dagster/docs/index.rst @@ -32,6 +32,7 @@ Intro Tutorial intro_tutorial/part_nine intro_tutorial/part_ten intro_tutorial/part_eleven + intro_tutorial/part_twelve API Reference diff --git a/python_modules/dagster/docs/intro_tutorial/part_twelve.rst b/python_modules/dagster/docs/intro_tutorial/part_twelve.rst index d7d02f0a6f234..f29bd443fbc1f 100644 --- a/python_modules/dagster/docs/intro_tutorial/part_twelve.rst +++ b/python_modules/dagster/docs/intro_tutorial/part_twelve.rst @@ -1,13 +1,193 @@ +User-defined Types +------------------ + +So far we have only used the built-in types the come with dagster to describe +data flowing between different solids. However this only gets one so far, and +is typically only useful for toy pipelines. You are going to want to define +our own custom types to describe your pipeline- and runtime-specific data +structures. + +For the first example, we'll show how to flow a plain python object +through the a pipeline and then describe that object in the type system. +Let's say we wanted to flow a tuple through a pipeline. + +.. code-block:: python + + StringTuple = namedtuple('StringTuple', 'str_one str_two') + + @lambda_solid + def produce_valid_value(): + return StringTuple(str_one='value_one', str_two='value_two') + + +And then we want to consume it. However, we want this to be type-checked +and metadata to be surface in tools like dagit. + +To do this we'll introduce a dagster type. + +.. code-block:: python + + StringTupleType = types.PythonObjectType( + 'StringTuple', + python_type=StringTuple, + description='A tuple of strings.', + ) + +And then annotate relevant functions with it. + +.. code-block:: python + + @lambda_solid(output=OutputDefinition(StringTupleType)) + def produce_valid_value(): + return StringTuple(str_one='value_one', str_two='value_two') + + @solid(inputs=[InputDefinition('string_tuple', StringTupleType)]) + def consume_string_tuple(info, string_tuple): + info.context.info( + 'Logging value {string_tuple}'.format( + string_tuple=string_tuple + ) + ) + + def define_part_twelve_step_one(): + return PipelineDefinition( + name='part_twelve_step_one', + solids=[produce_valid_value, consume_string_tuple], + dependencies={ + 'consume_string_tuple': { + 'string_tuple': + DependencyDefinition('produce_valid_value') + } + }, + ) + + if __name__ == '__main__': + execute_pipeline(define_part_twelve_step_()) + +.. code-block:: sh + + $ python part_twelve.py + ... log spew + 2018-09-17 06:55:06 - dagster - INFO - orig_message="Logging value StringTuple(str_one='value_one', str_two='value_two')" log_message_id="675f905d-c1f4-4539-af26-c28d23a757be" pipeline="part_twelve_step_one" solid="consume_string_tuple" + ... + +Now what if things go wrong? Imagine you made an error and wired up `consume_string_tuple` +to a solid + +.. code-block:: python + + @lambda_solid + def produce_invalid_value(): + return 'not_a_tuple' + + def define_part_twelve_step_two(): + return PipelineDefinition( + name='part_twelve_step_two', + solids=[produce_invalid_value, consume_string_tuple], + dependencies={ + 'consume_string_tuple': { + 'string_tuple': + DependencyDefinition('produce_invalid_value') + } + }, + ) + + if __name__ == '__main__': + execute_pipeline(define_part_twelve_step_two()) + +If you run this you'll get some helpful error messages + +.. code-block:: sh + + $ python part_twelve.py + ... log spew + 2018-09-17 07:00:11 - dagster - ERROR - orig_message="Solid consume_string_tuple input string_tuple received value not_a_tuple which does not pass the typecheck for Dagster type StringTuple. Compute node consume_string_tuple.transform" log_message_id="4070d30d-8b29-4130-bbd6-6049d40e742b" pipeline="part_twelve_step_two" + ... stack trace + dagster.core.errors.DagsterEvaluateValueError: Expected valid value for StringTuple but got 'not_a_tuple' + ... more stack trace + dagster.core.errors.DagsterTypeError: Solid consume_string_tuple input string_tuple received value not_a_tuple which does not pass the typecheck for Dagster type StringTuple. Compute node consume_string_tuple.transform + Custom Types ------------- +^^^^^^^^^^^^ + +The type system is very flexible, and values can by both type-checked and coerced by user-defined code. + +Imagine we wants to be able process social security numbers and ensure that they are well-formed +throughout the whole pipeline. + +In order to do this we'll define a type. + +.. code-block:: python + + class SSNString(str): + pass + + class SSNStringTypeClass(types.DagsterType): + def __init__(self): + super(SSNStringTypeClass, self).__init__(name='SSNString') + + def evaluate_value(self, value): + if isinstance(value, SSNString): + return value + + if not isinstance(value, str): + raise DagsterEvaluateValueError( + '{value} is not a string. SSNStringType typecheck failed'.format(value=repr(value)) + ) + + if not re.match(r'^(\d\d\d)-(\d\d)-(\d\d\d\d)$', value): + raise DagsterEvaluateValueError( + '{value} did not match SSN regex'.format(value=repr(value)) + ) + + return SSNString(value) + + + SSNStringType = SSNStringTypeClass() + +This type does a couple things. One is that ensures that any string that gets passed to +evaluate_value matches a strict regular expression. You'll also notice that it coerces +that incoming string type to a type called `SSNString`. This type just trivially inherits +from ``str``, but it signifies that the typecheck has already occured. That means if +evaluate_value is called again, the bulk of the typecheck can be short-circuited, saving +repeated processing through the pipeline. (Note: this is slightly silly because the amount +of computation here is trivial, but one can imagine types that require significant +amounts of computation to verify). + + +.. code-block:: python + + @lambda_solid + def produce_valid_ssn_string(): + return '394-30-2032' + + @solid(inputs=[InputDefinition('ssn', SSNStringType)]) + def consume_ssn(info, ssn): + if not isinstance(ssn, SSNString): + raise Exception('This should never be thrown') + info.context.info('ssn: {ssn}'.format(ssn=ssn)) + + def define_part_twelve_step_three(): + return PipelineDefinition( + name='part_twelve_step_three', + solids=[produce_valid_ssn_string, consume_ssn], + dependencies={ + 'consume_ssn': { + 'ssn': + DependencyDefinition('produce_valid_ssn_string') + } + }, + ) + + if __name__ == '__main__': + execute_pipeline(define_part_twelve_step_three()) -So far we have only used the built-in types the come with dagster -to describe data flowing between different solids. However this -only gets one so far, and is typically only useful for toy -pipelines. You are going to want to define our own custom types -to describe your pipeline- and runtime-specific data structures. +You'll note that the exception in ``consume_ssn`` was not thrown, meaning that the +str was coerced to an SSNString by the dagster type. -The most common operation will be to simply do a typecheck against -a particular python type. +Future Directions +^^^^^^^^^^^^^^^^^ -TODO: complete this tutorial \ No newline at end of file +1. Up-front type checking +2. Serializations +3. Hashing \ No newline at end of file From e8f4b9e4f372ce52b7f03e862b6321ceb0ac1531 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Mon, 17 Sep 2018 07:27:05 -0700 Subject: [PATCH 099/103] Cleanup top-level readme and redirect to readthedocs --- README.rst | 277 ++--------------------------------------------------- 1 file changed, 6 insertions(+), 271 deletions(-) diff --git a/README.rst b/README.rst index 45209101158bc..386f0e0d814b2 100644 --- a/README.rst +++ b/README.rst @@ -7,7 +7,11 @@ Introduction ============ -Dagster is an opinionated system and programming model for data pipelines. This process goes by many names -- ETL (extract-load-transform), ELT (extract-transform-load), model production, data integration, and so on -- but in essence they all describe the same activity: Performing a set of computations structured as a DAG (directed, acyclic graph) that end up producing data assets, whether those assets be tables, files, machine-learning models, etc. +Dagster is an opinionated system and programming model for data pipelines. This process goes by +many names -- ETL (extract-load-transform), ELT (extract-transform-load), model production, data +integration, and so on -- but in essence they all describe the same activity: Performing a set of +computations structured as a DAG (directed, acyclic graph) that end up producing data assets, +whether those assets be tables, files, machine-learning models, etc. There are a few tools in this repo @@ -17,273 +21,4 @@ This repo has a few primary components: * **Dagit**: Dagit is a rich viewer for Dagster assets. * **Dagster GE**: A Dagster integration with Great Expectations. (see https://github.com/great-expectations/great_expectations) -------------------- -Opinions and Values -------------------- - -As noted above Dagster has a point of view and values regarding how data pipelines should be built and structured. We list them in no particular order: - -* **Functional Data Pipelines**: We believe that data pipelines should be organized as DAGs of functional, idempotent computations. These computations injest input, do computation, and produce output, either with no side effects or well-known, un-externalized side effects. Given the same inputs and configuration, the computation should always produce the same output. These computations should also be parameterizable, so that they can execute in different environments. See https://bit.ly/2LxDgnr for an excellent overview of functional programing in batch computations. -* **Self-describing**: Pipelines should be self-describing with rich metadata and types. Users should be able to approach a pipeline, and use tooling to inspect the pipelines for their structure and capabilities. This metadata should be co-located with actual code of the pipeline. Documentation and code is delivered as a single artifact. -* **Compute-agnostic**: Dagster has opinions about the structure and best practices of data pipelines. It has no opinions about what libraries and engines use to do actual compute. The core computation within a dagster pipeline is user-specified code, meaning that it could be anything: Pandas, Spark, SQL computations on a data warehouse, vanilla python, or any combination therein. -* **Testable by design and by default**: Testing data pipelines is notoriously difficult. Because it is so difficult it is often never done, or done poorly. Dagster pipelines are designed to be tested. They have explicit support for pipeline authors to manage and maintain multiple operating environments -- for example, unit testing, integration testing, and production environments, among others. In addition dagster can execute arbitrary subsets and nodes of the pipeline, critical testability. (This capability happens to be useful in operational contexts as well). -* **First-class data quality tests**: Testing code is important in data pipelines, but it is not sufficient. Data quality tests -- run during every meaningful stage of production -- are critical to reduce the maintenance burden of data pipelines. Pipeline authors generally do not have control of their input data, and make many implicit assumptions about that data. The data formats can also change over time. In order to control this entropy, Dagster encourages users to computationally verify assumptions (known as expectations) about the data as part of the piplien process. This way if those assumptions are broken, the breakage can be reported quickly, easily, and with rich metadata and diagnostic information. These expectations can also serve as contracts between teams. See https://bit.ly/2mxDS1R for a primer on pipeline tests for data quality. -* **Gradual, optional typing**: Dagster contains a type system to describe the values flowing through the pipeline. This allows nodes in a pipeline know if they are compatible before execution, and serves as value documentation and runtime error checking. - - -The core abstraction in Dagster is a *solid*, a logical unit of computation in a data pipeline. At its core a solid is a configurable function that accepts abstract inputs and produces outputs. - -* Inputs: -* Outputs: -* Configuration: -* Transform: - -Inputs are things. - -* Name: -* Type: -* Expectations: - -Outputs are things: - -* Name -* Type -* Expectations - -Solids are group together in *pipelines*. Pipelines are comprised of: - -* Solids -* Dependencies -* Context Definitions - - - -Solid formally separates the notion of inputs, outputs, the core transform. The core goal of this is cleanly and clearly separate the domain logic of each transform and its operational environments. This allows solids or pipelines of solids to easily be executable in a number of environments: unit-testing, local development, integration tests, CI/CD, staging environments, production environments, and so on and so forth. - -Alongside with the core abstraction, Dagster provides helpers to create Solids that operate on Pandas dataframes and SQL databases. - -Example -------- - - - -.. code-block:: python - - - import pandas as pd - import dagster.core - from dagster.core.definitions import ( - InputDefinition, - OutputDefinition - ) - from dagster.core.decorators import ( - solid, - ) - - # Solids can be created by annotating transform function with - # a decorator - @solid( - # Solid inputs define arguments passed to transform function - inputs=[ - InputDefinition( - name='num', - - ) - ], - # Solid output determines what solid should return. - output=OutputDefinition(materializations=[ - dataframe_to_csv_materialization, - ]) - ) - def sum_solid(num): - sum_df = num.copy() - # Here we add a new column to dataframe to sum up num1 and - # num2 columns - sum_df['sum'] = sum_df['num1'] + sum_df['num2'] - return sum_df - - - @solid( - inputs=[ - InputDefinition(name="sum") - ], - output=OutputDefinition(materializations=[ - dataframe_to_csv_materialization, - ]) - ) - def sum_sq_solid(sum): - sum_sq = sum.copy() - sum_sq['sum_sq'] = sum['sum']**2 - return sum_sq - - - # After definining a solid, we are grouping them into a pipeline - pipeline = dagster.core.pipeline( - name='pandas_hello_world', - solids=[ - sum_solid, - sum_sq_solid, - ], - ) - -You might notice that there is no actual CSV file specified as inputs. This is because such parameters are passed in environment. This allows you to customize it in runtime. To run your solid, we'll pass that environment to the execution function. - -.. code-block:: python - - pipeline_result = dagster.execute_pipeline(pipeline, environment) - - -We can simplify the above example by using built-in dagster pandas inputs and outputs. - -.. code-block:: python - - # TODO updated example definition - -We can specify in order to get artifacts for the results. We can materialize output from any solid, this can be useful to see if intermediate results make sense. - -.. code-block:: python - - # TODO updated example config driving - -Dagster CLI -=========== - -In addition to programmatic API, you can also use dagster CLI to run the pipelines. In that case the environment is specified through yaml configuration files. - -The folder structure should be as follows. - -.. code-block - - pipeline_project_name/ - pipelines.yml - pipeline_module_1/ - env.yml - pipeline_module_2/ - env.yml - -Pipelines yml specify the pipelines that are present in current project. Env specifies environment for each particular pipeline. - -.. code-block:: yaml - - # TODO pipelines file - -.. code-block:: yaml - - # TODO example config file - - -.. code-block:: sh - -# TODO example CLI driving - - -Concepts -======== - -Transform ---------- - -This is core, user-defined transform that performs the logical data -computation. In this case the transform is ``hello_world_transform_fn`` -and it is passed as parameter into the solid. It takes one or more -inputs and produces an output. All other concepts in a solid are the -metadata and structure that surround this core computation - -Inputs ---------- - -For each argument to the transform function, there is one -``InputDefinition`` object. It has a name, which must match the -parameters to the transform function. The input definitions define a -name, a dependency for the input (what upstream solid produces its -value, see below) and a number of sources. An input definition must -specify at least a dependency or a source. The input can have any number -of sources. - - -Output ---------- - -The ``OutputDefinition`` represents the output of the transform -function. - - - -Higher-level APIs ------------------- - -# TODO keep this section? - -Execution ---------- - -These are useless without being able to execute them. In order to -execute a solid, you need to package it up into a pipeline. - -.. code-block:: python - - pipeline = dagster.PipelineDefinition(name='hello_world', solids=[sum_solid]) - -Then you an execute it by providing an environment. You must provide -enough source data to create all the inputs necessary for the pipeline. - -.. code-block:: python - - environment = config.Environment( - sources={ - 'sum' : { - 'num_df' : config.Source(name='CSV', args={'path': 'path/to/input.csv'}) - } - } - ) - - pipeline_result = dagster.execute_pipeline( - dagster.ExecutionContext(), - pipeline, - environment - ) - - print(pipeline_result.result_for_solid('sum').transformed_value) - - -Dependencies ------------- - -So far we have demonstrated a single stage pipeline, which is obviously -of limited value. - -Imagine we wanted to add another stage which took the sum we produced -and squared that value. (Fancy!) - -.. code-block:: python - - # TODO example dependencies - -Note that input specifies that dependency. This means that the input -value passed to the transform can be generated by an upstream dependency -OR by an external source. This allows for the solid to be executable in -isolation or in the context of a pipeline. - -.. code-block:: python - - # TODO example driving - -The above executed both solids, even though one input was provided. The -input into sum\_sq\_solid was provided by the upstream result from the -output of sum\_solid. - -You can also execute subsets of the pipeline. Given the above pipeline, -you could specify that you only want to specify the first solid: - -.. code-block:: python - - # TODO example subdag execution - - - -Expectations ------------- - -Expectations are another reason to introduce logical seams between data -computations. They are a way to perform data quality tests or -statistical process control on data pipelines. +Go to https://dagster.readthedocs.io/en/latest/ for documentation! \ No newline at end of file From 89d1c856c19cf30b5b51b709d02ac59e6fa60506 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Mon, 17 Sep 2018 07:36:32 -0700 Subject: [PATCH 100/103] Version bump --- python_modules/dagit/setup.py | 4 ++-- python_modules/dagster-ge/setup.py | 2 +- python_modules/dagster/dagster/sqlalchemy/common.py | 2 -- python_modules/dagster/dagster/version.py | 2 +- 4 files changed, 4 insertions(+), 6 deletions(-) diff --git a/python_modules/dagit/setup.py b/python_modules/dagit/setup.py index 81a9f7cd9f191..1d84796dc2348 100644 --- a/python_modules/dagit/setup.py +++ b/python_modules/dagit/setup.py @@ -18,7 +18,7 @@ def long_description(): version = {} with open("dagit/version.py") as fp: - exec(fp.read(), version) + exec(fp.read(), version) # pylint: disable=W0122 setup( name='dagit', @@ -55,7 +55,7 @@ def long_description(): 'click>=6.7', # dagster - 'dagster>=0.2.0.dev3', + 'dagster>=0.2.0', # graphql 'graphql-core>=2.1', diff --git a/python_modules/dagster-ge/setup.py b/python_modules/dagster-ge/setup.py index 1766922eb55aa..d2fbb5d99f9a2 100644 --- a/python_modules/dagster-ge/setup.py +++ b/python_modules/dagster-ge/setup.py @@ -26,7 +26,7 @@ # standard python 2/3 compatability things 'enum34>=1.1.6', 'future>=0.16.0', - 'dagster>=0.1.2', + 'dagster>=0.2.0', 'great-expectations>=0.4.2', ] # scripts=['bin/dagster'] diff --git a/python_modules/dagster/dagster/sqlalchemy/common.py b/python_modules/dagster/dagster/sqlalchemy/common.py index 231d3fa0f4045..787161376b229 100644 --- a/python_modules/dagster/dagster/sqlalchemy/common.py +++ b/python_modules/dagster/dagster/sqlalchemy/common.py @@ -2,8 +2,6 @@ from dagster import ( ExecutionContext, - Result, - SolidDefinition, check, ) diff --git a/python_modules/dagster/dagster/version.py b/python_modules/dagster/dagster/version.py index 8db444e674a2c..7fd229a32b5ee 100644 --- a/python_modules/dagster/dagster/version.py +++ b/python_modules/dagster/dagster/version.py @@ -1 +1 @@ -__version__ = '0.2.0.dev13' +__version__ = '0.2.0' From ed9907ff31e314b299c9d69eebc83ac51e264748 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Mon, 17 Sep 2018 07:44:49 -0700 Subject: [PATCH 101/103] Some minor setup.py cleanup --- python_modules/dagit/setup.py | 3 +-- python_modules/dagster/setup.py | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/python_modules/dagit/setup.py b/python_modules/dagit/setup.py index 1d84796dc2348..9d9d6091b3975 100644 --- a/python_modules/dagit/setup.py +++ b/python_modules/dagit/setup.py @@ -24,9 +24,8 @@ def long_description(): name='dagit', version=version['__version__'], author='Elementl', - author_email='schrockn@elementl.com', license='Apache-2.0', - description='Web UI for dagster, an opinionated pipeline runner.', + description='Web UI for dagster.', long_description=long_description(), long_description_content_type='text/markdown', url='https://github.com/dagster-io/dagster', diff --git a/python_modules/dagster/setup.py b/python_modules/dagster/setup.py index c27738edb16f0..eb84b542b9407 100644 --- a/python_modules/dagster/setup.py +++ b/python_modules/dagster/setup.py @@ -24,9 +24,8 @@ def long_description(): name='dagster', version=version['__version__'], author='Elementl', - author_email='schrockn@elementl.com', license='Apache-2.0', - description='Dagster is an opinionated pipeline runner.', + description='Dagster is an opinionated programming model for data pipelines.', long_description=long_description(), long_description_content_type='text/markdown', url='https://github.com/dagster-io/dagster', From e7cfe1ac3f558dea3d74f3bc245627e951a26d35 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Mon, 17 Sep 2018 07:50:03 -0700 Subject: [PATCH 102/103] update dagit and dagster-ge setup --- python_modules/dagit/dagit/version.py | 2 +- python_modules/dagster-ge/setup.py | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/python_modules/dagit/dagit/version.py b/python_modules/dagit/dagit/version.py index efbf9fb8ea7b5..7fd229a32b5ee 100644 --- a/python_modules/dagit/dagit/version.py +++ b/python_modules/dagit/dagit/version.py @@ -1 +1 @@ -__version__ = '0.2.0.dev3' +__version__ = '0.2.0' diff --git a/python_modules/dagster-ge/setup.py b/python_modules/dagster-ge/setup.py index d2fbb5d99f9a2..4c524e341cf63 100644 --- a/python_modules/dagster-ge/setup.py +++ b/python_modules/dagster-ge/setup.py @@ -10,9 +10,8 @@ setup( name='dagster-ge', - version='0.0.0', + version='0.2.0', author='Elementl', - author_email='schrockn@elementl.com', license='Apache-2.0', description='Great Expectations plugin for Dagster', url='https://github.com/dagster-io/dagster', From d9080eac5a1821a24db56aadfcbd19aa0f5fffe4 Mon Sep 17 00:00:00 2001 From: Nicholas Schrock Date: Mon, 17 Sep 2018 07:52:37 -0700 Subject: [PATCH 103/103] bump dagit version --- python_modules/dagit/dagit/version.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python_modules/dagit/dagit/version.py b/python_modules/dagit/dagit/version.py index 7fd229a32b5ee..fc79d63d5430b 100644 --- a/python_modules/dagit/dagit/version.py +++ b/python_modules/dagit/dagit/version.py @@ -1 +1 @@ -__version__ = '0.2.0' +__version__ = '0.2.1'