diff --git a/python_modules/dagster/dagster/core/host_representation/handle.py b/python_modules/dagster/dagster/core/host_representation/handle.py --- a/python_modules/dagster/dagster/core/host_representation/handle.py +++ b/python_modules/dagster/dagster/core/host_representation/handle.py @@ -64,9 +64,9 @@ opt_in_settings = instance.get_settings("opt_in") return ( - UserProcessApi.GRPC - if (opt_in_settings and opt_in_settings["local_servers"]) - else UserProcessApi.CLI + UserProcessApi.CLI + if (opt_in_settings and not opt_in_settings["local_servers"]) + else UserProcessApi.GRPC ) diff --git a/python_modules/dagster/dagster/core/host_representation/repository_location.py b/python_modules/dagster/dagster/core/host_representation/repository_location.py --- a/python_modules/dagster/dagster/core/host_representation/repository_location.py +++ b/python_modules/dagster/dagster/core/host_representation/repository_location.py @@ -495,7 +495,9 @@ step_keys_to_execute, retries=None, ): - raise NotImplementedError("execute_plan is not implemented for grpc servers") + raise NotImplementedError( + "execute_plan is not supported for out-of-process repository locations." + ) def execute_pipeline( self, instance, external_pipeline, pipeline_run, diff --git a/python_modules/dagster/dagster/core/instance/config.py b/python_modules/dagster/dagster/core/instance/config.py --- a/python_modules/dagster/dagster/core/instance/config.py +++ b/python_modules/dagster/dagster/core/instance/config.py @@ -53,6 +53,6 @@ "schedule_storage": config_field_for_configurable_class(), "scheduler": config_field_for_configurable_class(), "run_launcher": config_field_for_configurable_class(), - "telemetry": Field({"enabled": Field(Bool, default_value=True, is_required=False)}), - "opt_in": Field({"local_servers": Field(Bool, default_value=False, is_required=False)}), + "telemetry": Field({"enabled": Field(Bool, is_required=False)}), + "opt_in": Field({"local_servers": Field(Bool, is_required=False)}), } diff --git a/python_modules/dagster/dagster_tests/core_tests/instance_tests/test_instance.py b/python_modules/dagster/dagster_tests/core_tests/instance_tests/test_instance.py --- a/python_modules/dagster/dagster_tests/core_tests/instance_tests/test_instance.py +++ b/python_modules/dagster/dagster_tests/core_tests/instance_tests/test_instance.py @@ -5,6 +5,10 @@ from dagster import PipelineDefinition, check, execute_pipeline, pipeline, solid from dagster.core.errors import DagsterInvariantViolationError from dagster.core.execution.api import create_execution_plan +from dagster.core.host_representation.handle import ( + UserProcessApi, + python_user_process_api_from_instance, +) from dagster.core.instance import DagsterInstance, _dagster_home from dagster.core.snap import ( create_execution_plan_snapshot_id, @@ -151,3 +155,18 @@ ), ): _dagster_home() + + +def test_default_user_process_api(): + with instance_for_test() as instance: + assert python_user_process_api_from_instance(instance) == UserProcessApi.GRPC + + +def test_manually_set_grpc_instance(): + with instance_for_test(overrides={"opt_in": {"local_servers": True}}) as instance: + assert python_user_process_api_from_instance(instance) == UserProcessApi.GRPC + + +def test_manually_set_cli_api_instance(): + with instance_for_test(overrides={"opt_in": {"local_servers": False}}) as instance: + assert python_user_process_api_from_instance(instance) == UserProcessApi.CLI diff --git a/python_modules/dagster/dagster_tests/scheduler_tests/test_scheduler_run.py b/python_modules/dagster/dagster_tests/scheduler_tests/test_scheduler_run.py --- a/python_modules/dagster/dagster_tests/scheduler_tests/test_scheduler_run.py +++ b/python_modules/dagster/dagster_tests/scheduler_tests/test_scheduler_run.py @@ -242,7 +242,10 @@ @contextmanager def default_repo(): loadable_target_origin = LoadableTargetOrigin( - executable_path=sys.executable, python_file=__file__, attribute="the_repo", + executable_path=sys.executable, + python_file=__file__, + attribute="the_repo", + working_directory=os.getcwd(), ) with RepositoryLocationHandle.create_python_env_location(