Skip to content

Commit

Permalink
feat: Default native runner (#3608)
Browse files Browse the repository at this point in the history
Set swordfish as default

Note:
- Disables printing of physical plan explains for now

---------

Co-authored-by: Colin Ho <[email protected]>
  • Loading branch information
colin-ho and Colin Ho authored Dec 19, 2024
1 parent e706caa commit 5e40837
Show file tree
Hide file tree
Showing 4 changed files with 8 additions and 14 deletions.
9 changes: 1 addition & 8 deletions daft/context.py
Original file line number Diff line number Diff line change
Expand Up @@ -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:
Expand Down Expand Up @@ -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
Expand Down
7 changes: 4 additions & 3 deletions daft/dataframe/dataframe.py
Original file line number Diff line number Diff line change
Expand Up @@ -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.",
Expand Down
4 changes: 2 additions & 2 deletions daft/runners/pyrunner.py
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
2 changes: 1 addition & 1 deletion src/daft-local-plan/src/translate.rs
Original file line number Diff line number Diff line change
Expand Up @@ -197,7 +197,7 @@ pub fn translate(plan: &LogicalPlanRef) -> DaftResult<LocalPhysicalPlanRef> {
))
}
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) => {
Expand Down

0 comments on commit 5e40837

Please sign in to comment.