Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feat: Broadcast plan over port 3238 #3756

Open
wants to merge 18 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
29 changes: 29 additions & 0 deletions daft/context.py
Original file line number Diff line number Diff line change
Expand Up @@ -125,6 +125,10 @@ class DaftContext:

_runner: Runner | None = None

_enable_broadcast: bool = False
_broadcast_addr: str = "127.0.0.1"
_broadcast_port: int = 3238

_instance: ClassVar[DaftContext | None] = None
_lock: ClassVar[threading.Lock] = threading.Lock()

Expand Down Expand Up @@ -197,6 +201,31 @@ def get_context() -> DaftContext:
return _DaftContext


def broadcast_metrics(
enable: bool = True,
addr: str = "127.0.0.1",
port: int = 3238,
) -> DaftContext:
"""Enable (or disable) the broadcasting of Daft metrics over the given address.

This enables the Daft Dashboard to collect and display the data which is sent to it as Daft executes queries.

Args:
enable: Whether or not to enable broadcasting Daft metrics.
addr: The IP address on which to broadcast the metrics.
port: The port on which to broadcast the metrics.

Returns:
DaftContext: Daft context after enabling or disabling metrics broadcasting.
"""
ctx = get_context()
with ctx._lock:
ctx._enable_broadcast = enable
ctx._broadcast_addr = addr
ctx._broadcast_port = port
return ctx


