[
https://issues.apache.org/jira/browse/BEAM-4727?focusedWorklogId=121964&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-121964
]
ASF GitHub Bot logged work on BEAM-4727:
----------------------------------------
Author: ASF GitHub Bot
Created on: 11/Jul/18 17:17
Start Date: 11/Jul/18 17:17
Worklog Time Spent: 10m
Work Description: lostluck commented on a change in pull request #5884:
[BEAM-4727] Re-use metric context throughout bundle
URL: https://github.com/apache/beam/pull/5884#discussion_r201773998
##########
File path: sdks/go/pkg/beam/core/metrics/metrics.go
##########
@@ -69,18 +69,58 @@ type ctxKey string
const bundleKey ctxKey = "beam:bundle"
const ptransformKey ctxKey = "beam:ptransform"
+// context is a caching context since reads & writes are expensive.
+type beamCtx struct {
+ context.Context
+ bundleID, ptransformID string
+}
+
+// Lift the keys value for faster lookups when not available.
+func (ctx *beamCtx) Value(key interface{}) interface{} {
+ switch key {
+ case bundleKey:
+ if ctx.bundleID == "" {
+ if id := ctx.Value(key); id != nil {
+ ctx.bundleID = id.(string)
+ }
+ }
+ return ctx.bundleID
+ case ptransformKey:
+ if ctx.ptransformID == "" {
+ if id := ctx.Value(key); id != nil {
+ ctx.ptransformID = id.(string)
+ }
+ }
+ return ctx.ptransformID
+ }
+ return ctx.Context.Value(key)
+}
+
// SetBundleID sets the id of the current Bundle.
func SetBundleID(ctx context.Context, id string) context.Context {
- return context.WithValue(ctx, bundleKey, id)
+ var ptransformID string
+ if bctx, ok := ctx.(*beamCtx); ok {
+ ptransformID = bctx.ptransformID
+ }
+ return &beamCtx{Context: ctx, bundleID: id, ptransformID: ptransformID}
Review comment:
I considered that but then metrics following an emit would be incorrect,
short of retaining the parent PTransform ID and re-adding that, which would
work, but would be cumbersome. That approach is a viable alternative to caching
the Context in pardo.go though as it accomplishes the same goal.
Do you prefer that approach?
eg. re: correctness issue on ordering of event if the context isn't
re-allocated:
SDK sets PtransformID
SDK calls User ParDo
User ParDo1 emits - > SDK runs child ParDos which set their own PTransformIDs
User ParDo1 logs a metric -> Metric is associated with the wrong PTransform.
That is, we'd need to extract and re-assert the parent's PTransform ID
before returning from the ParDo process elements.
----------------------------------------------------------------
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: 121964)
Time Spent: 2h 10m (was: 2h)
> 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: 2h 10m
> 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)