diff --git a/examples/airline_demo/airline_demo/repository.yaml b/examples/airline_demo/airline_demo/repository.yaml --- a/examples/airline_demo/airline_demo/repository.yaml +++ b/examples/airline_demo/airline_demo/repository.yaml @@ -1,3 +1,4 @@ -repository: - module: airline_demo.repository - fn: airline_demo_repo +load_from: + - python_module: + module_name: airline_demo.repository + attribute: airline_demo_repo diff --git a/examples/airline_demo/repository.yaml b/examples/airline_demo/repository.yaml --- a/examples/airline_demo/repository.yaml +++ b/examples/airline_demo/repository.yaml @@ -1,3 +1,4 @@ -repository: - module: airline_demo - fn: define_internal_dagit_repository +load_from: + - python_module: + module_name: airline_demo + attribute: define_internal_dagit_repository diff --git a/examples/docs_snippets/docs_snippets/deploying/docker/repository.yaml b/examples/docs_snippets/docs_snippets/deploying/docker/repository.yaml --- a/examples/docs_snippets/docs_snippets/deploying/docker/repository.yaml +++ b/examples/docs_snippets/docs_snippets/deploying/docker/repository.yaml @@ -1,3 +1,4 @@ -repository: - module: demo - fn: demo_repo +load_from: + - python_module: + module_name: demo + attribute: demo_repo diff --git a/examples/docs_snippets/docs_snippets/intro_tutorial/advanced/repositories/repository.yaml b/examples/docs_snippets/docs_snippets/intro_tutorial/advanced/repositories/repository.yaml --- a/examples/docs_snippets/docs_snippets/intro_tutorial/advanced/repositories/repository.yaml +++ b/examples/docs_snippets/docs_snippets/intro_tutorial/advanced/repositories/repository.yaml @@ -1,3 +1,4 @@ -repository: - module: docs_snippets.intro_tutorial.advanced.repositories.repos - fn: hello_cereal_repository +load_from: + - python_module: + module_name: docs_snippets.intro_tutorial.advanced.repositories.repos + attribute: hello_cereal_repository diff --git a/examples/docs_snippets/docs_snippets/legacy/stocks/repository.yaml b/examples/docs_snippets/docs_snippets/legacy/stocks/repository.yaml --- a/examples/docs_snippets/docs_snippets/legacy/stocks/repository.yaml +++ b/examples/docs_snippets/docs_snippets/legacy/stocks/repository.yaml @@ -1,3 +1,4 @@ -repository: - module: dagster_examples.stocks.repository - fn: define_repo +load_from: + - python_module: + module_name: dagster_examples.stocks.repository + attribute: define_repo diff --git a/examples/legacy_examples/dagster_examples/bay_bikes/repository.yaml b/examples/legacy_examples/dagster_examples/bay_bikes/repository.yaml --- a/examples/legacy_examples/dagster_examples/bay_bikes/repository.yaml +++ b/examples/legacy_examples/dagster_examples/bay_bikes/repository.yaml @@ -1,3 +1,4 @@ -repository: - module: dagster_examples.bay_bikes.repository - fn: define_repo +load_from: + - python_module: + module_name: dagster_examples.bay_bikes.repository + attribute: define_repo diff --git a/examples/legacy_examples/dagster_examples/event_pipeline_demo/repository.yaml b/examples/legacy_examples/dagster_examples/event_pipeline_demo/repository.yaml --- a/examples/legacy_examples/dagster_examples/event_pipeline_demo/repository.yaml +++ b/examples/legacy_examples/dagster_examples/event_pipeline_demo/repository.yaml @@ -1,3 +1,4 @@ -repository: - module: dagster_examples.event_pipeline_demo.repository - fn: define_repo +load_from: + - python_module: + module_name: dagster_examples.event_pipeline_demo.repository + attribute: define_repo diff --git a/examples/legacy_examples/dagster_examples/gcp_data_platform/repository.yaml b/examples/legacy_examples/dagster_examples/gcp_data_platform/repository.yaml --- a/examples/legacy_examples/dagster_examples/gcp_data_platform/repository.yaml +++ b/examples/legacy_examples/dagster_examples/gcp_data_platform/repository.yaml @@ -1,3 +1,4 @@ -repository: - module: dagster_examples.gcp_data_platform.repo - fn: define_repo +load_from: + - python_module: + module_name: dagster_examples.gcp_data_platform.repo + attribute: define_repo diff --git a/examples/legacy_examples/dagster_examples/pandas_hello_world/repository.yaml b/examples/legacy_examples/dagster_examples/pandas_hello_world/repository.yaml --- a/examples/legacy_examples/dagster_examples/pandas_hello_world/repository.yaml +++ b/examples/legacy_examples/dagster_examples/pandas_hello_world/repository.yaml @@ -1,3 +1,4 @@ -repository: - file: repo.py - fn: define_repo +load_from: + - python_file: + relative_path: repo.py + attribute: define_repo diff --git a/examples/legacy_examples/dagster_examples/repository.yaml b/examples/legacy_examples/dagster_examples/repository.yaml --- a/examples/legacy_examples/dagster_examples/repository.yaml +++ b/examples/legacy_examples/dagster_examples/repository.yaml @@ -1,3 +1,4 @@ -repository: - module: dagster_examples - fn: define_demo_repo +load_from: + - python_module: + module_name: dagster_examples + attribute: define_demo_repo diff --git a/examples/legacy_examples/dagster_examples/simple_lakehouse/repository.yaml b/examples/legacy_examples/dagster_examples/simple_lakehouse/repository.yaml --- a/examples/legacy_examples/dagster_examples/simple_lakehouse/repository.yaml +++ b/examples/legacy_examples/dagster_examples/simple_lakehouse/repository.yaml @@ -1,3 +1,4 @@ -repository: - module: dagster_examples.simple_pyspark.repository - fn: define_repo +load_from: + - python_module: + module_name: dagster_examples.simple_pyspark.repository + attribute: define_repo diff --git a/examples/legacy_examples/dagster_examples/simple_pyspark/repository.yaml b/examples/legacy_examples/dagster_examples/simple_pyspark/repository.yaml --- a/examples/legacy_examples/dagster_examples/simple_pyspark/repository.yaml +++ b/examples/legacy_examples/dagster_examples/simple_pyspark/repository.yaml @@ -1,3 +1,4 @@ -repository: - module: dagster_examples.simple_pyspark.repository - fn: define_repo +load_from: + - python_module: + module_name: dagster_examples.simple_pyspark.repository + attribute: define_repo diff --git a/examples/legacy_examples/dagster_examples_tests/repository.yaml b/examples/legacy_examples/dagster_examples_tests/repository.yaml --- a/examples/legacy_examples/dagster_examples_tests/repository.yaml +++ b/examples/legacy_examples/dagster_examples_tests/repository.yaml @@ -1,3 +1,4 @@ -repository: - module: docs_snippets.docs_snippets.intro_tutorial.repos - fn: define_repo +load_from: + - python_module: + module_name: docs_snippets.docs_snippets.intro_tutorial.repos + attribute: define_repo diff --git a/python_modules/dagit/dagit_tests/repository.yaml b/python_modules/dagit/dagit_tests/repository.yaml --- a/python_modules/dagit/dagit_tests/repository.yaml +++ b/python_modules/dagit/dagit_tests/repository.yaml @@ -1,3 +1,4 @@ -repository: - file: pipeline.py - fn: test_repository +load_from: + - python_file: + relative_path: pipeline.py + attribute: test_repository diff --git a/python_modules/dagit/dagit_tests/repository_bad_module.yaml b/python_modules/dagit/dagit_tests/repository_bad_module.yaml --- a/python_modules/dagit/dagit_tests/repository_bad_module.yaml +++ b/python_modules/dagit/dagit_tests/repository_bad_module.yaml @@ -1,3 +1,4 @@ -repository: - module: not_a_module - fn: define_repository +load_from: + - python_module: + module_name: not_a_module + attribute: define_repository diff --git a/python_modules/dagit/dagit_tests/stress/repository.yaml b/python_modules/dagit/dagit_tests/stress/repository.yaml --- a/python_modules/dagit/dagit_tests/stress/repository.yaml +++ b/python_modules/dagit/dagit_tests/stress/repository.yaml @@ -1,3 +1,4 @@ -repository: - file: megadags.py - fn: define_repository +load_from: + - python_file: + relative_path: megadags.py + attribute: define_repository diff --git a/python_modules/dagit/dagit_tests/telemetry_repository.yaml b/python_modules/dagit/dagit_tests/telemetry_repository.yaml --- a/python_modules/dagit/dagit_tests/telemetry_repository.yaml +++ b/python_modules/dagit/dagit_tests/telemetry_repository.yaml @@ -1,3 +1,4 @@ -repository: - file: ../../dagster/dagster_tests/general_tests/test_repository.py - fn: dagster_test_repository +load_from: + - python_file: + relative_path: ../../dagster/dagster_tests/general_tests/test_repository.py + attribute: dagster_test_repository diff --git a/python_modules/dagster-graphql/dagster_graphql/test/utils.py b/python_modules/dagster-graphql/dagster_graphql/test/utils.py --- a/python_modules/dagster-graphql/dagster_graphql/test/utils.py +++ b/python_modules/dagster-graphql/dagster_graphql/test/utils.py @@ -7,13 +7,20 @@ from dagster import check from dagster.cli.workspace import Workspace from dagster.core.code_pointer import CodePointer -from dagster.core.definitions.reconstructable import ReconstructableRepository from dagster.core.host_representation import RepositoryLocationHandle from dagster.core.host_representation.handle import UserProcessApi from dagster.core.instance import DagsterInstance from dagster.core.types.loadable_target_origin import LoadableTargetOrigin +def main_repo_location_name(): + return "test_location" + + +def main_repo_name(): + return "test_repo" + + def execute_dagster_graphql(context, query, variables=None): result = graphql( create_schema(), @@ -41,8 +48,9 @@ return result -def define_context_for_file(python_file, fn_name, instance): +def define_in_process_context(python_file, fn_name, instance): check.inst_param(instance, "instance", DagsterInstance) + return DagsterGraphQLContext( workspace=Workspace( [ @@ -65,7 +73,7 @@ loadable_target_origin=LoadableTargetOrigin( executable_path=sys.executable, python_file=python_file, attribute=fn_name, ), - location_name="test_location", + location_name=main_repo_location_name(), user_process_api=UserProcessApi.CLI, ) ] @@ -74,20 +82,6 @@ ) -def define_context_for_repository_yaml(path, instance): - check.inst_param(instance, "instance", DagsterInstance) - return DagsterGraphQLContext( - workspace=Workspace( - [ - RepositoryLocationHandle.create_in_process_location( - ReconstructableRepository.from_legacy_repository_yaml(path).pointer - ) - ] - ), - instance=instance, - ) - - def infer_repository(graphql_context): if len(graphql_context.repository_locations) == 1: # This is to account for having a single in process repository diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/cli_test_workspace.yaml b/python_modules/dagster-graphql/dagster_graphql_tests/cli_test_workspace.yaml --- a/python_modules/dagster-graphql/dagster_graphql_tests/cli_test_workspace.yaml +++ b/python_modules/dagster-graphql/dagster_graphql_tests/cli_test_workspace.yaml @@ -1,3 +1,5 @@ -repository: - file: test_cli.py - fn: test +load_from: + - python_file: + relative_path: test_cli_repo.py + attribute: test + location_name: test_cli_location diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/conftest.py b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/conftest.py --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/conftest.py +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/conftest.py @@ -3,7 +3,7 @@ from dagster import seven from dagster.core.instance import DagsterInstance -from .setup import define_test_context +from .setup import define_test_in_process_context, define_test_out_of_process_context @pytest.yield_fixture(scope="function") @@ -19,4 +19,20 @@ } }, ) as instance: - yield define_test_context(instance) + yield define_test_out_of_process_context(instance) + + +@pytest.yield_fixture(scope="function") +def graphql_in_process_context(): + with seven.TemporaryDirectory() as temp_dir: + with DagsterInstance.local_temp( + temp_dir, + overrides={ + "scheduler": { + "module": "dagster.utils.test", + "class": "FilesystemTestScheduler", + "config": {"base_dir": temp_dir}, + } + }, + ) as instance: + yield define_test_in_process_context(instance) diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/empty_repo.yaml b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/empty_repo.yaml --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/empty_repo.yaml +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/empty_repo.yaml @@ -1,3 +1,4 @@ -repository: - file: setup.py - fn: empty_repo +load_from: + - python_file: + relative_path: setup.py + attribute: empty_repo diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/graphql_context_test_suite.py b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/graphql_context_test_suite.py --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/graphql_context_test_suite.py +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/graphql_context_test_suite.py @@ -27,9 +27,7 @@ def get_main_recon_repo(): - return ReconstructableRepository.from_legacy_repository_yaml( - file_relative_path(__file__, "repo.yaml") - ) + return ReconstructableRepository.for_file(file_relative_path(__file__, "setup.py"), "test_repo") @contextmanager @@ -346,10 +344,6 @@ """Goes out of process but same process as host process""" check.inst_param(recon_repo, "recon_repo", ReconstructableRepository) - empty_repo = ReconstructableRepository.from_legacy_repository_yaml( - file_relative_path(__file__, "empty_repo.yaml") - ) - with Workspace( [ RepositoryLocationHandle.create_python_env_location( @@ -360,7 +354,6 @@ ), location_name="test", ), - RepositoryLocationHandle.create_in_process_location(empty_repo.pointer), RepositoryLocationHandle.create_python_env_location( loadable_target_origin=LoadableTargetOrigin( executable_path=sys.executable, diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/repo.yaml b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/repo.yaml deleted file mode 100644 --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/repo.yaml +++ /dev/null @@ -1,3 +0,0 @@ -repository: - file: setup.py - fn: test_repo diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/setup.py b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/setup.py --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/setup.py +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/setup.py @@ -6,11 +6,12 @@ from collections import OrderedDict from copy import deepcopy -from dagster_graphql.implementation.context import DagsterGraphQLContext from dagster_graphql.test.utils import ( - define_context_for_file, + define_in_process_context, define_out_of_process_context, infer_pipeline_selector, + main_repo_location_name, + main_repo_name, ) from dagster import ( @@ -52,11 +53,12 @@ usable_as_dagster_type, weekly_schedule, ) -from dagster.cli.workspace import Workspace +from dagster.cli.workspace.load import location_handle_from_python_file from dagster.core.definitions.decorators import executable from dagster.core.definitions.partition import last_empty_partition from dagster.core.definitions.reconstructable import ReconstructableRepository -from dagster.core.host_representation import InProcessRepositoryLocation, RepositoryLocationHandle +from dagster.core.host_representation import RepositoryLocation +from dagster.core.host_representation.handle import python_user_process_api_from_instance from dagster.core.log_manager import coerce_valid_log_level from dagster.core.storage.tags import RESUME_RETRY_TAG from dagster.utils import file_relative_path, segfault @@ -88,41 +90,28 @@ def define_test_out_of_process_context(instance): check.inst_param(instance, "instance", DagsterInstance) - return define_out_of_process_context(__file__, "test_repo", instance) + return define_out_of_process_context(__file__, main_repo_name(), instance) -def define_test_context(instance): +def define_test_in_process_context(instance): check.inst_param(instance, "instance", DagsterInstance) - return define_context_for_file(__file__, "test_repo", instance) + return define_in_process_context(__file__, main_repo_name(), instance) def create_main_recon_repo(): - return ReconstructableRepository.for_file(__file__, "test_repo") + return ReconstructableRepository.for_file(__file__, main_repo_name()) -def get_main_external_repo(): - return InProcessRepositoryLocation( - ReconstructableRepository.from_legacy_repository_yaml( - file_relative_path(__file__, "repo.yaml") - ), - ).get_repository("test_repo") - - -def define_test_snapshot_context(): - return DagsterGraphQLContext( - instance=DagsterInstance.ephemeral(), - workspace=Workspace( - [RepositoryLocationHandle.create_in_process_location(create_main_recon_repo().pointer)] - ), - ) - - -def main_repo_location_name(): - return "<>" - - -def main_repo_name(): - return "test_repo" +def get_main_external_repo(instance): + return RepositoryLocation.from_handle( + location_handle_from_python_file( + python_file=file_relative_path(__file__, "setup.py"), + attribute=main_repo_name(), + working_directory=None, + user_process_api=python_user_process_api_from_instance(instance), + location_name=main_repo_location_name(), + ) + ).get_repository(main_repo_name()) @lambda_solid( diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_all_snapshot_ids.py b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_all_snapshot_ids.py --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_all_snapshot_ids.py +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_all_snapshot_ids.py @@ -1,3 +1,4 @@ +from dagster.core.test_utils import instance_for_test from dagster.serdes import serialize_pp from .setup import get_main_external_repo @@ -7,7 +8,8 @@ # This ensures that pipeline snapshots remain stable # If you 1) change any pipelines in dagster_graphql_test or 2) change the # schema of PipelineSnapshots you are free to rerecord - repo = get_main_external_repo() - for pipeline in sorted(repo.get_all_external_pipelines(), key=lambda p: p.name): - snapshot.assert_match(serialize_pp(pipeline.pipeline_snapshot)) - snapshot.assert_match(pipeline.computed_pipeline_snapshot_id) + with instance_for_test() as instance: + repo = get_main_external_repo(instance) + for pipeline in sorted(repo.get_all_external_pipelines(), key=lambda p: p.name): + snapshot.assert_match(serialize_pp(pipeline.pipeline_snapshot)) + snapshot.assert_match(pipeline.computed_pipeline_snapshot_id) diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_pipeline_snapshot.py b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_pipeline_snapshot.py --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_pipeline_snapshot.py +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_pipeline_snapshot.py @@ -3,12 +3,16 @@ _get_dauphin_pipeline_snapshot_from_instance, ) from dagster_graphql.implementation.utils import UserFacingGraphQLError -from dagster_graphql.test.utils import execute_dagster_graphql +from dagster_graphql.test.utils import ( + execute_dagster_graphql, + main_repo_location_name, + main_repo_name, +) from dagster import execute_pipeline from dagster.seven import json, mock -from .setup import main_repo_location_name, main_repo_name, noop_pipeline +from .setup import noop_pipeline SNAPSHOT_OR_ERROR_QUERY_BY_SNAPSHOT_ID = """ query PipelineSnapshotQueryBySnapshotID($snapshotId: String!) { diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_run_cancellation.py b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_run_cancellation.py --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_run_cancellation.py +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_run_cancellation.py @@ -5,10 +5,10 @@ from dagster_graphql.test.utils import execute_dagster_graphql, infer_pipeline_selector from dagster import execute_pipeline +from dagster.core.host_representation.handle import IN_PROCESS_NAME from dagster.utils import safe_tempfile_path from .graphql_context_test_suite import GraphQLContextVariant, make_graphql_context_test_suite -from .setup import main_repo_location_name RUN_CANCELLATION_QUERY = """ mutation($runId: String!) { @@ -137,7 +137,7 @@ def test_run_finished(self, graphql_context): instance = graphql_context.instance pipeline = graphql_context.get_repository_location( - main_repo_location_name() + IN_PROCESS_NAME ).get_reconstructable_pipeline("noop_pipeline") pipeline_result = execute_pipeline(pipeline, instance=instance) diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_runs.py b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_runs.py --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_runs.py +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_runs.py @@ -1,7 +1,7 @@ import copy from dagster_graphql.test.utils import ( - define_context_for_file, + define_out_of_process_context, execute_dagster_graphql, infer_pipeline_selector, ) @@ -325,7 +325,7 @@ instance=instance, ).run_id - context_at_time_1 = define_context_for_file(__file__, "get_repo_at_time_1", instance) + context_at_time_1 = define_out_of_process_context(__file__, "get_repo_at_time_1", instance) result = execute_dagster_graphql(context_at_time_1, ALL_RUNS_QUERY) assert result.data @@ -356,7 +356,7 @@ "solidSelection": ["solid_B"], } - context_at_time_2 = define_context_for_file(__file__, "get_repo_at_time_2", instance) + context_at_time_2 = define_out_of_process_context(__file__, "get_repo_at_time_2", instance) result = execute_dagster_graphql(context_at_time_2, ALL_RUNS_QUERY) assert result.data @@ -407,7 +407,7 @@ instance=instance, ).run_id - context_at_time_1 = define_context_for_file(__file__, "get_repo_at_time_1", instance) + context_at_time_1 = define_out_of_process_context(__file__, "get_repo_at_time_1", instance) result = execute_dagster_graphql(context_at_time_1, ALL_RUN_GROUPS_QUERY) assert result.data @@ -450,7 +450,7 @@ "solidSelection": ["solid_B"], } - context_at_time_2 = define_context_for_file(__file__, "get_repo_at_time_2", instance) + context_at_time_2 = define_out_of_process_context(__file__, "get_repo_at_time_2", instance) result = execute_dagster_graphql(context_at_time_2, ALL_RUN_GROUPS_QUERY) assert "runGroupsOrError" in result.data @@ -514,7 +514,7 @@ _run_id_2 = execute_pipeline( repo.get_pipeline("foo_pipeline"), instance=instance, tags={"run": "two"} ).run_id - context = define_context_for_file(__file__, "get_repo_at_time_1", instance) + context = define_out_of_process_context(__file__, "get_repo_at_time_1", instance) result = execute_dagster_graphql( context, FILTERED_RUN_QUERY, variables={"filter": {"runId": run_id_1}} ) @@ -552,7 +552,7 @@ execute_run(InMemoryPipeline(foo_pipeline), run, instance) runs.append(run) - context_at_time_1 = define_context_for_file(__file__, "get_repo_at_time_1", instance) + context_at_time_1 = define_out_of_process_context(__file__, "get_repo_at_time_1", instance) result_one = execute_dagster_graphql( context_at_time_1, RUN_GROUP_QUERY, variables={"runId": root_run_id}, @@ -579,7 +579,7 @@ def test_run_group_not_found(): with seven.TemporaryDirectory() as temp_dir: instance = DagsterInstance.local_temp(temp_dir) - context_at_time_1 = define_context_for_file(__file__, "get_repo_at_time_1", instance) + context_at_time_1 = define_out_of_process_context(__file__, "get_repo_at_time_1", instance) result = execute_dagster_graphql( context_at_time_1, RUN_GROUP_QUERY, variables={"runId": "foo"}, @@ -610,7 +610,7 @@ instance=instance, ) - context_at_time_1 = define_context_for_file(__file__, "get_repo_at_time_1", instance) + context_at_time_1 = define_out_of_process_context(__file__, "get_repo_at_time_1", instance) result = execute_dagster_graphql(context_at_time_1, ALL_RUN_GROUPS_QUERY) assert result.data diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_scheduler.py b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_scheduler.py --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_scheduler.py +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_scheduler.py @@ -2,12 +2,12 @@ execute_dagster_graphql, infer_repository_selector, infer_schedule_selector, + main_repo_location_name, + main_repo_name, ) from dagster.core.scheduler.scheduler import ScheduleStatus -from .setup import main_repo_location_name, main_repo_name - GET_SCHEDULE_STATES_QUERY = """ query ScheduleStateQuery($repositorySelector: RepositorySelector!) { scheduleStatesOrError(repositorySelector: $repositorySelector) { diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_sync_run_launcher.py b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_sync_run_launcher.py --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_sync_run_launcher.py +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_sync_run_launcher.py @@ -21,9 +21,6 @@ def test_sync_run_launcher_run(): - external_repo = get_main_external_repo() - external_pipeline = external_repo.get_full_external_pipeline("noop_pipeline") - with seven.TemporaryDirectory() as temp_dir: instance = DagsterInstance.local_temp( temp_dir, @@ -35,6 +32,9 @@ }, ) + external_repo = get_main_external_repo(instance) + external_pipeline = external_repo.get_full_external_pipeline("noop_pipeline") + run = create_run_for_test(instance=instance, pipeline_name=external_pipeline.name) run = instance.run_launcher.launch_run( diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/repository.yaml b/python_modules/dagster-graphql/dagster_graphql_tests/repository.yaml --- a/python_modules/dagster-graphql/dagster_graphql_tests/repository.yaml +++ b/python_modules/dagster-graphql/dagster_graphql_tests/repository.yaml @@ -1,3 +1,4 @@ -repository: - file: graphql/setup.py - fn: test_repo +load_from: + - python_file: + relative_path: graphql/setup.py + attribute: test_repo diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/test_cli.py b/python_modules/dagster-graphql/dagster_graphql_tests/test_cli.py --- a/python_modules/dagster-graphql/dagster_graphql_tests/test_cli.py +++ b/python_modules/dagster-graphql/dagster_graphql_tests/test_cli.py @@ -6,17 +6,7 @@ from click.testing import CliRunner from dagster_graphql.cli import ui -from dagster import ( - InputDefinition, - Int, - OutputDefinition, - ScheduleDefinition, - lambda_solid, - pipeline, - repository, - seven, - solid, -) +from dagster import seven from dagster.core.storage.pipeline_run import PipelineRunStatus from dagster.core.test_utils import instance_for_test_tempdir from dagster.utils import file_relative_path @@ -37,53 +27,6 @@ yield CliRunner(env={"DAGSTER_HOME": dagster_home_temp}) -@lambda_solid(input_defs=[InputDefinition("num", Int)], output_def=OutputDefinition(Int)) -def add_one(num): - return num + 1 - - -@lambda_solid(input_defs=[InputDefinition("num", Int)], output_def=OutputDefinition(Int)) -def mult_two(num): - return num * 2 - - -@pipeline -def math(): - mult_two(add_one()) - - -@solid(config_schema={"gimme": str}) -def needs_config(context): - return context.solid_config["gimme"] - - -@lambda_solid -def no_config(): - return "ok" - - -@pipeline -def subset_test(): - no_config() - needs_config() - - -def define_schedules(): - math_hourly_schedule = ScheduleDefinition( - name="math_hourly_schedule", - cron_schedule="0 0 * * *", - pipeline_name="math", - run_config={"solids": {"add_one": {"inputs": {"num": {"value": 123}}}}}, - ) - - return [math_hourly_schedule] - - -@repository -def test(): - return [math, subset_test] + define_schedules() - - def test_basic_introspection(): query = "{ __schema { types { name } } }" @@ -118,7 +61,7 @@ { ... on Pipeline { name } } } """ - variables = '{"pipelineName": "math", "repositoryName": "test", "repositoryLocationName": "<>"}' + variables = '{"pipelineName": "math", "repositoryName": "test", "repositoryLocationName": "test_cli_location"}' workspace_path = file_relative_path(__file__, "./cli_test_workspace.yaml") with dagster_cli_runner() as runner: @@ -161,7 +104,7 @@ { "executionParams": { "selector": { - "repositoryLocationName": "<>", + "repositoryLocationName": "test_cli_location", "repositoryName": "test", "pipelineName": "math", }, @@ -196,7 +139,7 @@ "executionParams": { "selector": { "pipelineName": "math", - "repositoryLocationName": "<>", + "repositoryLocationName": "test_cli_location", "repositoryName": "test", }, "runConfigData": {"solids": {"add_one": {"inputs": {"num": {"value": 123}}}}}, @@ -236,7 +179,7 @@ { "executionParams": { "selector": { - "repositoryLocationName": "<>", + "repositoryLocationName": "test_cli_location", "repositoryName": "test", "pipelineName": "math", }, @@ -283,7 +226,7 @@ { "executionParams": { "selector": { - "repositoryLocationName": "<>", + "repositoryLocationName": "test_cli_location", "repositoryName": "test", "pipelineName": "math", }, @@ -314,7 +257,7 @@ { "executionParams": { "selector": { - "repositoryLocationName": "<>", + "repositoryLocationName": "test_cli_location", "repositoryName": "test", "pipelineName": "math", }, diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/test_cli_repo.py b/python_modules/dagster-graphql/dagster_graphql_tests/test_cli_repo.py new file mode 100644 --- /dev/null +++ b/python_modules/dagster-graphql/dagster_graphql_tests/test_cli_repo.py @@ -0,0 +1,57 @@ +from dagster import ( + InputDefinition, + Int, + OutputDefinition, + ScheduleDefinition, + lambda_solid, + pipeline, + repository, + solid, +) + + +@lambda_solid(input_defs=[InputDefinition("num", Int)], output_def=OutputDefinition(Int)) +def add_one(num): + return num + 1 + + +@lambda_solid(input_defs=[InputDefinition("num", Int)], output_def=OutputDefinition(Int)) +def mult_two(num): + return num * 2 + + +@pipeline +def math(): + mult_two(add_one()) + + +@solid(config_schema={"gimme": str}) +def needs_config(context): + return context.solid_config["gimme"] + + +@lambda_solid +def no_config(): + return "ok" + + +@pipeline +def subset_test(): + no_config() + needs_config() + + +def define_schedules(): + math_hourly_schedule = ScheduleDefinition( + name="math_hourly_schedule", + cron_schedule="0 0 * * *", + pipeline_name="math", + run_config={"solids": {"add_one": {"inputs": {"num": {"value": 123}}}}}, + ) + + return [math_hourly_schedule] + + +@repository +def test(): + return [math, subset_test] + define_schedules() diff --git a/python_modules/dagster-test/dagster_test/dagster_core_docker_buildkite/workspace.yaml b/python_modules/dagster-test/dagster_test/dagster_core_docker_buildkite/workspace.yaml --- a/python_modules/dagster-test/dagster_test/dagster_core_docker_buildkite/workspace.yaml +++ b/python_modules/dagster-test/dagster_test/dagster_core_docker_buildkite/workspace.yaml @@ -1,3 +1,4 @@ -repository: - module: dagster_test.test_project.test_pipelines.repo - fn: bar_repo +load_from: + - python_module: + module_name: dagster_test.test_project.test_pipelines.repo + attribute: bar_repo diff --git a/python_modules/dagster-test/dagster_test/test_project/workspace.yaml b/python_modules/dagster-test/dagster_test/test_project/workspace.yaml --- a/python_modules/dagster-test/dagster_test/test_project/workspace.yaml +++ b/python_modules/dagster-test/dagster_test/test_project/workspace.yaml @@ -1,3 +1,4 @@ -repository: - module: dagster_test.test_project.test_pipelines.repo - fn: define_demo_execution_repo +load_from: + - python_module: + module_name: dagster_test.test_project.test_pipelines.repo + attribute: define_demo_execution_repo diff --git a/python_modules/dagster/dagster_tests/api_tests/repository_file.yaml b/python_modules/dagster/dagster_tests/api_tests/legacy_repository_file.yaml rename from python_modules/dagster/dagster_tests/api_tests/repository_file.yaml rename to python_modules/dagster/dagster_tests/api_tests/legacy_repository_file.yaml diff --git a/python_modules/dagster/dagster_tests/api_tests/utils.py b/python_modules/dagster/dagster_tests/api_tests/utils.py --- a/python_modules/dagster/dagster_tests/api_tests/utils.py +++ b/python_modules/dagster/dagster_tests/api_tests/utils.py @@ -68,7 +68,7 @@ def legacy_get_bar_repo_handle(): recon_repo = ReconstructableRepository.from_legacy_repository_yaml( - file_relative_path(__file__, "repository_file.yaml") + file_relative_path(__file__, "legacy_repository_file.yaml") ) return InProcessRepositoryLocation(recon_repo).get_repository("bar_repo").handle diff --git a/python_modules/dagster/dagster_tests/core_tests/definitions_tests/repository_file.yaml b/python_modules/dagster/dagster_tests/core_tests/definitions_tests/repository_file.yaml --- a/python_modules/dagster/dagster_tests/core_tests/definitions_tests/repository_file.yaml +++ b/python_modules/dagster/dagster_tests/core_tests/definitions_tests/repository_file.yaml @@ -1,3 +1,4 @@ -repository: - file: test_handle.py - fn: define_bar_repo +load_from: + - python_file: + relative_path: test_handle.py + attribute: define_bar_repo diff --git a/python_modules/dagster/dagster_tests/core_tests/launcher_tests/repo.yaml b/python_modules/dagster/dagster_tests/core_tests/launcher_tests/repo.yaml deleted file mode 100644 --- a/python_modules/dagster/dagster_tests/core_tests/launcher_tests/repo.yaml +++ /dev/null @@ -1,3 +0,0 @@ -repository: - file: test_default_run_launcher.py - fn: nope diff --git a/python_modules/dagster/dagster_tests/core_tests/launcher_tests/test_default_run_launcher.py b/python_modules/dagster/dagster_tests/core_tests/launcher_tests/test_default_run_launcher.py --- a/python_modules/dagster/dagster_tests/core_tests/launcher_tests/test_default_run_launcher.py +++ b/python_modules/dagster/dagster_tests/core_tests/launcher_tests/test_default_run_launcher.py @@ -6,7 +6,6 @@ import pytest from dagster import DefaultRunLauncher, file_relative_path, pipeline, repository, seven, solid -from dagster.core.definitions.reconstructable import ReconstructableRepository from dagster.core.host_representation.handle import RepositoryLocationHandle, UserProcessApi from dagster.core.host_representation.repository_location import ( GrpcServerRepositoryLocation, @@ -95,16 +94,11 @@ return [noop_pipeline, crashy_pipeline, sleepy_pipeline, slow_pipeline, math_diamond] -def test_repo_construction(): - repo_yaml = file_relative_path(__file__, "repo.yaml") - assert ReconstructableRepository.from_legacy_repository_yaml(repo_yaml).get_definition() - - @contextmanager def get_external_pipeline_from_grpc_server_repository(pipeline_name): - repo_yaml = file_relative_path(__file__, "repo.yaml") - recon_repo = ReconstructableRepository.from_legacy_repository_yaml(repo_yaml) - loadable_target_origin = recon_repo.get_origin().loadable_target_origin + loadable_target_origin = LoadableTargetOrigin( + attribute="nope", python_file=file_relative_path(__file__, "test_default_run_launcher.py"), + ) server_process = GrpcServerProcess(loadable_target_origin=loadable_target_origin) try: diff --git a/python_modules/dagster/dagster_tests/general_tests/compat_tests/repository.yaml b/python_modules/dagster/dagster_tests/general_tests/compat_tests/repository.yaml deleted file mode 100644 --- a/python_modules/dagster/dagster_tests/general_tests/compat_tests/repository.yaml +++ /dev/null @@ -1,11 +0,0 @@ -repository: - file: test_loader.py - fn: define_repository - -scheduler: - file: test_loader.py - fn: define_schedules - -partitions: - file: test_loader.py - fn: define_partitions diff --git a/python_modules/libraries/dagster-cron/dagster_cron_tests/repository.yaml b/python_modules/libraries/dagster-cron/dagster_cron_tests/repository.yaml --- a/python_modules/libraries/dagster-cron/dagster_cron_tests/repository.yaml +++ b/python_modules/libraries/dagster-cron/dagster_cron_tests/repository.yaml @@ -1,3 +1,4 @@ -repository: - file: test_cron_scheduler.py - fn: test_repository +load_from: + - python_file: + relative_path: test_cron_scheduler.py + attribute: test_repository diff --git a/python_modules/libraries/dagster-pandas/dagster_pandas/examples/repository.yaml b/python_modules/libraries/dagster-pandas/dagster_pandas/examples/repository.yaml --- a/python_modules/libraries/dagster-pandas/dagster_pandas/examples/repository.yaml +++ b/python_modules/libraries/dagster-pandas/dagster_pandas/examples/repository.yaml @@ -1,3 +1,4 @@ -repository: - module: dagster_pandas.examples - fn: define_pandas_repository +load_from: + - python_module: + module_name: dagster_pandas.examples + attribute: define_pandas_repository diff --git a/python_modules/libraries/dagstermill/dagstermill/examples/repository.yaml b/python_modules/libraries/dagstermill/dagstermill/examples/repository.yaml --- a/python_modules/libraries/dagstermill/dagstermill/examples/repository.yaml +++ b/python_modules/libraries/dagstermill/dagstermill/examples/repository.yaml @@ -1,3 +1,4 @@ -repository: - module: dagstermill.examples.repository - fn: define_example_repository +load_from: + - python_module: + module_name: dagstermill.examples.repository + attribute: define_example_repository diff --git a/python_modules/libraries/dagstermill/dagstermill_tests/repository.yaml b/python_modules/libraries/dagstermill/dagstermill_tests/repository.yaml --- a/python_modules/libraries/dagstermill/dagstermill_tests/repository.yaml +++ b/python_modules/libraries/dagstermill/dagstermill_tests/repository.yaml @@ -1,3 +1,4 @@ -repository: - module: dagstermill.examples.repository - fn: define_example_repository +load_from: + - python_module: + module_name: dagstermill.examples.repository + attribute: define_example_repository