[
https://issues.apache.org/jira/browse/BEAM-4727?focusedWorklogId=119981&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-119981
]
ASF GitHub Bot logged work on BEAM-4727:
----------------------------------------
Author: ASF GitHub Bot
Created on: 06/Jul/18 18:51
Start Date: 06/Jul/18 18:51
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_r200740947
##########
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 {
Review comment:
This is not an error.
BundleID will in practice always be set before any ptransformID, but I don't
like making that assumption.
This approach is the "silently not have a ptransformId" variant.
If this isn't a beamCtx that's OK, calling this method returns a beamCtx.
If it's not a beamCtx, the Value method is invoked. If a beamCtx is anywhere
on the stack, eventually the beamCtx.Value method will be called, which then
delegates down the stack if the ctx doesn't have the value cached already, and
then it caches it for later to prevent it from going deeper.
The necessity of the cache is partly mitigated by retaining an earlier
configured context in the execution node, but the cache is valuable in avoiding
unnecessary type assertion overhead.
----------------------------------------------------------------
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: 119981)
Time Spent: 1h 20m (was: 1h 10m)
> 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 20m
> 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)