robtandy commented on code in PR #60:
URL: https://github.com/apache/datafusion-ray/pull/60#discussion_r1947205698


##########
datafusion_ray/core.py:
##########
@@ -0,0 +1,327 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+
+from collections import defaultdict
+from logging import error, debug, info
+import os
+import pyarrow as pa
+import asyncio
+import ray
+import uuid
+import time
+
+from datafusion_ray._datafusion_ray_internal import (
+    RayContext as RayContextInternal,
+    RayDataFrame as RayDataFrameInternal,
+    prettify,
+)
+
+
+def setup_logging():
+    import logging
+
+    logging.addLevelName(5, "TRACE")
+
+    log_level = os.environ.get("DATAFUSION_RAY_LOG_LEVEL", "WARN").upper()
+
+    # this logger gets captured and routed to rust.   See src/lib.rs
+    logging.getLogger("datafusion_ray").setLevel(log_level)
+
+
+setup_logging()
+
+_log_level = os.environ.get("DATAFUSION_RAY_LOG_LEVEL", "ERROR")
+runtime_env = {
+    "worker_process_setup_hook": setup_logging,
+    "env_vars": {"DATAFUSION_RAY_LOG_LEVEL": _log_level, 
"RAY_worker_niceness": "0"},
+}
+
+
+class RayDataFrame:
+    def __init__(
+        self,
+        ray_internal_df: RayDataFrameInternal,
+        query_id: str,
+        batch_size=8192,
+        isolate_parititions=False,
+        prefetch_buffer_size=0,
+    ):
+        self.df = ray_internal_df
+        self.query_id = query_id
+        self._stages = None
+        self._batches = None
+        self.batch_size = batch_size
+        self.isolate_partitions = isolate_parititions
+        self.prefetch_buffer_size = prefetch_buffer_size
+
+    def stages(self):
+        # create our coordinator now, which we need to create stages
+        if not self._stages:
+            self._stages = self.df.stages(
+                self.batch_size, self.isolate_partitions, 
self.prefetch_buffer_size
+            )
+
+            self.coord = RayStageCoordinator.options(
+                name="RayQueryCoordinator:" + self.query_id,
+            ).remote(
+                self.query_id,
+            )
+
+        return self._stages
+
+    def execution_plan(self):
+        return self.df.execution_plan()
+
+    def logical_plan(self):
+        return self.df.logical_plan()
+
+    def optimized_logical_plan(self):
+        return self.df.optimized_logical_plan()
+
+    def collect(self) -> list[pa.RecordBatch]:
+        if not self._batches:
+            t1 = time.time()
+            self.stages()
+            t2 = time.time()
+            debug(f"creating stages took {t2 -t1}s")
+
+            last_stage = max([stage.stage_id for stage in self._stages])
+            debug("last stage is", last_stage)
+
+            self.create_ray_stages()
+            t3 = time.time()
+            debug(f"creating ray stage actors took {t3 -t2}s")
+            self.run_stages()
+
+            addrs = ray.get(self.coord.get_stage_addrs.remote())
+
+            reader = self.df.read_final_stage(last_stage, addrs[last_stage][0])
+            self._batches = list(reader)
+            self.coord.all_done.remote()
+        return self._batches
+
+    def show(self) -> None:
+        batches = self.collect()
+        print(prettify(batches))
+
+    def create_ray_stages(self):
+
+        # if we are doing each partition separate (isolate_partitions =True)
+        # then the plan generated will include a PartitionIsolator which
+        # will take care of that.  Our job is to then launch a stage for each
+        # partition.
+        #
+        refs = []
+        for stage in self.stages():
+            num_shadows = stage.num_shadow_partitions()
+            if self.isolate_partitions and num_shadows:
+                debug(f"stage {stage.stage_id} has {num_shadows} shadows")
+                for shadow in range(num_shadows):
+                    refs.append(
+                        self.coord.new_stage.remote(
+                            stage.stage_id, stage.plan_bytes(), shadow
+                        )
+                    )
+            else:
+                # we are running each stage as its own actor
+                refs.append(
+                    self.coord.new_stage.remote(
+                        stage.stage_id,
+                        stage.plan_bytes(),
+                        shadow_partition=None,
+                    )
+                )
+
+        # wait for all stages to be created
+        ray.wait(refs, num_returns=len(refs))
+
+    def run_stages(self):
+        self.coord.serve.remote()
+
+
+class RayContext:
+    def __init__(
+        self,
+        batch_size: int = 8192,
+        isolate_partitions: bool = False,
+        prefetch_buffer_size: int = 0,
+    ) -> None:
+        self.ctx = RayContextInternal()
+        self.batch_size = batch_size
+        self.isolate_partitions = isolate_partitions
+        self.prefetch_buffer_size = prefetch_buffer_size
+
+    def register_parquet(self, name: str, path: str):
+        self.ctx.register_parquet(name, path)
+
+    def register_listing_table(self, name: str, path: str, 
file_extention="parquet"):
+        self.ctx.register_listing_table(name, path, file_extention)
+
+    def sql(self, query: str) -> RayDataFrame:
+        query_id = str(uuid.uuid4())
+
+        df = self.ctx.sql(query)
+        return RayDataFrame(
+            df,
+            query_id,
+            self.batch_size,
+            self.isolate_partitions,
+            self.prefetch_buffer_size,
+        )
+
+    def set(self, option: str, value: str) -> None:
+        self.ctx.set(option, value)
+
+
+@ray.remote(num_cpus=0)

Review Comment:
   All stages need to be running in order for the results to stream through the 
distributed plan.   Setting `num_cpus=0` ensures all Actors will be scheduled 
by Ray.   If we had a different value, Ray may choose to wait for available 
resources and we, at the moment, do not have a way of knowing a stage is 
waiting to be scheduled.
   
   I think future PRs will included better specification of the resources 
required per query.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org
For additional commands, e-mail: github-h...@datafusion.apache.org

Reply via email to