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

Skip to content

Commit 7a02887

Browse files
authored
Merge pull request #9843 from kamilwu/portable-runners-metrics
[BEAM-4775] Converting MonitoringInfos to MetricResults in PortableRunner
2 parents 1ccecd3 + f53e47a commit 7a02887

8 files changed

Lines changed: 152 additions & 69 deletions

File tree

sdks/python/apache_beam/metrics/monitoring_infos.py

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -319,6 +319,13 @@ def parse_namespace_and_name(monitoring_info_proto):
319319
return split[0], split[1]
320320

321321

322+
def get_step_name(monitoring_info_proto):
323+
"""Returns a step name for the given monitoring info or None if step name
324+
cannot be specified."""
325+
# Right now only metrics that have a PTRANSFORM are taken into account
326+
return monitoring_info_proto.labels.get(PTRANSFORM_LABEL)
327+
328+
322329
def to_key(monitoring_info_proto):
323330
"""Returns a key based on the URN and labels.
324331

sdks/python/apache_beam/runners/portability/fn_api_runner.py

Lines changed: 13 additions & 32 deletions
Original file line numberDiff line numberDiff line change
@@ -39,13 +39,12 @@
3939

4040
import apache_beam as beam # pylint: disable=ungrouped-imports
4141
from apache_beam import coders
42-
from apache_beam import metrics
4342
from apache_beam.coders.coder_impl import create_InputStream
4443
from apache_beam.coders.coder_impl import create_OutputStream
44+
from apache_beam.metrics import metric
4545
from apache_beam.metrics import monitoring_infos
46-
from apache_beam.metrics.execution import MetricKey
46+
from apache_beam.metrics.execution import MetricResult
4747
from apache_beam.metrics.execution import MetricsEnvironment
48-
from apache_beam.metrics.metricbase import MetricName
4948
from apache_beam.options import pipeline_options
5049
from apache_beam.options.value_provider import RuntimeValueProvider
5150
from apache_beam.portability import common_urns
@@ -62,6 +61,7 @@
6261
from apache_beam.runners import runner
6362
from apache_beam.runners.portability import artifact_service
6463
from apache_beam.runners.portability import fn_api_runner_transforms
64+
from apache_beam.runners.portability import portable_metrics
6565
from apache_beam.runners.portability.fn_api_runner_transforms import create_buffer_id
6666
from apache_beam.runners.portability.fn_api_runner_transforms import only_element
6767
from apache_beam.runners.portability.fn_api_runner_transforms import split_buffer_id
@@ -1883,53 +1883,34 @@ def get(self, timeout=None):
18831883
return self._response
18841884

18851885

1886-
class FnApiMetrics(metrics.metric.MetricResults):
1886+
class FnApiMetrics(metric.MetricResults):
18871887
def __init__(self, step_monitoring_infos, user_metrics_only=True):
18881888
"""Used for querying metrics from the PipelineResult object.
18891889
18901890
step_monitoring_infos: Per step metrics specified as MonitoringInfos.
1891-
use_monitoring_infos: If true, return the metrics based on the
1892-
step_monitoring_infos.
1891+
user_metrics_only: If true, includes user metrics only.
18931892
"""
18941893
self._counters = {}
18951894
self._distributions = {}
18961895
self._gauges = {}
18971896
self._user_metrics_only = user_metrics_only
1898-
self._init_metrics_from_monitoring_infos(step_monitoring_infos)
18991897
self._monitoring_infos = step_monitoring_infos
19001898

1901-
def _init_metrics_from_monitoring_infos(self, step_monitoring_infos):
19021899
for smi in step_monitoring_infos.values():
1903-
# Only include user metrics.
1904-
for mi in smi:
1905-
if (self._user_metrics_only and
1906-
not monitoring_infos.is_user_monitoring_info(mi)):
1907-
continue
1908-
key = self._to_metric_key(mi)
1909-
if monitoring_infos.is_counter(mi):
1910-
self._counters[key] = (
1911-
monitoring_infos.extract_metric_result_map_value(mi))
1912-
elif monitoring_infos.is_distribution(mi):
1913-
self._distributions[key] = (
1914-
monitoring_infos.extract_metric_result_map_value(mi))
1915-
elif monitoring_infos.is_gauge(mi):
1916-
self._gauges[key] = (
1917-
monitoring_infos.extract_metric_result_map_value(mi))
1918-
1919-
def _to_metric_key(self, monitoring_info):
1920-
# Right now this assumes that all metrics have a PTRANSFORM
1921-
transform_id = monitoring_info.labels['PTRANSFORM']
1922-
namespace, name = monitoring_infos.parse_namespace_and_name(monitoring_info)
1923-
return MetricKey(transform_id, MetricName(namespace, name))
1900+
counters, distributions, gauges = \
1901+
portable_metrics.from_monitoring_infos(smi, user_metrics_only)
1902+
self._counters.update(counters)
1903+
self._distributions.update(distributions)
1904+
self._gauges.update(gauges)
19241905

19251906
def query(self, filter=None):
1926-
counters = [metrics.execution.MetricResult(k, v, v)
1907+
counters = [MetricResult(k, v, v)
19271908
for k, v in self._counters.items()
19281909
if self.matches(filter, k)]
1929-
distributions = [metrics.execution.MetricResult(k, v, v)
1910+
distributions = [MetricResult(k, v, v)
19301911
for k, v in self._distributions.items()
19311912
if self.matches(filter, k)]
1932-
gauges = [metrics.execution.MetricResult(k, v, v)
1913+
gauges = [MetricResult(k, v, v)
19331914
for k, v in self._gauges.items()
19341915
if self.matches(filter, k)]
19351916

sdks/python/apache_beam/runners/portability/local_job_service.py

Lines changed: 24 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,7 @@
3131
import grpc
3232
from google.protobuf import text_format
3333

34+
from apache_beam.metrics import monitoring_infos
3435
from apache_beam.portability.api import beam_artifact_api_pb2
3536
from apache_beam.portability.api import beam_artifact_api_pb2_grpc
3637
from apache_beam.portability.api import beam_fn_api_pb2_grpc
@@ -107,6 +108,26 @@ def stop(self, timeout=1):
107108
if os.path.exists(self._staging_dir) and self._cleanup_staging_dir:
108109
shutil.rmtree(self._staging_dir, ignore_errors=True)
109110

111+
def GetJobMetrics(self, request, context=None):
112+
if request.job_id not in self._jobs:
113+
raise LookupError("Job {} does not exist".format(request.job_id))
114+
115+
result = self._jobs[request.job_id].result
116+
monitoring_info_list = []
117+
for mi in result._monitoring_infos_by_stage.values():
118+
monitoring_info_list.extend(mi)
119+
120+
# Filter out system metrics
121+
user_monitoring_info_list = [
122+
x for x in monitoring_info_list
123+
if monitoring_infos._is_user_monitoring_info(x) or
124+
monitoring_infos._is_user_distribution_monitoring_info(x)
125+
]
126+
127+
return beam_job_api_pb2.GetJobMetricsResponse(
128+
metrics=beam_job_api_pb2.MetricResults(
129+
committed=user_monitoring_info_list))
130+
110131

111132
class SubprocessSdkWorker(object):
112133
"""Manages a SDK worker implemented as a subprocess communicating over grpc.
@@ -176,6 +197,7 @@ def __init__(self,
176197
self._log_queues = []
177198
self.state = beam_job_api_pb2.JobState.STARTING
178199
self.daemon = True
200+
self.result = None
179201

180202
@property
181203
def state(self):
@@ -204,11 +226,12 @@ def run(self):
204226
def _run_job(self):
205227
with JobLogHandler(self._log_queues):
206228
try:
207-
fn_api_runner.FnApiRunner(
229+
result = fn_api_runner.FnApiRunner(
208230
provision_info=self._provision_info).run_via_runner_api(
209231
self._pipeline_proto)
210232
logging.info('Successfully completed job.')
211233
self.state = beam_job_api_pb2.JobState.DONE
234+
self.result = result
212235
except: # pylint: disable=bare-except
213236
logging.exception('Error running pipeline.')
214237
logging.exception(traceback)
Lines changed: 69 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,69 @@
1+
#
2+
# Licensed to the Apache Software Foundation (ASF) under one or more
3+
# contributor license agreements. See the NOTICE file distributed with
4+
# this work for additional information regarding copyright ownership.
5+
# The ASF licenses this file to You under the Apache License, Version 2.0
6+
# (the "License"); you may not use this file except in compliance with
7+
# the License. You may obtain a copy of the License at
8+
#
9+
# http://www.apache.org/licenses/LICENSE-2.0
10+
#
11+
# Unless required by applicable law or agreed to in writing, software
12+
# distributed under the License is distributed on an "AS IS" BASIS,
13+
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
# See the License for the specific language governing permissions and
15+
# limitations under the License.
16+
#
17+
18+
from __future__ import absolute_import
19+
20+
import logging
21+
22+
from apache_beam.metrics import monitoring_infos
23+
from apache_beam.metrics.execution import MetricKey
24+
from apache_beam.metrics.metric import MetricName
25+
26+
27+
def from_monitoring_infos(monitoring_info_list, user_metrics_only=False):
28+
"""Groups MonitoringInfo objects into counters, distributions and gauges.
29+
30+
Args:
31+
monitoring_info_list: An iterable of MonitoringInfo objects.
32+
user_metrics_only: If true, includes user metrics only.
33+
Returns:
34+
A tuple containing three dictionaries: counters, distributions and gauges,
35+
respectively. Each dictionary contains (MetricKey, metric result) pairs.
36+
"""
37+
counters = {}
38+
distributions = {}
39+
gauges = {}
40+
41+
for mi in monitoring_info_list:
42+
if (user_metrics_only and
43+
not monitoring_infos.is_user_monitoring_info(mi)):
44+
continue
45+
46+
try:
47+
key = _create_metric_key(mi)
48+
except ValueError as e:
49+
logging.debug(str(e))
50+
continue
51+
metric_result = (monitoring_infos.extract_metric_result_map_value(mi))
52+
53+
if monitoring_infos.is_counter(mi):
54+
counters[key] = metric_result
55+
elif monitoring_infos.is_distribution(mi):
56+
distributions[key] = metric_result
57+
elif monitoring_infos.is_gauge(mi):
58+
gauges[key] = metric_result
59+
60+
return counters, distributions, gauges
61+
62+
63+
def _create_metric_key(monitoring_info):
64+
step_name = monitoring_infos.get_step_name(monitoring_info)
65+
if not step_name:
66+
raise ValueError('Failed to deduce step_name from MonitoringInfo: {}'
67+
.format(monitoring_info))
68+
namespace, name = monitoring_infos.parse_namespace_and_name(monitoring_info)
69+
return MetricKey(step_name, MetricName(namespace, name))

sdks/python/apache_beam/runners/portability/portable_runner.py

Lines changed: 24 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -28,7 +28,8 @@
2828
import grpc
2929

3030
from apache_beam import version as beam_version
31-
from apache_beam import metrics
31+
from apache_beam.metrics import metric
32+
from apache_beam.metrics.execution import MetricResult
3233
from apache_beam.options.pipeline_options import DebugOptions
3334
from apache_beam.options.pipeline_options import PortableOptions
3435
from apache_beam.options.pipeline_options import SetupOptions
@@ -41,6 +42,7 @@
4142
from apache_beam.runners.job import utils as job_utils
4243
from apache_beam.runners.portability import fn_api_runner_transforms
4344
from apache_beam.runners.portability import job_server
45+
from apache_beam.runners.portability import portable_metrics
4446
from apache_beam.runners.portability import portable_stager
4547
from apache_beam.runners.worker import sdk_worker_main
4648
from apache_beam.runners.worker import worker_pool_main
@@ -366,16 +368,30 @@ def add_runner_options(parser):
366368
state_stream, cleanup_callbacks)
367369

368370

369-
class PortableMetrics(metrics.metric.MetricResults):
371+
class PortableMetrics(metric.MetricResults):
370372
def __init__(self, job_metrics_response):
371-
# TODO(lgajowy): Convert portable metrics to MetricResults
372-
# and allow querying them (BEAM-4775)
373-
pass
373+
metrics = job_metrics_response.metrics
374+
self.attempted = portable_metrics.from_monitoring_infos(metrics.attempted)
375+
self.committed = portable_metrics.from_monitoring_infos(metrics.committed)
376+
377+
@staticmethod
378+
def _combine(committed, attempted, filter):
379+
all_keys = set(committed.keys()) | set(attempted.keys())
380+
return [
381+
MetricResult(key, committed.get(key), attempted.get(key))
382+
for key in all_keys
383+
if metric.MetricResults.matches(filter, key)
384+
]
374385

375386
def query(self, filter=None):
376-
return {'counters': [],
377-
'distributions': [],
378-
'gauges': []}
387+
counters, distributions, gauges = [
388+
self._combine(x, y, filter)
389+
for x, y in zip(self.committed, self.attempted)
390+
]
391+
392+
return {self.COUNTERS: counters,
393+
self.DISTRIBUTIONS: distributions,
394+
self.GAUGES: gauges}
379395

380396

381397
class PipelineResult(runner.PipelineResult):

sdks/python/apache_beam/testing/load_tests/load_test.py

Lines changed: 10 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -52,32 +52,24 @@ def setUp(self):
5252
self.input_options = json.loads(self.pipeline.get_option('input_options'))
5353
self.project_id = self.pipeline.get_option('project')
5454

55-
self.publish_to_big_query = self.pipeline.get_option('publish_to_big_query')
5655
self.metrics_dataset = self.pipeline.get_option('metrics_dataset')
5756
self.metrics_namespace = self.pipeline.get_option('metrics_table')
5857

59-
if not self.are_metrics_collected():
60-
logging.info('Metrics will not be collected')
61-
self.metrics_monitor = None
62-
else:
63-
self.metrics_monitor = MetricsReader(
64-
project_name=self.project_id,
65-
bq_table=self.metrics_namespace,
66-
bq_dataset=self.metrics_dataset,
67-
)
58+
self.metrics_monitor = MetricsReader(
59+
publish_to_bq=self.pipeline.get_option('publish_to_big_query') ==
60+
'true',
61+
project_name=self.project_id,
62+
bq_table=self.metrics_namespace,
63+
bq_dataset=self.metrics_dataset,
64+
# Apply filter to prevent system metrics from being published
65+
filters=MetricsFilter().with_namespace(self.metrics_namespace)
66+
)
6867

6968
def tearDown(self):
7069
result = self.pipeline.run()
7170
result.wait_until_finish()
7271

73-
if self.metrics_monitor:
74-
self.metrics_monitor.publish_metrics(result)
75-
76-
def apply_filter(self, allowed):
77-
"""Prevents metrics from namespaces other than specified in the argument
78-
from being published."""
79-
if allowed:
80-
self.metrics_monitor.filters = MetricsFilter().with_namespaces(allowed)
72+
self.metrics_monitor.publish_metrics(result)
8173

8274
def get_option_or_default(self, opt_name, default=0):
8375
"""Returns a pipeline option or a default value if it was not provided.
@@ -92,10 +84,6 @@ def get_option_or_default(self, opt_name, default=0):
9284
except ValueError as exc:
9385
self.fail(str(exc))
9486

95-
def are_metrics_collected(self):
96-
return self.publish_to_big_query == 'true' and None not in (
97-
self.project_id, self.metrics_dataset, self.metrics_namespace)
98-
9987

10088
if __name__ == '__main__':
10189
logging.getLogger().setLevel(logging.DEBUG)

sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py

Lines changed: 5 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -171,7 +171,7 @@ class MetricsReader(object):
171171
publishers = []
172172

173173
def __init__(self, project_name=None, bq_table=None, bq_dataset=None,
174-
filters=None):
174+
publish_to_bq=False, filters=None):
175175
"""Initializes :class:`MetricsReader` .
176176
177177
Args:
@@ -182,7 +182,8 @@ def __init__(self, project_name=None, bq_table=None, bq_dataset=None,
182182
"""
183183
self._namespace = bq_table
184184
self.publishers.append(ConsoleMetricsPublisher())
185-
check = project_name and bq_table and bq_dataset
185+
186+
check = project_name and bq_table and bq_dataset and publish_to_bq
186187
if check:
187188
bq_publisher = BigQueryMetricsPublisher(
188189
project_name, bq_table, bq_dataset)
@@ -311,8 +312,8 @@ def _prepare_runtime_metrics(self, distributions):
311312
min_values = []
312313
max_values = []
313314
for dist in distributions:
314-
min_values.append(dist.committed.min)
315-
max_values.append(dist.committed.max)
315+
min_values.append(dist.result.min)
316+
max_values.append(dist.result.max)
316317
# finding real start
317318
min_value = min(min_values)
318319
# finding real end

sdks/python/apache_beam/testing/load_tests/pardo_test.py

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -138,8 +138,6 @@
138138
class ParDoTest(LoadTest):
139139
def setUp(self):
140140
super(ParDoTest, self).setUp()
141-
if self.are_metrics_collected():
142-
self.apply_filter([self.metrics_namespace])
143141
self.iterations = self.get_option_or_default('iterations')
144142
self.number_of_counters = self.get_option_or_default('number_of_counters')
145143
self.number_of_operations = self.get_option_or_default(

0 commit comments

Comments
 (0)