github.com/apache/beam/sdks/v2@v2.48.2/go/container/boot.go (about) 1 // Licensed to the Apache Software Foundation (ASF) under one or more 2 // contributor license agreements. See the NOTICE file distributed with 3 // this work for additional information regarding copyright ownership. 4 // The ASF licenses this file to You under the Apache License, Version 2.0 5 // (the "License"); you may not use this file except in compliance with 6 // the License. You may obtain a copy of the License at 7 // 8 // http://www.apache.org/licenses/LICENSE-2.0 9 // 10 // Unless required by applicable law or agreed to in writing, software 11 // distributed under the License is distributed on an "AS IS" BASIS, 12 // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 // See the License for the specific language governing permissions and 14 // limitations under the License. 15 16 package main 17 18 import ( 19 "context" 20 "encoding/json" 21 "errors" 22 "flag" 23 "fmt" 24 "io" 25 "log" 26 "os" 27 "path/filepath" 28 "strings" 29 "time" 30 31 "github.com/apache/beam/sdks/v2/go/container/tools" 32 "github.com/apache/beam/sdks/v2/go/pkg/beam/artifact" 33 "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime" 34 35 // Import gcs filesystem so that it can be used to upload heap dumps 36 _ "github.com/apache/beam/sdks/v2/go/pkg/beam/io/filesystem/gcs" 37 fnpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/fnexecution_v1" 38 pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" 39 "github.com/apache/beam/sdks/v2/go/pkg/beam/util/diagnostics" 40 "github.com/apache/beam/sdks/v2/go/pkg/beam/util/execx" 41 "github.com/apache/beam/sdks/v2/go/pkg/beam/util/grpcx" 42 ) 43 44 var ( 45 // Contract: https://s.apache.org/beam-fn-api-container-contract. 46 47 id = flag.String("id", "", "Local identifier (required).") 48 loggingEndpoint = flag.String("logging_endpoint", "", "Local logging endpoint for FnHarness (required).") 49 artifactEndpoint = flag.String("artifact_endpoint", "", "Local artifact endpoint for FnHarness (required).") 50 provisionEndpoint = flag.String("provision_endpoint", "", "Local provision endpoint for FnHarness (required).") 51 controlEndpoint = flag.String("control_endpoint", "", "Local control endpoint for FnHarness (required).") 52 semiPersistDir = flag.String("semi_persist_dir", "/tmp", "Local semi-persistent directory (optional).") 53 ) 54 55 const ( 56 cloudProfilingJobName = "CLOUD_PROF_JOB_NAME" 57 cloudProfilingJobID = "CLOUD_PROF_JOB_ID" 58 enableGoogleCloudProfilerOption = "enable_google_cloud_profiler" 59 ) 60 61 func configureGoogleCloudProfilerEnvVars(ctx context.Context, logger *tools.Logger, metadata map[string]string) error { 62 if metadata == nil { 63 return errors.New("enable_google_cloud_profiler is set to true, but no metadata is received from provision server, profiling will not be enabled") 64 } 65 jobName, nameExists := metadata["job_name"] 66 if !nameExists { 67 return errors.New("required job_name missing from metadata, profiling will not be enabled without it") 68 } 69 jobID, idExists := metadata["job_id"] 70 if !idExists { 71 return errors.New("required job_id missing from metadata, profiling will not be enabled without it") 72 } 73 os.Setenv(cloudProfilingJobName, jobName) 74 os.Setenv(cloudProfilingJobID, jobID) 75 logger.Printf(ctx, "Cloud Profiling Job Name: %v, Job IDL %v", jobName, jobID) 76 return nil 77 } 78 79 func main() { 80 flag.Parse() 81 if *id == "" { 82 log.Fatal("No id provided.") 83 } 84 if *provisionEndpoint == "" { 85 log.Fatal("No provision endpoint provided.") 86 } 87 88 ctx := grpcx.WriteWorkerID(context.Background(), *id) 89 90 info, err := tools.ProvisionInfo(ctx, *provisionEndpoint) 91 if err != nil { 92 log.Fatalf("Failed to obtain provisioning information: %v", err) 93 } 94 log.Printf("Provision info:\n%v", info) 95 96 err = ensureEndpointsSet(info) 97 if err != nil { 98 log.Fatalf("Endpoint not set: %v", err) 99 } 100 logger := &tools.Logger{Endpoint: *loggingEndpoint} 101 logger.Printf(ctx, "Initializing Go harness: %v", strings.Join(os.Args, " ")) 102 103 // (1) Obtain the pipeline options 104 105 options, err := tools.ProtoToJSON(info.GetPipelineOptions()) 106 if err != nil { 107 logger.Fatalf(ctx, "Failed to convert pipeline options: %v", err) 108 } 109 110 // (2) Retrieve the staged files. 111 // 112 // The Go SDK harness downloads the worker binary and invokes 113 // it. The binary is required to be keyed as "worker", if there 114 // are more than one artifact. 115 116 dir := filepath.Join(*semiPersistDir, "staged") 117 artifacts, err := artifact.Materialize(ctx, *artifactEndpoint, info.GetDependencies(), info.GetRetrievalToken(), dir) 118 if err != nil { 119 logger.Fatalf(ctx, "Failed to retrieve staged files: %v", err) 120 } 121 122 name, err := getGoWorkerArtifactName(ctx, logger, artifacts) 123 if err != nil { 124 logger.Fatalf(ctx, "Failed to get Go Worker Artifact Name: %v", err) 125 } 126 127 // (3) The persist dir may be on a noexec volume, so we must 128 // copy the binary to a different location to execute. 129 const prog = "/bin/worker" 130 if err := copyExe(filepath.Join(dir, name), prog); err != nil { 131 logger.Fatalf(ctx, "Failed to copy worker binary: %v", err) 132 } 133 134 args := []string{ 135 "--worker=true", 136 "--id=" + *id, 137 "--logging_endpoint=" + *loggingEndpoint, 138 "--control_endpoint=" + *controlEndpoint, 139 "--semi_persist_dir=" + *semiPersistDir, 140 "--options=" + options, 141 } 142 if info.GetStatusEndpoint() != nil { 143 os.Setenv("STATUS_ENDPOINT", info.GetStatusEndpoint().GetUrl()) 144 } 145 146 if len(info.GetRunnerCapabilities()) > 0 { 147 os.Setenv("RUNNER_CAPABILITIES", strings.Join(info.GetRunnerCapabilities(), " ")) 148 } 149 150 enableGoogleCloudProfiler := strings.Contains(options, enableGoogleCloudProfilerOption) 151 if enableGoogleCloudProfiler { 152 err := configureGoogleCloudProfilerEnvVars(ctx, logger, info.Metadata) 153 if err != nil { 154 logger.Printf(ctx, "could not configure Google Cloud Profiler variables, got %v", err) 155 } 156 } 157 158 err = execx.Execute(prog, args...) 159 160 if err != nil { 161 var opt runtime.RawOptionsWrapper 162 err := json.Unmarshal([]byte(options), &opt) 163 if err == nil { 164 if tempLocation, ok := opt.Options.Options["temp_location"]; ok { 165 diagnostics.UploadHeapProfile(ctx, fmt.Sprintf("%v/heapProfiles/profile-%v-%d", strings.TrimSuffix(tempLocation, "/"), *id, time.Now().Unix())) 166 } 167 } 168 } 169 170 logger.Fatalf(ctx, "User program exited: %v", err) 171 } 172 173 func getGoWorkerArtifactName(ctx context.Context, logger *tools.Logger, artifacts []*pipepb.ArtifactInformation) (string, error) { 174 const worker = "worker" 175 name := worker 176 177 switch len(artifacts) { 178 case 0: 179 return "", errors.New("no artifacts staged") 180 case 1: 181 name, _ = artifact.MustExtractFilePayload(artifacts[0]) 182 return name, nil 183 default: 184 for _, a := range artifacts { 185 if a.GetRoleUrn() == artifact.URNGoWorkerBinaryRole { 186 name, _ = artifact.MustExtractFilePayload(a) 187 return name, nil 188 } 189 } 190 // TODO(https://github.com/apache/beam/issues/21459): Remove legacy hack once aged out. 191 for _, a := range artifacts { 192 n, _ := artifact.MustExtractFilePayload(a) 193 if n == worker { 194 logger.Printf(ctx, "Go worker binary found with legacy name '%v'", worker) 195 return n, nil 196 } 197 } 198 return "", fmt.Errorf("no artifact named '%v' found", worker) 199 } 200 } 201 202 func ensureEndpointsSet(info *fnpb.ProvisionInfo) error { 203 // TODO(BEAM-8201): Simplify once flags are no longer used. 204 if info.GetLoggingEndpoint().GetUrl() != "" { 205 *loggingEndpoint = info.GetLoggingEndpoint().GetUrl() 206 } 207 if info.GetArtifactEndpoint().GetUrl() != "" { 208 *artifactEndpoint = info.GetArtifactEndpoint().GetUrl() 209 } 210 if info.GetControlEndpoint().GetUrl() != "" { 211 *controlEndpoint = info.GetControlEndpoint().GetUrl() 212 } 213 214 if *loggingEndpoint == "" { 215 return errors.New("no logging endpoint provided") 216 } 217 if *artifactEndpoint == "" { 218 return errors.New("no artifact endpoint provided") 219 } 220 if *controlEndpoint == "" { 221 return errors.New("no control endpoint provided") 222 } 223 224 return nil 225 } 226 227 func copyExe(from, to string) error { 228 src, err := os.Open(from) 229 if err != nil { 230 return err 231 } 232 defer src.Close() 233 234 dst, err := os.OpenFile(to, os.O_WRONLY|os.O_CREATE, 0755) 235 if err != nil { 236 return err 237 } 238 239 if _, err := io.Copy(dst, src); err != nil { 240 return err 241 } 242 return dst.Close() 243 }