|
1 | 1 | from __future__ import annotations |
2 | 2 |
|
| 3 | +import json |
3 | 4 | import logging |
| 5 | +import os |
| 6 | +import sys |
4 | 7 | import time |
5 | 8 | import uuid |
6 | 9 | from collections.abc import Generator, Iterable, Iterator |
|
45 | 48 | FileFormatConfig, |
46 | 49 | FileInfos, |
47 | 50 | IOConfig, |
| 51 | + PyQueryMetadata, |
| 52 | + PyQueryResult, |
| 53 | + QueryEndState, |
48 | 54 | ) |
49 | 55 | from daft.datatype import DataType |
50 | 56 | from daft.filesystem import glob_path_with_stats |
@@ -548,19 +554,84 @@ def run_iter( |
548 | 554 | ctx = get_context() |
549 | 555 | query_id = str(uuid.uuid4()) |
550 | 556 | daft_execution_config = ctx.daft_execution_config |
| 557 | + output_schema = builder.schema() |
551 | 558 |
|
552 | | - # Optimize the logical plan. |
553 | | - builder = builder.optimize(daft_execution_config) |
554 | | - |
555 | | - distributed_plan = DistributedPhysicalPlan.from_logical_plan_builder( |
556 | | - builder._builder, query_id, daft_execution_config |
| 559 | + # Notify query start |
| 560 | + ray_dashboard_url = None |
| 561 | + try: |
| 562 | + ray_dashboard_url = ray.worker.get_dashboard_url() |
| 563 | + if ray_dashboard_url: |
| 564 | + if not ray_dashboard_url.startswith("http"): |
| 565 | + ray_dashboard_url = f"http://{ray_dashboard_url}" |
| 566 | + |
| 567 | + # Try to append Job ID |
| 568 | + try: |
| 569 | + job_id = ray.get_runtime_context().get_job_id() |
| 570 | + if job_id: |
| 571 | + ray_dashboard_url = f"{ray_dashboard_url}/#/jobs/{job_id}" |
| 572 | + except Exception: |
| 573 | + pass |
| 574 | + except Exception: |
| 575 | + pass |
| 576 | + |
| 577 | + entrypoint = "python " + " ".join(sys.argv) |
| 578 | + |
| 579 | + ctx._notify_query_start( |
| 580 | + query_id, |
| 581 | + PyQueryMetadata( |
| 582 | + output_schema._schema, builder.repr_json(), "Ray (Flotilla)", ray_dashboard_url, entrypoint |
| 583 | + ), |
557 | 584 | ) |
558 | | - if self.flotilla_plan_runner is None: |
559 | | - self.flotilla_plan_runner = FlotillaRunner() |
| 585 | + ctx._notify_optimization_start(query_id) |
560 | 586 |
|
561 | | - yield from self.flotilla_plan_runner.stream_plan( |
562 | | - distributed_plan, self._part_set_cache.get_all_partition_sets() |
563 | | - ) |
| 587 | + # Log Dashboard URL if configured |
| 588 | + dashboard_url = os.environ.get("DAFT_DASHBOARD_URL") |
| 589 | + if dashboard_url: |
| 590 | + print(f"Daft Dashboard: {dashboard_url}/query/{query_id}") |
| 591 | + |
| 592 | + try: |
| 593 | + # Optimize the logical plan. |
| 594 | + builder = builder.optimize(daft_execution_config) |
| 595 | + ctx._notify_optimization_end(query_id, builder.repr_json()) |
| 596 | + |
| 597 | + distributed_plan = DistributedPhysicalPlan.from_logical_plan_builder( |
| 598 | + builder._builder, query_id, daft_execution_config |
| 599 | + ) |
| 600 | + physical_plan_json = distributed_plan.repr_json() |
| 601 | + ctx._notify_exec_start(query_id, physical_plan_json) |
| 602 | + |
| 603 | + if self.flotilla_plan_runner is None: |
| 604 | + self.flotilla_plan_runner = FlotillaRunner() |
| 605 | + |
| 606 | + total_rows = 0 |
| 607 | + for result in self.flotilla_plan_runner.stream_plan( |
| 608 | + distributed_plan, self._part_set_cache.get_all_partition_sets() |
| 609 | + ): |
| 610 | + if result.metadata() is not None: |
| 611 | + total_rows += result.metadata().num_rows |
| 612 | + yield result |
| 613 | + |
| 614 | + # Mark all operators as finished to clean up the Dashboard UI before notify_exec_end |
| 615 | + try: |
| 616 | + plan_dict = json.loads(physical_plan_json) |
| 617 | + |
| 618 | + def notify_end(node: dict[str, Any]) -> None: |
| 619 | + if "id" in node: |
| 620 | + ctx._notify_exec_operator_end(query_id, node["id"]) |
| 621 | + if "children" in node: |
| 622 | + for child in node["children"]: |
| 623 | + notify_end(child) |
| 624 | + |
| 625 | + notify_end(plan_dict) |
| 626 | + except Exception: |
| 627 | + pass |
| 628 | + |
| 629 | + ctx._notify_exec_end(query_id) |
| 630 | + ctx._notify_query_end(query_id, PyQueryResult(QueryEndState.Finished, "")) |
| 631 | + |
| 632 | + except Exception as e: |
| 633 | + ctx._notify_query_end(query_id, PyQueryResult(QueryEndState.Failed, str(e))) |
| 634 | + raise |
564 | 635 |
|
565 | 636 | def run_iter_tables( |
566 | 637 | self, builder: LogicalPlanBuilder, results_buffer_size: int | None = None |
|
0 commit comments