Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Commit 4b2f31f

Browse files
author
Boyuan Zhang
committed
Add per_element_output_counter to python SDK
1 parent 2708329 commit 4b2f31f

6 files changed

Lines changed: 57 additions & 6 deletions

File tree

sdks/python/apache_beam/runners/common.pxd

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,9 @@ cimport cython
1919

2020
from apache_beam.utils.windowed_value cimport WindowedValue
2121
from apache_beam.metrics.execution cimport ScopedMetricsContainer
22+
from apache_beam.transforms.cy_dataflow_distribution_counter cimport DataflowDistributionCounter
23+
24+
from libc.stdint cimport int64_t
2225

2326

2427
cdef type TaggedOutput, TimestampedValue
@@ -87,15 +90,16 @@ cdef class DoFnRunner(Receiver):
8790

8891

8992
cdef class OutputProcessor(object):
90-
@cython.locals(windowed_value=WindowedValue)
93+
@cython.locals(windowed_value=WindowedValue,
94+
output_element_count=int64_t)
9195
cpdef process_outputs(self, WindowedValue element, results)
9296

9397

9498
cdef class _OutputProcessor(OutputProcessor):
9599
cdef object window_fn
96100
cdef Receiver main_receivers
97101
cdef object tagged_receivers
98-
102+
cdef DataflowDistributionCounter per_element_output_counter
99103

100104
cdef class DoFnContext(object):
101105
cdef object label

sdks/python/apache_beam/runners/common.py

Lines changed: 42 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -28,13 +28,16 @@
2828
import six
2929

3030
from apache_beam.internal import util
31+
from apache_beam.options.value_provider import RuntimeValueProvider
3132
from apache_beam.pvalue import TaggedOutput
3233
from apache_beam.transforms import DoFn
3334
from apache_beam.transforms import core
3435
from apache_beam.transforms.core import RestrictionProvider
3536
from apache_beam.transforms.window import GlobalWindow
3637
from apache_beam.transforms.window import TimestampedValue
3738
from apache_beam.transforms.window import WindowFn
39+
from apache_beam.utils.counters import Counter
40+
from apache_beam.utils.counters import CounterName
3841
from apache_beam.utils.windowed_value import WindowedValue
3942

4043

@@ -518,7 +521,8 @@ def __init__(self,
518521
step_name=None,
519522
logging_context=None,
520523
state=None,
521-
scoped_metrics_container=None):
524+
scoped_metrics_container=None,
525+
operation_name=None):
522526
"""Initializes a DoFnRunner.
523527
524528
Args:
@@ -532,6 +536,7 @@ def __init__(self,
532536
logging_context: a LoggingContext object
533537
state: handle for accessing DoFn state
534538
scoped_metrics_container: Context switcher for metrics container
539+
operation_name: The system name assigned by the runner for this operation.
535540
"""
536541
# Need to support multiple iterations.
537542
side_inputs = list(side_inputs)
@@ -548,8 +553,22 @@ def __init__(self,
548553

549554
# Optimize for the common case.
550555
main_receivers = tagged_receivers[None]
556+
557+
# TODO(BEAM-3937): Remove if block after output counter released.
558+
experiments = RuntimeValueProvider.get_value('experiments', str, [])
559+
# Experimental flag format: experimental-name_version-number.
560+
if 'outputs_per_element_counter_v0' in experiments:
561+
# TODO(BEAM-3955): Make step_name and operation_name less confused.
562+
output_counter_name = (CounterName('per-element-output-count',
563+
step_name=operation_name))
564+
per_element_output_counter = state._counter_factory.get_counter(
565+
output_counter_name, Counter.DATAFLOW_DISTRIBUTION).accumulator
566+
else:
567+
per_element_output_counter = None
568+
551569
output_processor = _OutputProcessor(
552-
windowing.windowfn, main_receivers, tagged_receivers)
570+
windowing.windowfn, main_receivers, tagged_receivers,
571+
per_element_output_counter)
553572

