[ https://issues.apache.org/jira/browse/BEAM-3042?focusedWorklogId=112379&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-112379 ]
ASF GitHub Bot logged work on BEAM-3042: ---------------------------------------- Author: ASF GitHub Bot Created on: 15/Jun/18 16:30 Start Date: 15/Jun/18 16:30 Worklog Time Spent: 10m Work Description: pabloem closed pull request #5309: [BEAM-3042] Adding time tracking of batch side inputs [low priority] URL: https://github.com/apache/beam/pull/5309 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/sdks/python/apache_beam/runners/worker/opcounters.pxd b/sdks/python/apache_beam/runners/worker/opcounters.pxd index 0bcd42848d2..1d7f296c5ce 100644 --- a/sdks/python/apache_beam/runners/worker/opcounters.pxd +++ b/sdks/python/apache_beam/runners/worker/opcounters.pxd @@ -19,13 +19,14 @@ cimport cython cimport libc.stdint from apache_beam.utils.counters cimport Counter +from apache_beam.runners.worker cimport statesampler_fast cdef class TransformIOCounter(object): cdef readonly object _counter_factory cdef readonly object _state_sampler cdef Counter bytes_read_counter - cdef object scoped_state + cdef statesampler_fast.ScopedState scoped_state cdef object _latest_step cpdef update_current_step(self) diff --git a/sdks/python/apache_beam/runners/worker/operations.py b/sdks/python/apache_beam/runners/worker/operations.py index f38283741b5..9aa29b87aa0 100644 --- a/sdks/python/apache_beam/runners/worker/operations.py +++ b/sdks/python/apache_beam/runners/worker/operations.py @@ -325,7 +325,7 @@ def _read_side_inputs(self, tags_and_types): sources.append(si.source) # The tracking of time spend reading and bytes read from side inputs is # behind an experiment flag to test its performance impact. - if 'sideinput_io_metrics' in RuntimeValueProvider.experiments: + if 'sideinput_io_metrics_v2' in RuntimeValueProvider.experiments: si_counter = opcounters.SideInputReadCounter( self.counter_factory, self.state_sampler, diff --git a/sdks/python/apache_beam/runners/worker/sideinputs.py b/sdks/python/apache_beam/runners/worker/sideinputs.py index c36182da95d..980a2088c7b 100644 --- a/sdks/python/apache_beam/runners/worker/sideinputs.py +++ b/sdks/python/apache_beam/runners/worker/sideinputs.py @@ -80,7 +80,7 @@ def __init__(self, def add_byte_counter(self, reader): """Adds byte counter observer to a side input reader. - If the 'sideinput_io_metrics' experiment flag is not passed in, then + If the 'sideinput_io_metrics_v2' experiment flag is not passed in, then nothing is attached to the reader. Args: @@ -123,7 +123,7 @@ def _reader_thread(self): # The tracking of time spend reading and bytes read from side # inputs is kept behind an experiment flag to test performance # impact. - if 'sideinput_io_metrics' in RuntimeValueProvider.experiments: + if 'sideinput_io_metrics_v2' in RuntimeValueProvider.experiments: self.add_byte_counter(reader) returns_windowed_values = reader.returns_windowed_values for value in reader: @@ -159,7 +159,8 @@ def __iter__(self): try: while True: try: - element = self.element_queue.get() + with self.read_counter: + element = self.element_queue.get() if element is READER_THREAD_IS_DONE_SENTINEL: num_readers_finished += 1 if num_readers_finished == self.num_reader_threads: diff --git a/sdks/python/apache_beam/runners/worker/sideinputs_test.py b/sdks/python/apache_beam/runners/worker/sideinputs_test.py index 050ecdc5003..c4240dd028c 100644 --- a/sdks/python/apache_beam/runners/worker/sideinputs_test.py +++ b/sdks/python/apache_beam/runners/worker/sideinputs_test.py @@ -92,7 +92,7 @@ def test_bytes_read_behind_experiment(self): def test_bytes_read_are_reported(self): RuntimeValueProvider.set_runtime_options( - {'experiments': ['sideinput_io_metrics', 'other']}) + {'experiments': ['sideinput_io_metrics_v2', 'other']}) mock_read_counter = mock.MagicMock() source_records = ['a', 'b', 'c', 'd'] sources = [ diff --git a/sdks/python/apache_beam/runners/worker/statesampler_fast.pxd b/sdks/python/apache_beam/runners/worker/statesampler_fast.pxd new file mode 100644 index 00000000000..76b379b7a11 --- /dev/null +++ b/sdks/python/apache_beam/runners/worker/statesampler_fast.pxd @@ -0,0 +1,61 @@ +# +# 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. +# + +cimport cython + +from apache_beam.metrics.execution cimport MetricsContainer + +from cpython cimport pythread +from libc.stdint cimport int32_t, int64_t + +cdef class StateSampler(object): + """Tracks time spent in states during pipeline execution.""" + cdef int _sampling_period_ms + cdef int _sampling_period_ms_start + cdef double _sampling_period_ratio + + cdef list scoped_states_by_index + + cdef public bint started + cdef public bint finished + cdef object sampling_thread + + # This lock guards members that are shared between threads, specificaly + # finished, scoped_states_by_index, and the nsecs field of each state therein. + cdef pythread.PyThread_type_lock lock + + cdef public int64_t state_transition_count + cdef public int64_t time_since_transition + + cdef int32_t current_state_index + + cpdef _scoped_state(self, counter_name, output_counter, metrics_container) + +cdef class ScopedState(object): + """Context manager class managing transitions for a given sampler state.""" + + cdef readonly StateSampler sampler + cdef readonly int32_t state_index + cdef readonly object counter + cdef readonly object name + cdef readonly int64_t _nsecs + cdef int32_t old_state_index + cdef readonly MetricsContainer _metrics_container + + cpdef __enter__(self) + + cpdef __exit__(self, unused_exc_type, unused_exc_value, unused_traceback) diff --git a/sdks/python/apache_beam/runners/worker/statesampler_fast.pyx b/sdks/python/apache_beam/runners/worker/statesampler_fast.pyx index 67d9cc8a856..fdf496979f9 100644 --- a/sdks/python/apache_beam/runners/worker/statesampler_fast.pyx +++ b/sdks/python/apache_beam/runners/worker/statesampler_fast.pyx @@ -71,24 +71,6 @@ cdef inline int64_t get_nsec_time() nogil: cdef class StateSampler(object): """Tracks time spent in states during pipeline execution.""" - cdef int _sampling_period_ms - cdef int _sampling_period_ms_start - cdef double _sampling_period_ratio - - cdef list scoped_states_by_index - - cdef public bint started - cdef public bint finished - cdef object sampling_thread - - # This lock guards members that are shared between threads, specificaly - # finished, scoped_states_by_index, and the nsecs field of each state therein. - cdef pythread.PyThread_type_lock lock - - cdef public int64_t state_transition_count - cdef public int64_t time_since_transition - - cdef int32_t current_state_index def __init__(self, sampling_period_ms, @@ -172,7 +154,7 @@ cdef class StateSampler(object): return self.scoped_states_by_index[self.current_state_index] cpdef _scoped_state(self, counter_name, output_counter, - metrics_container=None): + metrics_container): """Returns a context manager managing transitions for a given state. Args: counter_name: A CounterName object with information about the execution @@ -201,14 +183,6 @@ cdef class StateSampler(object): cdef class ScopedState(object): """Context manager class managing transitions for a given sampler state.""" - cdef readonly StateSampler sampler - cdef readonly int32_t state_index - cdef readonly object counter - cdef readonly object name - cdef readonly int64_t _nsecs - cdef int32_t old_state_index - cdef readonly MetricsContainer _metrics_container - def __init__( self, sampler, name, state_index, counter=None, metrics_container=None): self.sampler = sampler diff --git a/sdks/python/apache_beam/tools/sideinput_microbenchmark.py b/sdks/python/apache_beam/tools/sideinput_microbenchmark.py index 15e1b9bf33a..16202937282 100644 --- a/sdks/python/apache_beam/tools/sideinput_microbenchmark.py +++ b/sdks/python/apache_beam/tools/sideinput_microbenchmark.py @@ -48,6 +48,7 @@ def run_benchmark(num_runs=50, input_per_source=4000, num_sources=4): for i in range(num_runs): counter_factory = CounterFactory() state_sampler = statesampler.StateSampler('basic', counter_factory) + state_sampler.start() with state_sampler.scoped_state('step1', 'state'): si_counter = opcounters.SideInputReadCounter( counter_factory, state_sampler, 'step1', 1) @@ -61,6 +62,7 @@ def run_benchmark(num_runs=50, input_per_source=4000, num_sources=4): list(iterator_fn()) time_cost = time.time() - start times.append(time_cost) + state_sampler.stop() print("Runtimes:", times) ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 112379) Time Spent: 7h 10m (was: 7h) > Add tracking of bytes read / time spent when reading side inputs > ---------------------------------------------------------------- > > Key: BEAM-3042 > URL: https://issues.apache.org/jira/browse/BEAM-3042 > Project: Beam > Issue Type: Bug > Components: sdk-py-core > Reporter: Pablo Estrada > Assignee: Pablo Estrada > Priority: Major > Time Spent: 7h 10m > Remaining Estimate: 0h > > It is difficult for Dataflow users to understand how modifying a pipeline or > data set can affect how much inter-transform IO is used in their job. The > intent of this feature request is to help users understand how side inputs > behave when they are consumed. > This will allow users to understand how much time and how much data their > pipeline uses to read/write to inter-transform IO. Users will also be able to > modify their pipelines and understand how their changes affect these IO > metrics. > For further information, please review the internal Google doc > go/insights-transform-io-design-doc. -- This message was sent by Atlassian JIRA (v7.6.3#76005)