xinyuiscool commented on code in PR #26437:
URL: https://github.com/apache/beam/pull/26437#discussion_r1184102173
##########
runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineOptions.java:
##########
@@ -111,6 +111,12 @@ public interface SamzaPipelineOptions extends
PipelineOptions {
void setEnableMetrics(Boolean enableMetrics);
+ @Description("Enable/disable Beam Transform throughput, latency metrics in
Samza Runner")
+ @Default.Boolean(true)
Review Comment:
Let's keep default to be false so we can enable it only for jobs need this.
##########
runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java:
##########
@@ -145,9 +150,30 @@ protected <KeyT, OutT> void registerInputMessageStreams(
}
public <OutT> void registerMessageStream(PValue pvalue,
MessageStream<OpMessage<OutT>> stream) {
+ registerMessageStream(pvalue, stream, true);
+ }
+
+ public <OutT> void registerMessageStreamWithoutMetricOp(
+ PValue pvalue, MessageStream<OpMessage<OutT>> stream) {
+ registerMessageStream(pvalue, stream, false);
Review Comment:
Please remove this boolean flag. We should attach the MetricOp if the
TransformMetric is enabled. But inside the op, we can decide what to do with it.
##########
runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaInputMetricOp.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.
+ */
+package org.apache.beam.runners.samza.runtime;
+
+import java.math.BigInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.beam.runners.samza.metrics.BeamTransformMetricRegistry;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * SamzaInputMetricOp is a {@link SamzaMetricOp} that emits & maintains
default transform metrics
+ * for input PCollection to the transform. It emits the input throughput and
maintains avg arrival
+ * time for input PCollection per watermark.
+ *
+ * <p>Assumes that {@code SamzaInputMetricOp#processWatermark(Instant,
OpEmitter)} is exclusive of
+ * {@code SamzaInputMetricOp#processElement(Instant, OpEmitter)}.
Specifically, the processWatermark
+ * method assumes that no calls to processElement will be made during its
execution, and vice versa.
+ *
+ * @param <T> The type of the elements in the input PCollection.
+ */
+public class SamzaInputMetricOp<T> extends SamzaMetricOp<T> {
Review Comment:
Can we consolidate them in one Op and do the computation based on the inputs
and outputs present?
##########
runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java:
##########
@@ -145,9 +150,30 @@ protected <KeyT, OutT> void registerInputMessageStreams(
}
public <OutT> void registerMessageStream(PValue pvalue,
MessageStream<OpMessage<OutT>> stream) {
+ registerMessageStream(pvalue, stream, true);
+ }
+
+ public <OutT> void registerMessageStreamWithoutMetricOp(
+ PValue pvalue, MessageStream<OpMessage<OutT>> stream) {
+ registerMessageStream(pvalue, stream, false);
+ }
+
+ private <OutT> void registerMessageStream(
+ PValue pvalue, MessageStream<OpMessage<OutT>> stream, boolean
attachTransformMetricOp) {
if (messsageStreams.containsKey(pvalue)) {
throw new IllegalArgumentException("Stream already registered for
pvalue: " + pvalue);
}
+ // add a step to attach OutputMetricOp if registered for Op Stream
+ final Boolean userOverride =
getPipelineOptions().getEnableTransformMetrics();
Review Comment:
Update the var name.
##########
runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/BeamTransformMetricRegistry.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.
+ */
+package org.apache.beam.runners.samza.metrics;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+import org.apache.samza.context.Context;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * BeamTransformMetricRegistry is a registry that maintains the metrics for
each transform. It
+ * maintains the average arrival time for each PCollection for a primitive
transform.
+ *
+ * <p>For a non-data shuffling primitive transform, the average arrival time
is calculated per
+ * watermark, per PCollection {@link org.apache.beam.sdk.values.PValue} and
updated in
+ * avgArrivalTimeMap
+ */
+public class BeamTransformMetricRegistry implements Serializable {
+ private static final Logger LOG =
LoggerFactory.getLogger(BeamTransformMetricRegistry.class);
+
+ // TransformName -> PValue for pCollection -> Map<WatermarkId,
AvgArrivalTime>
+ private ConcurrentHashMap<String, ConcurrentHashMap<String,
ConcurrentHashMap<Long, Long>>>
Review Comment:
In essense, we are keeping track of each PCollection's timestamp, regardless
of using per transform or not. In the above graph, we should use the latency
due to traversing 4 first. Otherwise we are not reporting correctly the
latency. In another world we might do some optimization to execute 4 and 5 in
parallel, then we won't be able see the latency improvement. I think this
should be a simple equation, basically `timestamp(input PC) - timestamp(output
PC)` for each PTransform.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]