Sanil15 commented on code in PR #26437: URL: https://github.com/apache/beam/pull/26437#discussion_r1188046880
########## runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaOutputMetricOp.java: ########## @@ -0,0 +1,149 @@ +/* + * 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.math.BigInteger; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.beam.runners.samza.runtime.KeyedTimerData; +import org.apache.beam.runners.samza.runtime.Op; +import org.apache.beam.runners.samza.runtime.OpEmitter; +import org.apache.beam.runners.samza.util.PipelineJsonRenderer; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.samza.config.Config; +import org.apache.samza.context.Context; +import org.apache.samza.operators.Scheduler; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * SamzaOutputMetricOp is a metric Op that emits & maintains default transform metrics for output + * PCollection to the transform. It emits the output throughput and maintains avg arrival time for + * output PCollection per watermark. + * + * <p>Assumes that {@code SamzaOutputMetricOp#processWatermark(Instant, OpEmitter)} is exclusive of + * {@code SamzaOutputMetricOp#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 output PCollection. + */ +class SamzaOutputMetricOp<T> implements Op<T, T, Void> { + // Unique name of the PTransform this MetricOp is associated with + protected final String transformFullName; + protected final SamzaTransformMetricRegistry samzaTransformMetricRegistry; + // Name or identifier of the PCollection which PTransform is processing + protected final String pValue; + // List of input PValue(s) for all PCollections processing the PTransform + protected transient List<String> transformInputs; + // List of output PValue(s) for all PCollections processing the PTransform + protected transient List<String> transformOutputs; + // Name of the task, for logging purpose + protected transient String task; + + private static final Logger LOG = LoggerFactory.getLogger(SamzaOutputMetricOp.class); + // Counters for output throughput + private AtomicLong count; + private AtomicReference<BigInteger> sumOfTimestamps; Review Comment: done, my branch rebase deleted some updated in previous pr, fixed ########## runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java: ########## @@ -168,6 +177,81 @@ public <OutT> MessageStream<OpMessage<OutT>> getMessageStream(PValue pvalue) { return stream; } + public <InT extends PValue, OutT extends PValue> void attachTransformMetricOp( + PTransform<InT, OutT> transform, + TransformHierarchy.Node node, + SamzaMetricOpFactory.OpType opType) { + final Boolean enableTransformMetrics = getPipelineOptions().getEnableTransformMetrics(); + final String urn = PTransformTranslation.urnForTransformOrNull(transform); + + // skip attach transform if user override is false or transform is GBK + if (!enableTransformMetrics + || urn == null + || urn.equals(PTransformTranslation.COMBINE_PER_KEY_TRANSFORM_URN) + || urn.equals(PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN)) { + return; + } + + // skip attach transform if transform is reading from external sources + if (isIOTransform(node, opType)) { + return; + } + + for (PValue pValue : getPValueForTransform(opType, transform, node)) { + // add another step for default metric computation + getMessageStream(pValue) + .flatMapAsync( + OpAdapter.adapt( + SamzaMetricOpFactory.createMetricOp( + pValue.getName(), + getTransformFullName(), + opType, + samzaTransformMetricRegistry), + this)); + } + } + + // Get the input or output PValue for a transform + private <InT extends PValue, OutT extends PValue> List<PValue> getPValueForTransform( + SamzaMetricOpFactory.OpType opType, + @NonNull PTransform<InT, OutT> transform, + @NonNull TransformHierarchy.Node node) { + switch (opType) { + case INPUT: + { + if (node.getInputs().size() > 1) { + return node.getInputs().entrySet().stream() + .map(Map.Entry::getValue) + .collect(Collectors.toList()); + } else { + return ImmutableList.of(getInput(transform)); + } + } + case OUTPUT: + if (node.getOutputs().size() > 1) { + return node.getOutputs().entrySet().stream() + .map(Map.Entry::getValue) + .collect(Collectors.toList()); + } + return ImmutableList.of(getOutput(transform)); + default: + throw new IllegalArgumentException("Unknown opType: " + opType); + } + } + + // Transforms that read or write to/from external sources are not supported + private boolean isIOTransform( Review Comment: done -- 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]
