From 5e40837ad9c7b860371fa8ee916c840e54a82233 Mon Sep 17 00:00:00 2001 From: Colin Ho Date: Wed, 18 Dec 2024 22:09:47 -0800 Subject: [PATCH] feat: Default native runner (#3608) Set swordfish as default Note: - Disables printing of physical plan explains for now --------- Co-authored-by: Colin Ho --- daft/context.py | 9 +-------- daft/dataframe/dataframe.py | 7 ++++--- daft/runners/pyrunner.py | 4 ++-- src/daft-local-plan/src/translate.rs | 2 +- 4 files changed, 8 insertions(+), 14 deletions(-) diff --git a/daft/context.py b/daft/context.py index dbec115099..6c18ca2998 100644 --- a/daft/context.py +++ b/daft/context.py @@ -7,7 +7,6 @@ import warnings from typing import TYPE_CHECKING, ClassVar, Literal -from daft import get_build_type from daft.daft import IOConfig, PyDaftExecutionConfig, PyDaftPlanningConfig if TYPE_CHECKING: @@ -109,14 +108,8 @@ def _get_runner_config_from_env() -> _RunnerConfig: max_task_backlog=task_backlog, force_client_mode=ray_force_client_mode, ) - - # Use native runner if in dev mode - elif get_build_type() == "dev": - return _NativeRunnerConfig() - - # Fall back on PyRunner else: - return _PyRunnerConfig(use_thread_pool=use_thread_pool) + return _NativeRunnerConfig() @dataclasses.dataclass diff --git a/daft/dataframe/dataframe.py b/daft/dataframe/dataframe.py index 4264883c28..ca23c73cbb 100644 --- a/daft/dataframe/dataframe.py +++ b/daft/dataframe/dataframe.py @@ -208,9 +208,10 @@ def explain( print_to_file("\n== Optimized Logical Plan ==\n") builder = builder.optimize() print_to_file(builder.pretty_print(simple)) - print_to_file("\n== Physical Plan ==\n") - physical_plan_scheduler = builder.to_physical_plan_scheduler(get_context().daft_execution_config) - print_to_file(physical_plan_scheduler.pretty_print(simple, format=format)) + if get_context().get_or_create_runner().name != "native": + print_to_file("\n== Physical Plan ==\n") + physical_plan_scheduler = builder.to_physical_plan_scheduler(get_context().daft_execution_config) + print_to_file(physical_plan_scheduler.pretty_print(simple, format=format)) else: print_to_file( "\n \nSet `show_all=True` to also see the Optimized and Physical plans. This will run the query optimizer.", diff --git a/daft/runners/pyrunner.py b/daft/runners/pyrunner.py index 3fb6b61277..450bc4eb57 100644 --- a/daft/runners/pyrunner.py +++ b/daft/runners/pyrunner.py @@ -340,8 +340,8 @@ def run_iter( results_buffer_size: int | None = None, ) -> Iterator[LocalMaterializedResult]: warnings.warn( - "PyRunner will be deprecated in v0.4.0 and the new NativeRunner will become the default for local execution." - "We recommend switching to the NativeRunner now via `daft.context.set_runner_native()` or by setting the env variable `DAFT_RUNNER=native`. " + "PyRunner is deprecated and the new NativeRunner is now the default for local execution." + "Please switch to the NativeRunner now via `daft.context.set_runner_native()` or by setting the env variable `DAFT_RUNNER=native`. " "Please report any issues at github.com/Eventual-Inc/Daft/issues", ) # NOTE: Freeze and use this same execution config for the entire execution diff --git a/src/daft-local-plan/src/translate.rs b/src/daft-local-plan/src/translate.rs index e876c350a8..0547e9e737 100644 --- a/src/daft-local-plan/src/translate.rs +++ b/src/daft-local-plan/src/translate.rs @@ -197,7 +197,7 @@ pub fn translate(plan: &LogicalPlanRef) -> DaftResult { )) } LogicalPlan::Repartition(repartition) => { - log::warn!("Repartition Not supported for Local Executor!; This will be a No-Op"); + log::warn!("Repartition not supported on the NativeRunner. This will be a no-op. Please use the RayRunner instead if you need to repartition"); translate(&repartition.input) } LogicalPlan::MonotonicallyIncreasingId(monotonically_increasing_id) => {