[ 
https://issues.apache.org/jira/browse/BEAM-6100?focusedWorklogId=169822&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-169822
 ]

ASF GitHub Bot logged work on BEAM-6100:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 27/Nov/18 15:25
            Start Date: 27/Nov/18 15:25
    Worklog Time Spent: 10m 
      Work Description: lgajowy commented on a change in pull request #7087: 
[BEAM-6100] Collect metrics properly in Load tests
URL: https://github.com/apache/beam/pull/7087#discussion_r236711795
 
 

 ##########
 File path: 
sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/metrics/TimeMonitor.java
 ##########
 @@ -0,0 +1,39 @@
+/*
+ * 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.sdk.loadtests.metrics;
+
+import org.apache.beam.sdk.metrics.Distribution;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.values.KV;
+
+/** Monitor that records processing time distribution in the pipeline. */
 
 Review comment:
   I investigated this a bit. 
   
   For start time collection it looks simple. We could create a pipeline 
factory method that returns it with a "substitute" of source, like so: 
   
   ```
   public PCollection<KV<byte[], byte[]>> getPipelineWithAppliedSource() {
       pipeline = Pipeline.create(options);
       return pipeline.apply(Create.of(KV.of(new byte[0], new byte[0])));
   
     }
   ```
   after that we could replace this easily with a composite transform of source 
(bounded or, Unbounded) + `ParDo.of(TimeMonitor)`
   
   
   A question though:
    - the code you mentioned is in _beam-runners-core-construction-java_. That 
means the _load_tests_ package would need to depend on it. Is it ok to have 
such dependency? 
   
   And some concerns: 
    - I didn't check well enough yet but it seems feasible to look for all the 
leaves of the Job DAG, am I right (or maybe I didn't see something coming)? 
That would be needed to find out where should we put TimeMonitors to collect 
end times. 
    -  If the type of a job graph is different than `PCollection<KV<byte[], 
byte[]>>` then we might have trouble because I don't think we can infer what 
types TimeMonitor should accept. 
   
   Let me know what you think. Either way, I suggest creating a separate 
ticket/PR for this too in order not to block other work. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 169822)
    Time Spent: 2h 50m  (was: 2h 40m)

> Runtime and total bytes metrics are not collected properly
> ----------------------------------------------------------
>
>                 Key: BEAM-6100
>                 URL: https://issues.apache.org/jira/browse/BEAM-6100
>             Project: Beam
>          Issue Type: Sub-task
>          Components: testing
>            Reporter: Lukasz Gajowy
>            Assignee: Lukasz Gajowy
>            Priority: Major
>          Time Spent: 2h 50m
>  Remaining Estimate: 0h
>
> Currently, we collect time (distribution) and bytes (counter) metrics from 
> one ParDo (called MetricsMonitor) that is put in pipelines in one, 
> arbitrarily chosen place (usually "in the middle" of pipeline's graph. In 
> some cases, invalid time (or total bytes count) is registered. 
> Taking [this|https://github.com/apache/beam/pull/6987#discussion_r231976671] 
> discussion into account, ideally, we'd like to:
>  - collect runtime by recording time at the root and sink(s) of the pipeline
>  - collect total bytes in a separate ParDo that allows deciding what byte 
> amount do we actually want to collect (now it's coupled to the 
> time-collecting Monitor which is inconvenient).



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to