|
39 | 39 |
|
40 | 40 | import apache_beam as beam # pylint: disable=ungrouped-imports |
41 | 41 | from apache_beam import coders |
42 | | -from apache_beam import metrics |
43 | 42 | from apache_beam.coders.coder_impl import create_InputStream |
44 | 43 | from apache_beam.coders.coder_impl import create_OutputStream |
| 44 | +from apache_beam.metrics import metric |
45 | 45 | from apache_beam.metrics import monitoring_infos |
46 | | -from apache_beam.metrics.execution import MetricKey |
| 46 | +from apache_beam.metrics.execution import MetricResult |
47 | 47 | from apache_beam.metrics.execution import MetricsEnvironment |
48 | | -from apache_beam.metrics.metricbase import MetricName |
49 | 48 | from apache_beam.options import pipeline_options |
50 | 49 | from apache_beam.options.value_provider import RuntimeValueProvider |
51 | 50 | from apache_beam.portability import common_urns |
|
62 | 61 | from apache_beam.runners import runner |
63 | 62 | from apache_beam.runners.portability import artifact_service |
64 | 63 | from apache_beam.runners.portability import fn_api_runner_transforms |
| 64 | +from apache_beam.runners.portability import portable_metrics |
65 | 65 | from apache_beam.runners.portability.fn_api_runner_transforms import create_buffer_id |
66 | 66 | from apache_beam.runners.portability.fn_api_runner_transforms import only_element |
67 | 67 | from apache_beam.runners.portability.fn_api_runner_transforms import split_buffer_id |
@@ -1883,53 +1883,34 @@ def get(self, timeout=None): |
1883 | 1883 | return self._response |
1884 | 1884 |
|
1885 | 1885 |
|
1886 | | -class FnApiMetrics(metrics.metric.MetricResults): |
| 1886 | +class FnApiMetrics(metric.MetricResults): |
1887 | 1887 | def __init__(self, step_monitoring_infos, user_metrics_only=True): |
1888 | 1888 | """Used for querying metrics from the PipelineResult object. |
1889 | 1889 |
|
1890 | 1890 | 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. |
1893 | 1892 | """ |
1894 | 1893 | self._counters = {} |
1895 | 1894 | self._distributions = {} |
1896 | 1895 | self._gauges = {} |
1897 | 1896 | self._user_metrics_only = user_metrics_only |
1898 | | - self._init_metrics_from_monitoring_infos(step_monitoring_infos) |
1899 | 1897 | self._monitoring_infos = step_monitoring_infos |
1900 | 1898 |
|
1901 | | - def _init_metrics_from_monitoring_infos(self, step_monitoring_infos): |
1902 | 1899 | 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) |
1924 | 1905 |
|
1925 | 1906 | def query(self, filter=None): |
1926 | | - counters = [metrics.execution.MetricResult(k, v, v) |
| 1907 | + counters = [MetricResult(k, v, v) |
1927 | 1908 | for k, v in self._counters.items() |
1928 | 1909 | if self.matches(filter, k)] |
1929 | | - distributions = [metrics.execution.MetricResult(k, v, v) |
| 1910 | + distributions = [MetricResult(k, v, v) |
1930 | 1911 | for k, v in self._distributions.items() |
1931 | 1912 | if self.matches(filter, k)] |
1932 | | - gauges = [metrics.execution.MetricResult(k, v, v) |
| 1913 | + gauges = [MetricResult(k, v, v) |
1933 | 1914 | for k, v in self._gauges.items() |
1934 | 1915 | if self.matches(filter, k)] |
1935 | 1916 |
|
|
0 commit comments