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

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

                Author: ASF GitHub Bot
            Created on: 06/Jul/18 18:32
            Start Date: 06/Jul/18 18:32
    Worklog Time Spent: 10m 
      Work Description: wcn3 commented on a change in pull request #5885: 
[BEAM-4727] Improve metric lookup overhead for predeclared metrics
URL: https://github.com/apache/beam/pull/5885#discussion_r200736367
 
 

 ##########
 File path: sdks/go/pkg/beam/core/metrics/metrics.go
 ##########
 @@ -211,36 +268,57 @@ func (m *counter) toProto() *fnexecution_v1.Metrics_User 
{
 // Distribution is a simple distribution of values.
 type Distribution struct {
        name name
+
+       // The following are a fast cache of the key and storage
+       mu sync.Mutex
+       k  key
+       d  *distribution
 }
 
-func (m Distribution) String() string {
+func (m *Distribution) String() string {
        return fmt.Sprintf("Distribution metric %s", m.name)
 }
 
 // NewDistribution returns the Distribution with the given namespace and name.
-func NewDistribution(ns, n string) Distribution {
+func NewDistribution(ns, n string) *Distribution {
        mn := newName(ns, n)
-       return Distribution{
+       return &Distribution{
                name: mn,
        }
 }
 
 // Update updates the distribution within the given PTransform context with v.
-func (m Distribution) Update(ctx context.Context, v int64) {
+func (m *Distribution) Update(ctx context.Context, v int64) {
        key := getContextKey(ctx, m.name)
        ds := &distribution{
                count: 1,
                sum:   v,
                min:   v,
                max:   v,
        }
-       if m, loaded := distributions.LoadOrStore(key, ds); loaded {
-               d := m.(*distribution)
+       // if "collisions.distribution" == m.name.String() {
 
 Review comment:
   Please remove this.

----------------------------------------------------------------
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: 119973)
    Time Spent: 1h  (was: 50m)

> Reduce metrics overhead
> -----------------------
>
>                 Key: BEAM-4727
>                 URL: https://issues.apache.org/jira/browse/BEAM-4727
>             Project: Beam
>          Issue Type: Sub-task
>          Components: sdk-go
>            Reporter: Robert Burke
>            Assignee: Robert Burke
>            Priority: Major
>          Time Spent: 1h
>  Remaining Estimate: 0h
>
> There are a few opportunities to avoid metrics overhead.
> First when setting state in the context, we allocate a new one for the stored 
> value, per element. This generates a fair amount of objects for the garbage 
> collector too. If we retain and re-use contexts within a bundle, we would 
> have the opportunity to save on these costs.
> Also, it's possible that we have overhead on the metric updating paths. We 
> can possibly do better than the general sync.Map, and avoid the type 
> assertion cost for extracting values of known types from the maps.



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

Reply via email to