def set_runner_ray(
address: str | None = None,
noop_if_initialized: bool = False,
Expand Down
76 changes: 71 additions & 5 deletions daft/dataframe/dataframe.py
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@
import typing
import warnings
from dataclasses import dataclass
from datetime import datetime, timezone
from functools import partial, reduce
from typing import (
TYPE_CHECKING,
Expand All @@ -28,6 +29,7 @@
TypeVar,
Union,
)
from uuid import uuid4

from daft.api_annotations import DataframePublicAPI
from daft.context import get_context
Expand Down Expand Up @@ -65,6 +67,17 @@
ManyColumnsInputType = Union[ColumnInputType, Iterable[ColumnInputType]]


def BroadcastMetrics(func):
"""Calls `self._explain_broadcast()` prior to executing the rest of the function."""

def wrapper(self, *args, **kwargs):
result = func(self, *args, **kwargs)
self._explain_broadcast()
return result

return wrapper


def to_logical_plan_builder(*parts: MicroPartition) -> LogicalPlanBuilder:
"""Creates a Daft DataFrame from a single Table.

Expand Down Expand Up @@ -158,6 +171,44 @@ def _result(self) -> Optional[PartitionSet]:
else:
return self._result_cache.value

def _explain_broadcast(self):
"""Broadcast the mermaid-formatted plan on the given port (assuming metrics-broadcasting is enabled)."""
import requests

from daft.dataframe.display import MermaidFormatter

ctx = get_context()
if not ctx._enable_broadcast:
return

addr = ctx._broadcast_addr
port = ctx._broadcast_port
is_cached = self._result_cache is not None
plan_time_start = datetime.now(timezone.utc)
mermaid_formatter = MermaidFormatter(builder=self.__builder, show_all=True, simple=False, is_cached=is_cached)
mermaid_plan: str = mermaid_formatter._repr_markdown_()
plan_time_end = datetime.now(timezone.utc)

try:
id = uuid4()
requests.post(
f"http://{addr}:{port}",
json={
"id": str(id),
"mermaid-plan": mermaid_plan,
"plan-time-start": str(plan_time_start),
"plan-time-end": str(plan_time_end),
},
)
print(f"Query ID: {id}")
except requests.exceptions.ConnectionError as conn_error:
warnings.warn(
"Unable to broadcast daft query plan over http."
" Are you sure the dashboard (and proxy server) are running?"
)
raise conn_error

@BroadcastMetrics
@DataframePublicAPI
def explain(
self, show_all: bool = False, format: str = "ascii", simple: bool = False, file: Optional[io.IOBase] = None
Expand Down Expand Up @@ -592,6 +643,7 @@ def _from_schema(cls, schema: Schema) -> "DataFrame":
# Write methods
###

@BroadcastMetrics
@DataframePublicAPI
def write_parquet(
self,
Expand Down Expand Up @@ -671,6 +723,7 @@ def write_parquet(
}
)

@BroadcastMetrics
@DataframePublicAPI
def write_csv(
self,
Expand Down Expand Up @@ -742,6 +795,7 @@ def write_csv(
}
)

@BroadcastMetrics
@DataframePublicAPI
def write_iceberg(
self, table: "pyiceberg.table.Table", mode: str = "append", io_config: Optional[IOConfig] = None
Expand Down Expand Up @@ -892,6 +946,7 @@ def write_iceberg(
# This is due to the fact that the logical plan of the write_iceberg returns datafiles but we want to return the above data
return from_pydict(with_operations)

@BroadcastMetrics
@DataframePublicAPI
def write_deltalake(
self,
Expand Down Expand Up @@ -1103,6 +1158,7 @@ def write_deltalake(

return with_operations

@BroadcastMetrics
@DataframePublicAPI
def write_lance(
self,
Expand Down Expand Up @@ -2308,9 +2364,9 @@ def transform(self, func: Callable[..., "DataFrame"], *args: Any, **kwargs: Any)
DataFrame: Transformed DataFrame.
"""
result = func(self, *args, **kwargs)
assert isinstance(result, DataFrame), (
f"Func returned an instance of type [{type(result)}], " "should have been DataFrame."
)
assert isinstance(
result, DataFrame
), f"Func returned an instance of type [{type(result)}], should have been DataFrame."
return result

def _agg(
Expand Down Expand Up @@ -2588,7 +2644,11 @@ def groupby(self, *group_by: ManyColumnsInputType) -> "GroupedDataFrame":
>>> import daft
>>> from daft import col
>>> df = daft.from_pydict(
... {"pet": ["cat", "dog", "dog", "cat"], "age": [1, 2, 3, 4], "name": ["Alex", "Jordan", "Sam", "Riley"]}
... {
... "pet": ["cat", "dog", "dog", "cat"],
... "age": [1, 2, 3, 4],
... "name": ["Alex", "Jordan", "Sam", "Riley"],
... }
... )
>>> grouped_df = df.groupby("pet").agg(
... col("age").min().alias("min_age"),
Expand Down Expand Up @@ -2804,6 +2864,7 @@ def _materialize_results(self) -> None:
assert result is not None
result.wait()

@BroadcastMetrics
@DataframePublicAPI
def collect(self, num_preview_rows: Optional[int] = 8) -> "DataFrame":
"""Executes the entire DataFrame and materializes the results.
Expand Down Expand Up @@ -2875,6 +2936,7 @@ def _construct_show_display(self, n: int) -> "DataFrameDisplay":

return DataFrameDisplay(preview, self.schema(), num_rows=n)

@BroadcastMetrics
@DataframePublicAPI
def show(self, n: int = 8) -> None:
"""Executes enough of the DataFrame in order to display the first ``n`` rows.
Expand Down Expand Up @@ -3358,7 +3420,11 @@ def agg(self, *to_agg: Union[Expression, Iterable[Expression]]) -> "DataFrame":
>>> import daft
>>> from daft import col
>>> df = daft.from_pydict(
... {"pet": ["cat", "dog", "dog", "cat"], "age": [1, 2, 3, 4], "name": ["Alex", "Jordan", "Sam", "Riley"]}
... {
... "pet": ["cat", "dog", "dog", "cat"],
... "age": [1, 2, 3, 4],
... "name": ["Alex", "Jordan", "Sam", "Riley"],
... }
... )
>>> grouped_df = df.groupby("pet").agg(
... col("age").min().alias("min_age"),
Expand Down
Loading