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