Repository: beam Updated Branches: refs/heads/master bf16b4a80 -> 321ec8d78
Add javadoc to the Metric interfaces Project: http://git-wip-us.apache.org/repos/asf/beam/repo Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/113917b6 Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/113917b6 Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/113917b6 Branch: refs/heads/master Commit: 113917b6a4b651168709ec7cb0c27d0da67d88f0 Parents: bf16b4a Author: bchambers <[email protected]> Authored: Wed Apr 19 14:24:27 2017 -0700 Committer: Dan Halperin <[email protected]> Committed: Thu Apr 20 11:36:04 2017 -0700 ---------------------------------------------------------------------- .../apache/beam/sdk/metrics/MetricResults.java | 29 +++++++++++++++++++- .../org/apache/beam/sdk/metrics/Metrics.java | 27 ++++++++++++++++++ .../apache/beam/sdk/metrics/MetricsFilter.java | 9 +++++- 3 files changed, 63 insertions(+), 2 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/beam/blob/113917b6/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricResults.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricResults.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricResults.java index dab65ea..1d83391 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricResults.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricResults.java @@ -28,7 +28,34 @@ import org.apache.beam.sdk.annotations.Experimental.Kind; @Experimental(Kind.METRICS) public abstract class MetricResults { /** - * Query for all metrics that match the filter. + * Query for all metric values that match a given filter. + * + * The {@code filter} may filter based on the namespace and/or name of the metric, as well as + * the step that reported the metric. + * + * <p>For each type of metric, the result contains an iterable of all metrics of that type that + * matched the filter. Each {@link MetricResult} includes the name of the metric, the step in + * which it was reported and the {@link MetricResult#committed} and + * {@link MetricResult#attempted} values. + * + * <p>Note that runners differ in their support for committed and attempted values. + * + * <p>Example: Querying the metrics reported from the {@code SomeDoFn} example in {@link Metrics} + * could be done as follows: + * <pre>{@code + * Pipeline p = ...; + * p.apply("create1", Create.of("hello")).apply("myStepName1", ParDo.of(new SomeDoFn())); + * p.apply("create2", Create.of("world")).apply("myStepName2", ParDo.of(new SomeDoFn())); + * PipelineResult result = p.run(); + * MetricResults metrics = result.metrics(); + * MetricQueryResults metricResults = metrics.queryMetrics(new MetricsFilter.Builder() + * .addNameFilter("my-counter") + * .addStepFilter("myStepName1").addStepFilter("myStepName2") + * .build()); + * Iterable<MetricResult<Long>> counters = metricResults.counters(); + * // counters should contain the value of my-counter reported from each of the ParDo + * // applications. + * }</pre> */ public abstract MetricQueryResults queryMetrics(MetricsFilter filter); } http://git-wip-us.apache.org/repos/asf/beam/blob/113917b6/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java index 121698d..587241a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java @@ -24,6 +24,33 @@ import org.apache.beam.sdk.annotations.Experimental.Kind; /** * The <code>Metrics</code> is a utility class for producing various kinds of metrics for * reporting properties of an executing pipeline. + * + * <p>Metrics are created by calling one of the static methods in this class. Each metric is + * associated with a namespace and a name. The namespace allows grouping related metrics together + * based on the definition while also disambiguating common names based on where they are defined. + * + * <p>Reported metrics are implicitly scoped to the transform within the pipeline that reported + * them. This allows reporting the same metric name in multiple places and identifying the value + * each transform reported, as well as aggregating the metric across + * + * <p>It is runner-dependent whether Metrics are accessible during pipeline execution or only after + * jobs have completed. + * + * <p>Example: + * <pre>{@code + * class SomeDoFn extends DoFn<String, String> { + * private Counter counter = Metrics.counter(SomeDoFn.class, "my-counter"); + * + * {@literal @}ProcessElement + * public void processElement(ProcessContext c) { + * counter.inc(); + * Metrics.counter(SomeDoFn.class, "my-counter2").inc(); + * } + * } + * }</pre> + * + * <p>See {@link MetricResults} (available from the {@code PipelineResults} interface) for an + * example off how to query metrics. */ @Experimental(Kind.METRICS) public class Metrics { http://git-wip-us.apache.org/repos/asf/beam/blob/113917b6/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsFilter.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsFilter.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsFilter.java index ec81251..6cad70f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsFilter.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsFilter.java @@ -59,7 +59,6 @@ public abstract class MetricsFilter { * * <p>If no name filters are specified then all metric names will be inculded. * - * * <p>If one or more name filters are specified, then only metrics that match one or more of the * filters will be included. */ @@ -75,6 +74,14 @@ public abstract class MetricsFilter { * * <p>If one or more steps are specified, then metrics will be included if they are part of * any of the specified steps. + * + * <p>The step names of metrics are identified as a path within the pipeline. So for example, a + * transform that is applied with the name "bar" in a composite that was applied with the name + * "foo" would have a step name of "foo/bar". + * + * <p>Step name filters may be either a full name (such as "foo/bar/baz") or a partial name such + * as "foo", "bar" or "foo/bar". However, each component of the step name must be completely + * matched, so the filter "foo" will not match the step name such as "fool/bar/foot". */ public Builder addStep(String step) { immutableStepsBuilder().add(step);
