Sanil15 commented on code in PR #26437:
URL: https://github.com/apache/beam/pull/26437#discussion_r1181766990


##########
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:
   Previously: We proposed to use the arrival time at the exit of the previous 
transform as the start time for the next transform.  Then use the formula 
   
   Avg Latency for a PTransform = Avg Exit Arrival Time from PTrasform - Avg 
Entry Arrival Time (= Exit arrival time from the previous transform)
   
   The above formula won't work for DAG split case (see the image below as an 
example). Because if a dag splits out then an element traverses left sub-tree 
first and then traverses the right sub-tree 
   
   Let's take an example, in this case shown below, 4 is traversed after 5, 
hence exit arrival time at 2 cannot be used for entry arrival time at 4 since 
that includes latency of traversing 5
   
   
![image](https://user-images.githubusercontent.com/16806974/235499463-777aa2b1-4200-471f-bb5c-e2f59a6c1286.png)
   
   Hence to solve this we attach MetricOp at entry and exit of each transform. 
What we need to keep the arrival time per pValue at the entry and exit per 
transform,
   
   This was brought up during the review, can explain offline
   
   



-- 
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]

Reply via email to