def execute_solids_within_pipeline( pipeline_def, solid_names, inputs=None, run_config=None, mode=None, preset=None, tags=None, instance=None, environment_dict=None, ): '''Execute a set of solids within an existing pipeline. Intended to support tests. Input values may be passed directly. Args: pipeline_def (PipelineDefinition): The pipeline within which to execute the solid. solid_names (FrozenSet[str]): A set of the solid names, or the aliased solids, to execute. inputs (Optional[Dict[str, Dict[str, Any]]]): A dict keyed on solid names, whose values are dicts of input names to input values, used to pass input values to the solids directly. You may also use the ``run_config`` to configure any inputs that are configurable. run_config (Optional[dict]): The environment configuration that parameterized this execution, as a dict. mode (Optional[str]): The name of the pipeline mode to use. You may not set both ``mode`` and ``preset``. preset (Optional[str]): The name of the pipeline preset to use. You may not set both ``mode`` and ``preset``. tags (Optional[Dict[str, Any]]): Arbitrary key-value pairs that will be added to pipeline logs. instance (Optional[DagsterInstance]): The instance to execute against. If this is ``None``, an ephemeral instance will be used, and no artifacts will be persisted from the run. Returns: Dict[str, Union[CompositeSolidExecutionResult, SolidExecutionResult]]: The results of executing the solids, keyed by solid name. ''' check.inst_param(pipeline_def, 'pipeline_def', PipelineDefinition) check.set_param(solid_names, 'solid_names', of_type=str) inputs = check.opt_dict_param(inputs, 'inputs', key_type=str, value_type=dict) # backcompact run_config = canonicalize_run_config(run_config, environment_dict) sub_pipeline = pipeline_def.get_pipeline_subset_def(solid_names) stubbed_pipeline = build_pipeline_with_input_stubs(sub_pipeline, inputs) result = execute_pipeline( stubbed_pipeline, run_config=run_config, mode=mode, preset=preset, tags=tags, instance=instance, ) return {sr.solid.name: sr for sr in result.solid_result_list}
def __new__(cls, name, run_config=None, solid_selection=None, mode=None, environment_dict=None): run_config = canonicalize_run_config(run_config, environment_dict) return super(PresetDefinition, cls).__new__( cls, name=check.str_param(name, 'name'), run_config=run_config, solid_selection=check.opt_nullable_list_param( solid_selection, 'solid_selection', of_type=str ), mode=check.opt_str_param(mode, 'mode', DEFAULT_MODE_NAME), )
def execute_solid_within_pipeline( pipeline_def, solid_name, inputs=None, run_config=None, mode=None, preset=None, tags=None, instance=None, environment_dict=None, ): '''Execute a single solid within an existing pipeline. Intended to support tests. Input values may be passed directly. Args: pipeline_def (PipelineDefinition): The pipeline within which to execute the solid. solid_name (str): The name of the solid, or the aliased solid, to execute. inputs (Optional[Dict[str, Any]]): A dict of input names to input values, used to pass input values to the solid directly. You may also use the ``run_config`` to configure any inputs that are configurable. run_config (Optional[dict]): The environment configuration that parameterized this execution, as a dict. mode (Optional[str]): The name of the pipeline mode to use. You may not set both ``mode`` and ``preset``. preset (Optional[str]): The name of the pipeline preset to use. You may not set both ``mode`` and ``preset``. tags (Optional[Dict[str, Any]]): Arbitrary key-value pairs that will be added to pipeline logs. instance (Optional[DagsterInstance]): The instance to execute against. If this is ``None``, an ephemeral instance will be used, and no artifacts will be persisted from the run. Returns: Union[CompositeSolidExecutionResult, SolidExecutionResult]: The result of executing the solid. ''' # backcompact run_config = canonicalize_run_config(run_config, environment_dict) return execute_solids_within_pipeline( pipeline_def, solid_names={solid_name}, inputs={solid_name: inputs} if inputs else None, run_config=run_config, mode=mode, preset=preset, tags=tags, instance=instance, )[solid_name]
def __new__( cls, pipeline_name, task_id, recon_repo=None, run_config=None, mode=None, step_keys=None, dag=None, instance_ref=None, op_kwargs=None, pipeline_snapshot=None, execution_plan_snapshot=None, parent_pipeline_snapshot=None, environment_dict=None, ): run_config = canonicalize_run_config(run_config, environment_dict) # backcompact check_storage_specified(run_config) return super(DagsterOperatorParameters, cls).__new__( cls, recon_repo=check.opt_inst_param(recon_repo, 'recon_repo', ReconstructableRepository), pipeline_name=check.str_param(pipeline_name, 'pipeline_name'), run_config=check.opt_dict_param(run_config, 'run_config', key_type=str), mode=check.opt_str_param(mode, 'mode'), task_id=check.str_param(task_id, 'task_id'), step_keys=check.opt_list_param(step_keys, 'step_keys', of_type=str), dag=check.opt_inst_param(dag, 'dag', DAG), instance_ref=check.opt_inst_param(instance_ref, 'instance_ref', InstanceRef), op_kwargs=check.opt_dict_param(op_kwargs.copy(), 'op_kwargs', key_type=str), pipeline_snapshot=check.inst_param(pipeline_snapshot, 'pipeline_snapshot', PipelineSnapshot), execution_plan_snapshot=check.inst_param( execution_plan_snapshot, 'execution_plan_snapshot', ExecutionPlanSnapshot), parent_pipeline_snapshot=check.opt_inst_param( parent_pipeline_snapshot, 'parent_pipeline_snapshot', PipelineSnapshot), )
def _logged_execute_pipeline( pipeline, run_config=None, mode=None, preset=None, tags=None, solid_selection=None, instance=None, raise_on_error=True, environment_dict=None, ): # stack level is to punch through helper functions and telemetry wrapper run_config = canonicalize_run_config(run_config, environment_dict, stacklevel=7) ( pipeline, run_config, instance, mode, tags, solids_to_execute, solid_selection, ) = _check_execute_pipeline_args( pipeline=pipeline, run_config=run_config, mode=mode, preset=preset, tags=tags, solid_selection=solid_selection, instance=instance, ) log_repo_stats(instance=instance, pipeline=pipeline, source='execute_pipeline') pipeline_run = instance.create_run_for_pipeline( pipeline_def=pipeline.get_definition(), run_config=run_config, mode=mode, solid_selection=solid_selection, solids_to_execute=solids_to_execute, tags=tags, ) return execute_run(pipeline, pipeline_run, instance, raise_on_error=raise_on_error)
def make_airflow_dag_containerized( module_name, pipeline_name, image, run_config=None, mode=None, dag_id=None, dag_description=None, dag_kwargs=None, op_kwargs=None, environment_dict=None, ): '''Construct a containerized Airflow DAG corresponding to a given Dagster pipeline. Tasks in the resulting DAG will execute the Dagster logic they encapsulate by calling the dagster-graphql API exposed by a container run using a subclass of :py:class:`DockerOperator <airflow:airflow.operators.docker_operator.DockerOperator>`. As a consequence, both dagster, any Python dependencies required by your solid logic, and the module containing your pipeline definition must be available in the container spun up by this operator. Typically you'll want to install these requirements onto the image you're using. This function should be invoked in an Airflow DAG definition file, such as that created by an invocation of the dagster-airflow scaffold CLI tool. Args: module_name (str): The name of the importable module in which the pipeline definition can be found. pipeline_name (str): The name of the pipeline definition. image (str): The name of the Docker image to use for execution (passed through to :py:class:`DockerOperator <airflow:airflow.operators.docker_operator.DockerOperator>`). run_config (Optional[dict]): The environment config, if any, with which to compile the pipeline to an execution plan, as a Python dict. mode (Optional[str]): The mode in which to execute the pipeline. instance (Optional[DagsterInstance]): The Dagster instance to use to execute the pipeline. dag_id (Optional[str]): The id to use for the compiled Airflow DAG (passed through to :py:class:`DAG <airflow:airflow.models.DAG>`). dag_description (Optional[str]): The description to use for the compiled Airflow DAG (passed through to :py:class:`DAG <airflow:airflow.models.DAG>`) dag_kwargs (Optional[dict]): Any additional kwargs to pass to the Airflow :py:class:`DAG <airflow:airflow.models.DAG>` constructor, including ``default_args``. op_kwargs (Optional[dict]): Any additional kwargs to pass to the underlying Airflow operator (a subclass of :py:class:`DockerOperator <airflow:airflow.operators.docker_operator.DockerOperator>`). Returns: (airflow.models.DAG, List[airflow.models.BaseOperator]): The generated Airflow DAG, and a list of its constituent tasks. ''' check.str_param(module_name, 'module_name') recon_repo = ReconstructableRepository.for_module(module_name, pipeline_name) run_config = canonicalize_run_config(run_config, environment_dict) # backcompact op_kwargs = check.opt_dict_param(op_kwargs, 'op_kwargs', key_type=str) op_kwargs['image'] = image return _make_airflow_dag( recon_repo=recon_repo, pipeline_name=pipeline_name, run_config=run_config, mode=mode, dag_id=dag_id, dag_description=dag_description, dag_kwargs=dag_kwargs, op_kwargs=op_kwargs, operator=DagsterDockerOperator, )
def make_airflow_dag_for_operator( recon_repo, pipeline_name, operator, run_config=None, mode=None, dag_id=None, dag_description=None, dag_kwargs=None, op_kwargs=None, environment_dict=None, ): '''Construct an Airflow DAG corresponding to a given Dagster pipeline and custom operator. `Custom operator template <https://github.com/dagster-io/dagster/blob/master/examples/legacy_examples/dagster_examples/dagster_airflow/custom_operator.py>`_ Tasks in the resulting DAG will execute the Dagster logic they encapsulate run by the given Operator :py:class:`BaseOperator <airflow.models.BaseOperator>`. If you are looking for a containerized solution to provide better isolation, see instead :py:func:`make_airflow_dag_containerized`. This function should be invoked in an Airflow DAG definition file, such as that created by an invocation of the dagster-airflow scaffold CLI tool. Args: recon_repo (:class:`dagster.ReconstructableRepository`): reference to a Dagster RepositoryDefinition that can be reconstructed in another process pipeline_name (str): The name of the pipeline definition. operator (type): The operator to use. Must be a class that inherits from :py:class:`BaseOperator <airflow.models.BaseOperator>` run_config (Optional[dict]): The environment config, if any, with which to compile the pipeline to an execution plan, as a Python dict. mode (Optional[str]): The mode in which to execute the pipeline. instance (Optional[DagsterInstance]): The Dagster instance to use to execute the pipeline. dag_id (Optional[str]): The id to use for the compiled Airflow DAG (passed through to :py:class:`DAG <airflow:airflow.models.DAG>`). dag_description (Optional[str]): The description to use for the compiled Airflow DAG (passed through to :py:class:`DAG <airflow:airflow.models.DAG>`) dag_kwargs (Optional[dict]): Any additional kwargs to pass to the Airflow :py:class:`DAG <airflow:airflow.models.DAG>` constructor, including ``default_args``. op_kwargs (Optional[dict]): Any additional kwargs to pass to the underlying Airflow operator. Returns: (airflow.models.DAG, List[airflow.models.BaseOperator]): The generated Airflow DAG, and a list of its constituent tasks. ''' check.subclass_param(operator, 'operator', BaseOperator) run_config = canonicalize_run_config(run_config, environment_dict) # backcompact return _make_airflow_dag( recon_repo=recon_repo, pipeline_name=pipeline_name, run_config=run_config, mode=mode, dag_id=dag_id, dag_description=dag_description, dag_kwargs=dag_kwargs, op_kwargs=op_kwargs, operator=operator, )
def make_airflow_dag( module_name, pipeline_name, run_config=None, mode=None, instance=None, dag_id=None, dag_description=None, dag_kwargs=None, op_kwargs=None, environment_dict=None, ): '''Construct an Airflow DAG corresponding to a given Dagster pipeline. Tasks in the resulting DAG will execute the Dagster logic they encapsulate as a Python callable, run by an underlying :py:class:`PythonOperator <airflow:PythonOperator>`. As a consequence, both dagster, any Python dependencies required by your solid logic, and the module containing your pipeline definition must be available in the Python environment within which your Airflow tasks execute. If you cannot install requirements into this environment, or you are looking for a containerized solution to provide better isolation, see instead :py:func:`make_airflow_dag_containerized`. This function should be invoked in an Airflow DAG definition file, such as that created by an invocation of the dagster-airflow scaffold CLI tool. Args: module_name (str): The name of the importable module in which the pipeline definition can be found. pipeline_name (str): The name of the pipeline definition. run_config (Optional[dict]): The environment config, if any, with which to compile the pipeline to an execution plan, as a Python dict. mode (Optional[str]): The mode in which to execute the pipeline. instance (Optional[DagsterInstance]): The Dagster instance to use to execute the pipeline. dag_id (Optional[str]): The id to use for the compiled Airflow DAG (passed through to :py:class:`DAG <airflow:airflow.models.DAG>`). dag_description (Optional[str]): The description to use for the compiled Airflow DAG (passed through to :py:class:`DAG <airflow:airflow.models.DAG>`) dag_kwargs (Optional[dict]): Any additional kwargs to pass to the Airflow :py:class:`DAG <airflow:airflow.models.DAG>` constructor, including ``default_args``. op_kwargs (Optional[dict]): Any additional kwargs to pass to the underlying Airflow operator (a subclass of :py:class:`PythonOperator <airflow:airflow.operators.python_operator.PythonOperator>`). Returns: (airflow.models.DAG, List[airflow.models.BaseOperator]): The generated Airflow DAG, and a list of its constituent tasks. ''' check.str_param(module_name, 'module_name') recon_repo = ReconstructableRepository.for_module(module_name, pipeline_name) run_config = canonicalize_run_config(run_config, environment_dict) # backcompact return _make_airflow_dag( recon_repo=recon_repo, pipeline_name=pipeline_name, run_config=run_config, mode=mode, instance=instance, dag_id=dag_id, dag_description=dag_description, dag_kwargs=dag_kwargs, op_kwargs=op_kwargs, )
def execute_pipeline( pipeline, run_config=None, mode=None, preset=None, tags=None, solid_selection=None, instance=None, raise_on_error=True, environment_dict=None, ): '''Execute a pipeline synchronously. Users will typically call this API when testing pipeline execution, or running standalone scripts. Parameters: pipeline (Union[ExecutablePipeline, PipelineDefinition]): The pipeline to execute. environment_dict (Optional[dict]): The environment configuration that parametrizes this run, as a dict. mode (Optional[str]): The name of the pipeline mode to use. You may not set both ``mode`` and ``preset``. preset (Optional[str]): The name of the pipeline preset to use. You may not set both ``mode`` and ``preset``. tags (Optional[Dict[str, Any]]): Arbitrary key-value pairs that will be added to pipeline logs. instance (Optional[DagsterInstance]): The instance to execute against. If this is ``None``, an ephemeral instance will be used, and no artifacts will be persisted from the run. raise_on_error (Optional[bool]): Whether or not to raise exceptions when they occur. Defaults to ``True``, since this is the most useful behavior in test. solid_selection (Optional[List[str]]): A list of solid selection queries (including single solid names) to execute. For example: - ['some_solid']: select "some_solid" itself. - ['*some_solid']: select "some_solid" and all its ancestors (upstream dependencies). - ['*some_solid+++']: select "some_solid", all its ancestors, and its descendants (downstream dependencies) within 3 levels down. - ['*some_solid', 'other_solid_a', 'other_solid_b+']: select "some_solid" and all its ancestors, "other_solid_a" itself, and "other_solid_b" and its direct child solids. Returns: :py:class:`PipelineExecutionResult`: The result of pipeline execution. For the asynchronous version, see :py:func:`execute_pipeline_iterator`. This is the entrypoint for dagster CLI execution. For the dagster-graphql entrypoint, see ``dagster.core.execution.api.execute_plan()``. ''' # stack level is to punch through helper function and telemetry wrapper environment_dict = canonicalize_run_config(run_config, environment_dict, stacklevel=5) ( pipeline, environment_dict, instance, mode, tags, solids_to_execute, solid_selection, ) = _check_execute_pipeline_args( pipeline=pipeline, environment_dict=environment_dict, mode=mode, preset=preset, tags=tags, solid_selection=solid_selection, instance=instance, ) log_repo_stats(instance=instance, pipeline=pipeline, source='execute_pipeline') pipeline_run = instance.create_run_for_pipeline( pipeline_def=pipeline.get_definition(), environment_dict=environment_dict, mode=mode, solid_selection=solid_selection, solids_to_execute=solids_to_execute, tags=tags, ) return execute_run(pipeline, pipeline_run, instance, raise_on_error=raise_on_error)
def execute_pipeline_iterator( pipeline, run_config=None, mode=None, preset=None, tags=None, solid_selection=None, instance=None, environment_dict=None, ): '''Execute a pipeline iteratively. Rather than package up the result of running a pipeline into a single object, like :py:func:`execute_pipeline`, this function yields the stream of events resulting from pipeline execution. This is intended to allow the caller to handle these events on a streaming basis in whatever way is appropriate. Parameters: pipeline (Union[ExecutablePipeline, PipelineDefinition]): The pipeline to execute. environment_dict (Optional[dict]): The environment configuration that parametrizes this run, as a dict. mode (Optional[str]): The name of the pipeline mode to use. You may not set both ``mode`` and ``preset``. preset (Optional[str]): The name of the pipeline preset to use. You may not set both ``mode`` and ``preset``. tags (Optional[Dict[str, Any]]): Arbitrary key-value pairs that will be added to pipeline logs. solid_selection (Optional[List[str]]): A list of solid selection queries (including single solid names) to execute. For example: - ['some_solid']: select "some_solid" itself. - ['*some_solid']: select "some_solid" and all its ancestors (upstream dependencies). - ['*some_solid+++']: select "some_solid", all its ancestors, and its descendants (downstream dependencies) within 3 levels down. - ['*some_solid', 'other_solid_a', 'other_solid_b+']: select "some_solid" and all its ancestors, "other_solid_a" itself, and "other_solid_b" and its direct child solids. instance (Optional[DagsterInstance]): The instance to execute against. If this is ``None``, an ephemeral instance will be used, and no artifacts will be persisted from the run. Returns: Iterator[DagsterEvent]: The stream of events resulting from pipeline execution. ''' # stack level is to punch through helper function environment_dict = canonicalize_run_config(run_config, environment_dict, stacklevel=4) ( pipeline, environment_dict, instance, mode, tags, solids_to_execute, solid_selection, ) = _check_execute_pipeline_args( pipeline=pipeline, environment_dict=environment_dict, mode=mode, preset=preset, tags=tags, solid_selection=solid_selection, instance=instance, ) pipeline_run = instance.create_run_for_pipeline( pipeline_def=pipeline.get_definition(), environment_dict=environment_dict, mode=mode, solid_selection=solid_selection, solids_to_execute=solids_to_execute, tags=tags, ) return execute_run_iterator(pipeline, pipeline_run, instance)
def execute_solid( solid_def, mode_def=None, input_values=None, tags=None, run_config=None, raise_on_error=True, environment_dict=None, ): '''Execute a single solid in an ephemeral pipeline. Intended to support unit tests. Input values may be passed directly, and no pipeline need be specified -- an ephemeral pipeline will be constructed. Args: solid_def (SolidDefinition): The solid to execute. mode_def (Optional[ModeDefinition]): The mode within which to execute the solid. Use this if, e.g., custom resources, loggers, or executors are desired. input_values (Optional[Dict[str, Any]]): A dict of input names to input values, used to pass inputs to the solid directly. You may also use the ``run_config`` to configure any inputs that are configurable. tags (Optional[Dict[str, Any]]): Arbitrary key-value pairs that will be added to pipeline logs. run_config (Optional[dict]): The environment configuration that parameterized this execution, as a dict. raise_on_error (Optional[bool]): Whether or not to raise exceptions when they occur. Defaults to ``True``, since this is the most useful behavior in test. Returns: Union[CompositeSolidExecutionResult, SolidExecutionResult]: The result of executing the solid. ''' check.inst_param(solid_def, 'solid_def', ISolidDefinition) check.opt_inst_param(mode_def, 'mode_def', ModeDefinition) input_values = check.opt_dict_param(input_values, 'input_values', key_type=str) # backcompact run_config = canonicalize_run_config(run_config, environment_dict) solid_defs = [solid_def] def create_value_solid(input_name, input_value): @lambda_solid(name=input_name) def input_solid(): return input_value return input_solid dependencies = defaultdict(dict) for input_name, input_value in input_values.items(): dependencies[solid_def.name][input_name] = DependencyDefinition(input_name) solid_defs.append(create_value_solid(input_name, input_value)) result = execute_pipeline( PipelineDefinition( name='ephemeral_{}_solid_pipeline'.format(solid_def.name), solid_defs=solid_defs, dependencies=dependencies, mode_defs=[mode_def] if mode_def else None, ), run_config=run_config, mode=mode_def.name if mode_def else None, tags=tags, raise_on_error=raise_on_error, ) return result.result_for_handle(solid_def.name)