You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/07/08 16:45:39 UTC

[GitHub] [beam] lostluck commented on a diff in pull request #22196: Collect heap dumps on Go worker panic for Dataflow

lostluck commented on code in PR #22196:
URL: https://github.com/apache/beam/pull/22196#discussion_r916984079


##########
sdks/go/pkg/beam/core/runtime/harness/init/init.go:
##########
@@ -88,10 +90,24 @@ func hook() {
 		runtime.GlobalOptions.Import(opt.Options)
 	}
 
+	// Since Init() is hijacking main, it's appropriate to do as main
+	// does, and establish the background context here.
+	// We produce a cancelFn here so runs in Loopback mode and similar can clean up
+	// any leftover goroutines.
+	ctx, cancelFn := context.WithCancel(context.Background())
+	defer cancelFn()
+	ctx = grpcx.WriteWorkerID(ctx, *id)
 	defer func() {
 		if r := recover(); r != nil {
 			fmt.Fprintf(os.Stderr, "Worker panic: %v\n", r)
 			debug.PrintStack()
+			log.Warnf(ctx, "Worker panic detected: %v", r)
+
+			if tempLocation := beam.PipelineOptions.Get("temp_location"); tempLocation != "" {
+				log.Warnf(ctx, "Capturing heap dump")
+				harness.UploadHeapDump(ctx, fmt.Sprintf("%v/heapDumps/dump-%d", strings.TrimSuffix(tempLocation, "/"), time.Now().Unix()))

Review Comment:
   Please also use the worker-id `*id` and not just the time.



##########
sdks/go/pkg/beam/runners/dataflow/dataflow.go:
##########
@@ -117,18 +117,19 @@ var flagFilter = map[string]bool{
 	"network":                        true,
 	"subnetwork":                     true,
 	"no_use_public_ips":              true,
-	"temp_location":                  true,
-	"template_location":              true,
-	"worker_machine_type":            true,
-	"min_cpu_platform":               true,
-	"dataflow_worker_jar":            true,
-	"worker_region":                  true,
-	"worker_zone":                    true,
-	"teardown_policy":                true,
-	"cpu_profiling":                  true,
-	"session_recording":              true,
-	"update":                         true,
-	"transform_name_mapping":         true,
+	// Don't filter temp_location since we need this included in PipelineOptions to correctly upload heap dumps.
+	// "temp_location":                  true,

Review Comment:
   I'd remove this line entirely, but still have a comment about it somewhere.



-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org