Skip to content

Commit 662b24e

Browse files
committed
implement basic watermarking
1 parent 41b141f commit 662b24e

File tree

3 files changed

+67
-36
lines changed

3 files changed

+67
-36
lines changed

ray_beam_runner/portability/context_management.py

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -163,15 +163,18 @@ def setup(self):
163163
pcoll_id,
164164
self.execution_context.safe_coders.get(coder_id, coder_id),
165165
)
166+
166167
elif transform.spec.urn == bundle_processor.DATA_OUTPUT_URN:
167168
data_output[transform.unique_name] = pcoll_id
168169
coder_id = self.execution_context.data_channel_coders[
169170
translations.only_element(transform.inputs.values())
170171
]
171172
else:
172173
raise NotImplementedError
173-
data_spec = beam_fn_api_pb2.RemoteGrpcPort(coder_id=coder_id)
174-
transform.spec.payload = data_spec.SerializeToString()
174+
if pcoll_id != translations.IMPULSE_BUFFER:
175+
data_spec = beam_fn_api_pb2.RemoteGrpcPort(coder_id=coder_id)
176+
transform.spec.payload = data_spec.SerializeToString()
177+
175178
elif transform.spec.urn in translations.PAR_DO_URNS:
176179
payload = proto_utils.parse_Bytes(
177180
transform.spec.payload, beam_runner_api_pb2.ParDoPayload

ray_beam_runner/portability/execution.py

Lines changed: 9 additions & 23 deletions
Original file line numberDiff line numberDiff line change
@@ -348,20 +348,9 @@ def clear(self, pcoll):
348348

349349
@ray.remote
350350
class RayWatermarkManager(watermark_manager.WatermarkManager):
351-
def __init__(self):
352-
# the original WatermarkManager performs a lot of computation
353-
# in its __init__ method. Because Ray calls __init__ whenever
354-
# it deserializes an object, we'll move its setup elsewhere.
355-
self._initialized = False
356-
self._pcollections_by_name = {}
357-
self._stages_by_name = {}
358-
359-
def setup(self, stages):
360-
if self._initialized:
361-
return
362-
logging.debug("initialized the RayWatermarkManager")
363-
self._initialized = True
364-
watermark_manager.WatermarkManager.setup(self, stages)
351+
def set_pcoll_produced_watermark(self, name, watermark):
352+
element = self._pcollections_by_name[name]
353+
element.set_produced_watermark(watermark)
365354

366355

367356
class RayRunnerExecutionContext(object):
@@ -374,6 +363,7 @@ def __init__(
374363
state_servicer: Optional[RayStateManager] = None,
375364
worker_manager: Optional[RayWorkerHandlerManager] = None,
376365
pcollection_buffers: PcollectionBufferManager = None,
366+
watermark_manager: Optional[RayWatermarkManager] = None,
377367
) -> None:
378368
ray.util.register_serializer(
379369
beam_runner_api_pb2.Components,
@@ -408,7 +398,9 @@ def __init__(
408398
for t in s.transforms
409399
if t.spec.urn == bundle_processor.DATA_INPUT_URN
410400
}
411-
self._watermark_manager = RayWatermarkManager.remote()
401+
self.watermark_manager = watermark_manager or RayWatermarkManager.remote(
402+
self.stages
403+
)
412404
self.pipeline_context = pipeline_context.PipelineContext(pipeline_components)
413405
self.safe_windowing_strategies = {
414406
# TODO: Enable safe_windowing_strategy after
@@ -422,14 +414,6 @@ def __init__(
422414
self.worker_manager = worker_manager or RayWorkerHandlerManager()
423415
self.timer_coder_ids = self._build_timer_coders_id_map()
424416

425-
@property
426-
def watermark_manager(self):
427-
# We don't need to wait for this line to execute with ray.get,
428-
# because any further calls to the watermark manager actor will
429-
# have to wait for it.
430-
self._watermark_manager.setup.remote(self.stages)
431-
return self._watermark_manager
432-
433417
@staticmethod
434418
def next_uid():
435419
# TODO(pabloem): Use stats actor for UIDs.
@@ -468,6 +452,7 @@ def __reduce__(self):
468452
self.state_servicer,
469453
self.worker_manager,
470454
self.pcollection_buffers,
455+
self.watermark_manager,
471456
)
472457

473458
def deserializer(*args):
@@ -479,6 +464,7 @@ def deserializer(*args):
479464
args[4],
480465
args[5],
481466
args[6],
467+
args[7],
482468
)
483469

484470
return (deserializer, data)

ray_beam_runner/portability/ray_fn_runner.py

Lines changed: 53 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -228,7 +228,9 @@ def _run_stage(
228228
bundle_context_manager (execution.BundleContextManager): A description of
229229
the stage to execute, and its context.
230230
"""
231+
231232
bundle_context_manager.setup()
233+
232234
runner_execution_context.worker_manager.register_process_bundle_descriptor(
233235
bundle_context_manager.process_bundle_descriptor
234236
)
@@ -247,6 +249,8 @@ def _run_stage(
247249
for k in bundle_context_manager.transform_to_buffer_coder
248250
}
249251

252+
watermark_manager = runner_execution_context.watermark_manager
253+
250254
final_result = None # type: Optional[beam_fn_api_pb2.InstructionResponse]
251255

252256
while True:
@@ -263,19 +267,28 @@ def _run_stage(
263267

264268
final_result = merge_stage_results(final_result, last_result)
265269
if not delayed_applications and not fired_timers:
270+
# Processing has completed; marking all outputs as completed
271+
# TODO: why is it necessary to set both the watermark and produced_watermark?
272+
# How do they interact?
273+
for output_pc in bundle_outputs:
274+
_, update_output_pc = translations.split_buffer_id(output_pc)
275+
watermark_manager.set_pcoll_produced_watermark.remote(
276+
update_output_pc, timestamp.MAX_TIMESTAMP
277+
)
266278
break
267279
else:
268-
# TODO: Enable following assertion after watermarking is implemented
269-
# assert (ray.get(
270-
# runner_execution_context.watermark_manager
271-
# .get_stage_node.remote(
272-
# bundle_context_manager.stage.name)).output_watermark()
273-
# < timestamp.MAX_TIMESTAMP), (
274-
# 'wrong timestamp for %s. '
275-
# % ray.get(
276-
# runner_execution_context.watermark_manager
277-
# .get_stage_node.remote(
278-
# bundle_context_manager.stage.name)))
280+
assert (
281+
ray.get(
282+
watermark_manager.get_stage_node.remote(
283+
bundle_context_manager.stage.name
284+
)
285+
).output_watermark()
286+
< timestamp.MAX_TIMESTAMP
287+
), "wrong timestamp for %s. " % ray.get(
288+
watermark_manager.get_stage_node.remote(
289+
bundle_context_manager.stage.name
290+
)
291+
)
279292
input_data = delayed_applications
280293
input_timers = fired_timers
281294

@@ -289,6 +302,20 @@ def _run_stage(
289302
# TODO(pabloem): Make sure that side inputs are being stored somewhere.
290303
# runner_execution_context.commit_side_inputs_to_state(data_side_input)
291304

305+
# assert that the output watermark was correctly set for this stage
306+
stage_node = ray.get(
307+
runner_execution_context.watermark_manager.get_stage_node.remote(
308+
bundle_context_manager.stage.name
309+
)
310+
)
311+
assert (
312+
stage_node.output_watermark() == timestamp.MAX_TIMESTAMP
313+
), "wrong output watermark for %s. Expected %s, but got %s." % (
314+
stage_node,
315+
timestamp.MAX_TIMESTAMP,
316+
stage_node.output_watermark(),
317+
)
318+
292319
return final_result
293320

294321
def _run_bundle(
@@ -352,6 +379,21 @@ def _run_bundle(
352379
# coder_impl=bundle_context_manager.get_input_coder_impl(
353380
# other_input))
354381

382+
# TODO: fill expected timers and pcolls with da
383+
watermark_updates = fn_runner.FnApiRunner._build_watermark_updates(
384+
runner_execution_context,
385+
transform_to_buffer_coder.keys(),
386+
bundle_context_manager.stage_timers.keys(), # expected_timers
387+
set(), # pcolls_with_da
388+
delayed_applications.keys(),
389+
watermarks_by_transform_and_timer_family,
390+
)
391+
392+
for pc_name, watermark in watermark_updates.items():
393+
runner_execution_context.watermark_manager.set_pcoll_watermark.remote(
394+
pc_name, watermark
395+
)
396+
355397
return result, newly_set_timers, delayed_applications, output
356398

357399
@staticmethod

0 commit comments

Comments
 (0)