Skip to content

Commit

Permalink
Export build_reconstructable_target to match cragified docs (#6317)
Browse files Browse the repository at this point in the history
* change crag-ify build_reconstructable_target docs

* rebuild sphinx

* rename from build_reconstructable_target => build_reconstructable_job

* isort

* update docs snapshots
  • Loading branch information
prha committed Feb 18, 2022
1 parent 19b106c commit 47e8533
Show file tree
Hide file tree
Showing 9 changed files with 51 additions and 42 deletions.
2 changes: 1 addition & 1 deletion docs/content/api/modules.json

Large diffs are not rendered by default.

2 changes: 1 addition & 1 deletion docs/content/api/searchindex.json

Large diffs are not rendered by default.

2 changes: 1 addition & 1 deletion docs/content/api/sections.json

Large diffs are not rendered by default.

Binary file modified docs/next/public/objects.inv
Binary file not shown.
9 changes: 8 additions & 1 deletion docs/sphinx/sections/api/apidocs/jobs.rst
Original file line number Diff line number Diff line change
Expand Up @@ -10,4 +10,11 @@ Jobs are created by calling :py:meth:`GraphDefinition.to_job` on a graph instanc
.. autodecorator:: job

.. autoclass:: JobDefinition
:members:
:members:

Reconstructable jobs
-------------------------
.. autoclass:: reconstructable
:noindex:

.. autofunction:: build_reconstructable_job
2 changes: 0 additions & 2 deletions docs/sphinx/sections/api/apidocs/pipeline.rst
Original file line number Diff line number Diff line change
Expand Up @@ -46,8 +46,6 @@ Reconstructable pipelines

.. currentmodule:: dagster.core.definitions.reconstructable

.. autofunction:: build_reconstructable_pipeline

.. autoclass:: ReconstructablePipeline
:members:

Expand Down
2 changes: 2 additions & 0 deletions python_modules/dagster/dagster/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -101,6 +101,7 @@
WeeklyPartitionsDefinition,
asset_sensor,
build_init_logger_context,
build_reconstructable_job,
build_schedule_from_partitioned_job,
composite_solid,
config_mapping,
Expand Down Expand Up @@ -418,6 +419,7 @@
"in_process_executor",
"multiprocess_executor",
"multiple_process_executor_requirements",
"build_reconstructable_job",
"reconstructable",
"reexecute_pipeline_iterator",
"reexecute_pipeline",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -105,6 +105,7 @@
from .preset import PresetDefinition
from .reconstructable import (
ReconstructablePipeline,
build_reconstructable_job,
build_reconstructable_pipeline,
reconstructable,
)
Expand Down
73 changes: 37 additions & 36 deletions python_modules/dagster/dagster/core/definitions/reconstructable.py
Original file line number Diff line number Diff line change
Expand Up @@ -307,8 +307,8 @@ def define_my_job():
call), or in interactive environments such as the Python REPL or Jupyter notebooks.
If you need to reconstruct objects constructed in these ways, you should use
:py:func:`~dagster.core.definitions.reconstructable.build_reconstructable_pipeline` instead,
which allows you to specify your own reconstruction strategy.
:py:func:`~dagster.reconstructable.build_reconstructable_job` instead, which allows you to
specify your own reconstruction strategy.
Examples:
Expand Down Expand Up @@ -353,15 +353,15 @@ def make_bar_job():
raise DagsterInvariantViolationError(
"Reconstructable target can not be a lambda. Use a function or "
"decorated function defined at module scope instead, or use "
"build_reconstructable_target."
"build_reconstructable_job."
)

