xinyuiscool commented on code in PR #26649: URL: https://github.com/apache/beam/pull/26649#discussion_r1192618185
########## runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaOutputGBKMetricOp.java: ########## @@ -0,0 +1,163 @@ +/* + * 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 edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +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.sdk.transforms.windowing.BoundedWindow; +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.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * SamzaOutputGBKMetricOp is a {@link Op} that emits & maintains default metrics for output + * PCollection for GroupByKey. It emits the output throughput and maintains avg output time for + * output PCollection per windowId (). It is also responsible for emitting latency metric per + * windowId once the watermark passes the end of window timestamp. + * + * <p>Assumes that {@code SamzaOutputGBKMetricOp#processWatermark(Instant, OpEmitter)} is exclusive + * of {@code SamzaOutputGBKMetricOp#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. + */ +public class SamzaOutputGBKMetricOp<T> implements Op<T, T, Void> { Review Comment: Again this class looks very close to the InputGBKMetricsOp. Is it possible to pass in a flag, like your OpType, and do extra logic for output once the flag is Output? SImilar thing for the previous InputMetricOp, seems we can consolidate the logic into one class and use the flag to do something more. ########## runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaInputGBKMetricOp.java: ########## @@ -0,0 +1,154 @@ +/* + * 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 edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +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.sdk.transforms.windowing.BoundedWindow; +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.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * SamzaInputGBKMetricOp is a {@link Op} that emits & maintains default metrics for input + * PCollection for GroupByKey. It emits the input throughput and maintains avg input time for input + * PCollection per windowId. + * + * <p>Assumes that {@code SamzaInputGBKMetricOp#processWatermark(Instant, OpEmitter)} is exclusive + * of {@code SamzaInputGBKMetricOp#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 SamzaInputGBKMetricOp<T> implements Op<T, T, Void> { Review Comment: package private? ########## runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaTransformMetricRegistry.java: ########## @@ -42,26 +44,33 @@ public class SamzaTransformMetricRegistry implements Serializable { // TransformName -> PValue for pCollection -> Map<WatermarkId, AvgArrivalTime> private final ConcurrentHashMap<String, ConcurrentHashMap<String, ConcurrentHashMap<Long, Long>>> avgArrivalTimeMap; + // TransformName -> Map<WindowId, AvgArrivalTime> + @SuppressFBWarnings("SE_BAD_FIELD") + private final ConcurrentHashMap<String, ConcurrentHashMap<BoundedWindow, Long>> Review Comment: I think we should add some metrics to measure the size of these two maps: avgArrivalTimeMap and avgArrivalTimeMapForGbk, to prevent memory problems. The metric will be the entry count of avgArrivalTimes. -- 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]