554573
self.do_fn_invoker = DoFnInvoker.create_invoker(
555574
do_fn_signature, output_processor, self.context, side_inputs, args,
@@ -617,17 +636,24 @@ def process_outputs(self, windowed_input_element, results):
617636
class _OutputProcessor(OutputProcessor):
618637
"""Processes output produced by DoFn method invocations."""
619638

620-
def __init__(self, window_fn, main_receivers, tagged_receivers):
639+
def __init__(self,
640+
window_fn,
641+
main_receivers,
642+
tagged_receivers,
643+
per_element_output_counter):
621644
"""Initializes ``_OutputProcessor``.
622645
623646
Args:
624647
window_fn: a windowing function (WindowFn).
625648
main_receivers: a dict of tag name to Receiver objects.
626649
tagged_receivers: main receiver object.
650+
per_element_output_counter: per_element_output_counter of one work_item.
651+
could be none if experimental flag turn off
627652
"""
628653
self.window_fn = window_fn
629654
self.main_receivers = main_receivers
630655
self.tagged_receivers = tagged_receivers
656+
self.per_element_output_counter = per_element_output_counter
631657

632658
def process_outputs(self, windowed_input_element, results):
633659
"""Dispatch the result of process computation to the appropriate receivers.
@@ -636,9 +662,17 @@ def process_outputs(self, windowed_input_element, results):
636662
then dispatched to the appropriate indexed output.
637663
"""
638664
if results is None:
665+
# TODO(BEAM-3937): Remove if block after output counter released.
666+
# Only enable per_element_output_counter when counter cythonized.
667+
if (self.per_element_output_counter is not None and
668+
self.per_element_output_counter.is_cythonized):
669+
self.per_element_output_counter.add_input(0)
639670
return
640671

672+
output_element_count = 0
641673
for result in results:
674+
# results here may be a generator, which cannot call len on it.
675+
output_element_count += 1
642676
tag = None
643677
if isinstance(result, TaggedOutput):
644678
tag = result.tag
@@ -663,6 +697,11 @@ def process_outputs(self, windowed_input_element, results):
663697
self.main_receivers.receive(windowed_value)
664698
else:
665699
self.tagged_receivers[tag].receive(windowed_value)
700+
# TODO(BEAM-3937): Remove if block after output counter released.
701+
# Only enable per_element_output_counter when counter cythonized
702+
if (self.per_element_output_counter is not None and
703+
self.per_element_output_counter.is_cythonized):
704+
self.per_element_output_counter.add_input(output_element_count)
666705

667706
def start_bundle_outputs(self, results):
668707
"""Validate that start_bundle does not output any elements"""

sdks/python/apache_beam/runners/worker/operations.py

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -391,7 +391,8 @@ def start(self):
391391
logging_context=logger.PerThreadLoggingContext(
392392
step_name=self.name_context.logging_name()),
393393
state=state,
394-
scoped_metrics_container=None)
394+
scoped_metrics_container=None,
395+
operation_name=self.name_context.metrics_name())
395396
self.dofn_receiver = (self.dofn_runner
396397
if isinstance(self.dofn_runner, Receiver)
397398
else DoFnRunnerReceiver(self.dofn_runner))

sdks/python/apache_beam/transforms/cy_dataflow_distribution_counter.pxd

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -42,3 +42,4 @@ cdef class DataflowDistributionCounter(object):
4242
cpdef void translate_to_histogram(self, histogram)
4343
cpdef bint add_inputs_for_test(self, elements) except -1
4444
cpdef int64_t calculate_bucket_index(self, int64_t element)
45+
cpdef merge(self, accumulators)

sdks/python/apache_beam/transforms/cy_dataflow_distribution_counter.pyx

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -133,3 +133,6 @@ cdef class DataflowDistributionCounter(object):
133133
cdef calculate_bucket_index cannot be called directly from def.
134134
"""
135135
return self._fast_calculate_bucket_index(element)
136+
137+
cpdef merge(self, accumulators):
138+
pass

sdks/python/apache_beam/transforms/py_dataflow_distribution_counter.py

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -113,3 +113,6 @@ def translate_to_histogram(self, histogram):
113113
histogram.firstBucketOffset = first_bucket_offset
114114
histogram.bucketCounts = (
115115
self.buckets[first_bucket_offset:last_bucket_offset + 1])
116+
117+
def merge(self, accumulators):
118+
pass

0 commit comments

Comments
 (0)