if seven.qualname_differs(target):
raise DagsterInvariantViolationError(
'Reconstructable target "{target.__name__}" has a different '
'__qualname__ "{target.__qualname__}" indicating it is not '
"defined at module scope. Use a function or decorated function "
"defined at module scope instead, or use build_reconstructable_pipeline.".format(
"defined at module scope instead, or use build_reconstructable_job.".format(
target=target
)
)
Expand All @@ -379,10 +379,9 @@ def make_bar_job():
python_file = get_python_file_from_target(target)
if not python_file:
raise DagsterInvariantViolationError(
"reconstructable() can not reconstruct jobs or pipelines defined in interactive environments "
"like <stdin>, IPython, or Jupyter notebooks. "
"Use a pipeline defined in a module or file instead, or "
"use build_reconstructable_target."
"reconstructable() can not reconstruct jobs or pipelines defined in interactive "
"environments like <stdin>, IPython, or Jupyter notebooks. "
"Use a pipeline defined in a module or file instead, or use build_reconstructable_job."
)

pointer = FileCodePointer(
Expand All @@ -393,7 +392,7 @@ def make_bar_job():


@experimental
def build_reconstructable_target(
def build_reconstructable_job(
reconstructor_module_name,
reconstructor_function_name,
reconstructable_args=None,
Expand All @@ -403,63 +402,63 @@ def build_reconstructable_target(
"""
Create a :py:class:`dagster.core.definitions.reconstructable.ReconstructablePipeline`.
When your pipeline must cross process boundaries, e.g., for execution on multiple nodes or
in different systems (like ``dagstermill``), Dagster must know how to reconstruct the pipeline
When your job must cross process boundaries, e.g., for execution on multiple nodes or in
different systems (like ``dagstermill``), Dagster must know how to reconstruct the job
on the other side of the process boundary.
This function allows you to use the strategy of your choice for reconstructing pipelines, so
that you can reconstruct certain kinds of pipelines that are not supported by
This function allows you to use the strategy of your choice for reconstructing jobs, so
that you can reconstruct certain kinds of jobs that are not supported by
:py:func:`~dagster.reconstructable`, such as those defined by lambdas, in nested scopes (e.g.,
dynamically within a method call), or in interactive environments such as the Python REPL or
Jupyter notebooks.
If you need to reconstruct pipelines constructed in these ways, use this function instead of
If you need to reconstruct jobs constructed in these ways, use this function instead of
:py:func:`~dagster.reconstructable`.
Args:
reconstructor_module_name (str): The name of the module containing the function to use to
reconstruct the pipeline.
reconstruct the job.
reconstructor_function_name (str): The name of the function to use to reconstruct the
pipeline.
reconstructable_args (Tuple): Args to the function to use to reconstruct the pipeline.
job.
reconstructable_args (Tuple): Args to the function to use to reconstruct the job.
Values of the tuple must be JSON serializable.
reconstructable_kwargs (Dict[str, Any]): Kwargs to the function to use to reconstruct the
pipeline. Values of the dict must be JSON serializable.
job. Values of the dict must be JSON serializable.
Examples:
.. code-block:: python
# module: mymodule
from dagster import PipelineDefinition, pipeline, build_reconstructable_pipeline
from dagster import JobDefinition, job, build_reconstructable_job
class PipelineFactory:
def make_pipeline(*args, **kwargs):
class JobFactory:
def make_job(*args, **kwargs):
@pipeline
def _pipeline(...):
@job
def _job(...):
...
return _pipeline
return _job
def reconstruct_pipeline(*args):
factory = PipelineFactory()
return factory.make_pipeline(*args)
def reconstruct_job(*args):
factory = JobFactory()
return factory.make_job(*args)
factory = PipelineFactory()
factory = JobFactory()
foo_pipeline_args = (...,...)
foo_job_args = (...,...)
foo_pipeline_kwargs = {...:...}
foo_job_kwargs = {...:...}
foo_pipeline = factory.make_pipeline(*foo_pipeline_args, **foo_pipeline_kwargs)
foo_job = factory.make_job(*foo_job_args, **foo_job_kwargs)
reconstructable_foo_pipeline = build_reconstructable_pipeline(
reconstructable_foo_job = build_reconstructable_job(
'mymodule',
'reconstruct_pipeline',
foo_pipeline_args,
foo_pipeline_kwargs,
'reconstruct_job',
foo_job_args,
foo_job_kwargs,
)
"""
check.str_param(reconstructor_module_name, "reconstructor_module_name")
Expand Down Expand Up @@ -494,7 +493,9 @@ def reconstruct_pipeline(*args):
)


build_reconstructable_pipeline = build_reconstructable_target
# back compat, in case users have imported these directly
build_reconstructable_pipeline = build_reconstructable_job
build_reconstructable_target = build_reconstructable_job


def bootstrap_standalone_recon_pipeline(pointer):
Expand Down

1 comment on commit 47e8533

@vercel
Copy link

@vercel vercel bot commented on 47e8533 Feb 18, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please sign in to comment.