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

Ensuring tests can run without passing context around #40

Merged
merged 1 commit into from
Sep 12, 2022
Merged
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
5 changes: 1 addition & 4 deletions ray_beam_runner/portability/context_management.py
Original file line number Diff line number Diff line change
Expand Up @@ -33,11 +33,8 @@
from apache_beam.runners.worker import bundle_processor
from apache_beam.utils import proto_utils

import ray
from ray_beam_runner.portability.execution import RayRunnerExecutionContext

ENCODED_IMPULSE_REFERENCE = ray.put([fn_execution.ENCODED_IMPULSE_VALUE])


class RayBundleContextManager:
def __init__(
Expand Down Expand Up @@ -155,7 +152,7 @@ def setup(self):
if pcoll_id == translations.IMPULSE_BUFFER:
pcoll_id = transform.unique_name.encode("utf8")
self.execution_context.pcollection_buffers.put.remote(
pcoll_id, [ENCODED_IMPULSE_REFERENCE]
pcoll_id, [self.execution_context.encoded_impulse_ref]
)
else:
pass
Expand Down
1 change: 1 addition & 0 deletions ray_beam_runner/portability/execution.py
Original file line number Diff line number Diff line change
Expand Up @@ -421,6 +421,7 @@ def __init__(
self._uid = 0
self.worker_manager = worker_manager or RayWorkerHandlerManager()
self.timer_coder_ids = self._build_timer_coders_id_map()
self.encoded_impulse_ref = ray.put([fn_execution.ENCODED_IMPULSE_VALUE])

@property
def watermark_manager(self):
Expand Down
26 changes: 15 additions & 11 deletions ray_beam_runner/portability/ray_runner_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -94,10 +94,12 @@ def contains_labels(mi, labels):


class RayFnApiRunnerTest(unittest.TestCase):
@classmethod
def setUpClass(cls) -> None:
def setUp(self) -> None:
if not ray.is_initialized():
ray.init()
ray.init(num_cpus=1, include_dashboard=False)

def tearDown(self) -> None:
ray.shutdown()

def create_pipeline(self, is_drain=False):
return beam.Pipeline(
Expand Down Expand Up @@ -1210,10 +1212,12 @@ def test_pack_combiners(self):
# the sampling counter.
@unittest.skip("Metrics not yet supported.")
class RayRunnerMetricsTest(unittest.TestCase):
@classmethod
def setUpClass(cls) -> None:
def setUp(self) -> None:
if not ray.is_initialized():
ray.init()
ray.init(num_cpus=1, include_dashboard=False)

def tearDown(self) -> None:
ray.shutdown()

def assert_has_counter(self, mon_infos, urn, labels, value=None, ge_value=None):
found = 0
Expand Down Expand Up @@ -1631,10 +1635,12 @@ def has_mi_for_ptransform(mon_infos, ptransform):

@unittest.skip("Runner-initiated splitting not yet supported")
class RayRunnerSplitTest(unittest.TestCase):
@classmethod
def setUpClass(cls) -> None:
def setUp(self) -> None:
if not ray.is_initialized():
ray.init()
ray.init(num_cpus=1, include_dashboard=False)

def tearDown(self) -> None:
ray.shutdown()

def create_pipeline(self, is_drain=False):
return beam.Pipeline(
Expand Down Expand Up @@ -2073,8 +2079,6 @@ def process(self, element, *side_inputs):
yield self._name


logging.getLogger().setLevel(logging.INFO)

if __name__ == "__main__":
logging.getLogger().setLevel(logging.INFO)
unittest.main()