/
management.go
399 lines (360 loc) · 12.9 KB
/
management.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
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
// 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 jobservices
import (
"context"
"errors"
"fmt"
"sync"
"sync/atomic"
jobpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/jobmanagement_v1"
pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1"
"github.com/apache/beam/sdks/v2/go/pkg/beam/runners/prism/internal/urns"
"golang.org/x/exp/maps"
"golang.org/x/exp/slog"
"google.golang.org/protobuf/proto"
"google.golang.org/protobuf/types/known/timestamppb"
)
var (
// ErrCancel represents a pipeline cancellation by the user.
ErrCancel = errors.New("pipeline canceled")
)
func (s *Server) nextId() string {
v := atomic.AddUint32(&s.index, 1)
return fmt.Sprintf("job-%03d", v)
}
type unimplementedError struct {
feature string
value any
}
func (err unimplementedError) Error() string {
return fmt.Sprintf("unsupported feature %q set with value %v", err.feature, err.value)
}
func (err unimplementedError) LogValue() slog.Value {
return slog.GroupValue(
slog.String("feature", err.feature),
slog.Any("value", err.value))
}
// TODO migrate to errors.Join once Beam requires go1.20+
type joinError struct {
errs []error
}
func (e *joinError) Error() string {
var b []byte
for i, err := range e.errs {
if i > 0 {
b = append(b, '\n')
}
b = append(b, err.Error()...)
}
return string(b)
}
func (s *Server) Prepare(ctx context.Context, req *jobpb.PrepareJobRequest) (*jobpb.PrepareJobResponse, error) {
s.mu.Lock()
defer s.mu.Unlock()
// Since jobs execute in the background, they should not be tied to a request's context.
rootCtx, cancelFn := context.WithCancelCause(context.Background())
job := &Job{
key: s.nextId(),
Pipeline: req.GetPipeline(),
jobName: req.GetJobName(),
options: req.GetPipelineOptions(),
streamCond: sync.NewCond(&sync.Mutex{}),
RootCtx: rootCtx,
CancelFn: cancelFn,
artifactEndpoint: s.Endpoint(),
}
// Queue initial state of the job.
job.state.Store(jobpb.JobState_STOPPED)
s.jobs[job.key] = job
if err := isSupported(job.Pipeline.GetRequirements()); err != nil {
job.Failed(err)
slog.Error("unable to run job", slog.String("error", err.Error()), slog.String("jobname", req.GetJobName()))
return nil, err
}
var errs []error
check := func(feature string, got any, wants ...any) {
for _, want := range wants {
if got == want {
return
}
}
err := unimplementedError{
feature: feature,
value: got,
}
errs = append(errs, err)
}
// Inspect Transforms for unsupported features.
bypassedWindowingStrategies := map[string]bool{}
ts := job.Pipeline.GetComponents().GetTransforms()
var testStreamIds []string
for tid, t := range ts {
urn := t.GetSpec().GetUrn()
switch urn {
case urns.TransformImpulse,
urns.TransformGBK,
urns.TransformFlatten,
urns.TransformCombinePerKey,
urns.TransformCombineGlobally, // Used by Java SDK
urns.TransformCombineGroupedValues, // Used by Java SDK
urns.TransformAssignWindows:
// Very few expected transforms types for submitted pipelines.
// Most URNs are for the runner to communicate back to the SDK for execution.
case urns.TransformReshuffle:
// Reshuffles use features we don't yet support, but we would like to
// support them by making them the no-op they are, and be precise about
// what we're ignoring.
var cols []string
for _, stID := range t.GetSubtransforms() {
st := ts[stID]
// Only check the outputs, since reshuffle re-instates any previous WindowingStrategy
// so we still validate the strategy used by the input, avoiding skips.
cols = append(cols, maps.Values(st.GetOutputs())...)
}
pcs := job.Pipeline.GetComponents().GetPcollections()
for _, col := range cols {
wsID := pcs[col].GetWindowingStrategyId()
bypassedWindowingStrategies[wsID] = true
}
case urns.TransformParDo:
var pardo pipepb.ParDoPayload
if err := proto.Unmarshal(t.GetSpec().GetPayload(), &pardo); err != nil {
return nil, fmt.Errorf("unable to unmarshal ParDoPayload for %v - %q: %w", tid, t.GetUniqueName(), err)
}
// Validate all the state features
for _, spec := range pardo.GetStateSpecs() {
check("StateSpec.Protocol.Urn", spec.GetProtocol().GetUrn(), urns.UserStateBag, urns.UserStateMultiMap)
}
// Validate all the timer features
for _, spec := range pardo.GetTimerFamilySpecs() {
check("TimerFamilySpecs.TimeDomain.Urn", spec.GetTimeDomain(), pipepb.TimeDomain_EVENT_TIME)
}
check("OnWindowExpirationTimerFamily", pardo.GetOnWindowExpirationTimerFamilySpec(), "") // Unsupported for now.
case "":
// Composites can often have no spec
if len(t.GetSubtransforms()) > 0 {
continue
}
fallthrough
case urns.TransformTestStream:
var testStream pipepb.TestStreamPayload
if err := proto.Unmarshal(t.GetSpec().GetPayload(), &testStream); err != nil {
return nil, fmt.Errorf("unable to unmarshal TestStreamPayload for %v - %q: %w", tid, t.GetUniqueName(), err)
}
for _, ev := range testStream.GetEvents() {
if ev.GetProcessingTimeEvent() != nil {
check("TestStream.Event - ProcessingTimeEvents unsupported.", ev.GetProcessingTimeEvent())
}
}
t.EnvironmentId = "" // Unset the environment, to ensure it's handled prism side.
testStreamIds = append(testStreamIds, tid)
default:
check("PTransform.Spec.Urn", urn+" "+t.GetUniqueName(), "<doesn't exist>")
}
}
// At most one test stream per pipeline.
if len(testStreamIds) > 1 {
check("Multiple TestStream Transforms in Pipeline", testStreamIds)
}
// Inspect Windowing strategies for unsupported features.
for wsID, ws := range job.Pipeline.GetComponents().GetWindowingStrategies() {
check("WindowingStrategy.AllowedLateness", ws.GetAllowedLateness(), int64(0))
check("WindowingStrategy.ClosingBehaviour", ws.GetClosingBehavior(), pipepb.ClosingBehavior_EMIT_IF_NONEMPTY)
check("WindowingStrategy.AccumulationMode", ws.GetAccumulationMode(), pipepb.AccumulationMode_DISCARDING)
if ws.GetWindowFn().GetUrn() != urns.WindowFnSession {
check("WindowingStrategy.MergeStatus", ws.GetMergeStatus(), pipepb.MergeStatus_NON_MERGING)
}
if !bypassedWindowingStrategies[wsID] {
check("WindowingStrategy.OnTimeBehavior", ws.GetOnTimeBehavior(), pipepb.OnTimeBehavior_FIRE_IF_NONEMPTY, pipepb.OnTimeBehavior_FIRE_ALWAYS)
check("WindowingStrategy.OutputTime", ws.GetOutputTime(), pipepb.OutputTime_END_OF_WINDOW)
// Non nil triggers should fail.
if ws.GetTrigger().GetDefault() == nil {
check("WindowingStrategy.Trigger", ws.GetTrigger(), &pipepb.Trigger_Default{})
}
}
}
if len(errs) > 0 {
jErr := &joinError{errs: errs}
slog.Error("unable to run job", slog.String("cause", "unimplemented features"), slog.String("jobname", req.GetJobName()), slog.String("errors", jErr.Error()))
err := fmt.Errorf("found %v uses of features unimplemented in prism in job %v:\n%v", len(errs), req.GetJobName(), jErr)
job.Failed(err)
return nil, err
}
return &jobpb.PrepareJobResponse{
PreparationId: job.key,
StagingSessionToken: job.key,
ArtifactStagingEndpoint: &pipepb.ApiServiceDescriptor{
Url: s.Endpoint(),
},
}, nil
}
func (s *Server) Run(ctx context.Context, req *jobpb.RunJobRequest) (*jobpb.RunJobResponse, error) {
s.mu.Lock()
job := s.jobs[req.GetPreparationId()]
s.mu.Unlock()
// Bring up a background goroutine to allow the job to continue processing.
go s.execute(job)
return &jobpb.RunJobResponse{
JobId: job.key,
}, nil
}
// Cancel a Job requested by the CancelJobRequest for jobs not in an already terminal state.
// Otherwise, returns nil if Job does not exist or the Job's existing state as part of the CancelJobResponse.
func (s *Server) Cancel(_ context.Context, req *jobpb.CancelJobRequest) (*jobpb.CancelJobResponse, error) {
s.mu.Lock()
job, ok := s.jobs[req.GetJobId()]
s.mu.Unlock()
if !ok {
return nil, nil
}
state := job.state.Load().(jobpb.JobState_Enum)
switch state {
case jobpb.JobState_CANCELLED, jobpb.JobState_DONE, jobpb.JobState_DRAINED, jobpb.JobState_UPDATED, jobpb.JobState_FAILED:
// Already at terminal state.
return &jobpb.CancelJobResponse{
State: state,
}, nil
}
job.SendMsg("canceling " + job.String())
job.Canceling()
job.CancelFn(ErrCancel)
return &jobpb.CancelJobResponse{
State: jobpb.JobState_CANCELLING,
}, nil
}
// GetMessageStream subscribes to a stream of state changes and messages from the job. If throughput
// is high, this may cause losses of messages.
func (s *Server) GetMessageStream(req *jobpb.JobMessagesRequest, stream jobpb.JobService_GetMessageStreamServer) error {
s.mu.Lock()
job, ok := s.jobs[req.GetJobId()]
s.mu.Unlock()
if !ok {
return fmt.Errorf("job with id %v not found", req.GetJobId())
}
job.streamCond.L.Lock()
defer job.streamCond.L.Unlock()
curMsg := job.minMsg
curState := job.stateIdx
state := job.state.Load().(jobpb.JobState_Enum)
for {
for (curMsg >= job.maxMsg || len(job.msgs) == 0) && curState > job.stateIdx {
switch state {
case jobpb.JobState_CANCELLED, jobpb.JobState_DONE, jobpb.JobState_DRAINED, jobpb.JobState_UPDATED:
// Reached terminal state.
return nil
case jobpb.JobState_FAILED:
// Ensure we send an error message with the cause of the job failure.
stream.Send(&jobpb.JobMessagesResponse{
Response: &jobpb.JobMessagesResponse_MessageResponse{
MessageResponse: &jobpb.JobMessage{
MessageText: job.failureErr.Error(),
Importance: jobpb.JobMessage_JOB_MESSAGE_ERROR,
},
},
})
return nil
}
job.streamCond.Wait()
select { // Quit out if the external connection is done.
case <-stream.Context().Done():
return context.Cause(stream.Context())
default:
}
}
if curMsg < job.minMsg {
// TODO report missed messages for this stream.
curMsg = job.minMsg
}
for curMsg < job.maxMsg && len(job.msgs) > 0 {
msg := job.msgs[curMsg-job.minMsg]
stream.Send(&jobpb.JobMessagesResponse{
Response: &jobpb.JobMessagesResponse_MessageResponse{
MessageResponse: &jobpb.JobMessage{
MessageText: msg,
Importance: jobpb.JobMessage_JOB_MESSAGE_BASIC,
},
},
})
curMsg++
}
if curState <= job.stateIdx {
state = job.state.Load().(jobpb.JobState_Enum)
curState = job.stateIdx + 1
job.streamCond.L.Unlock()
stream.Send(&jobpb.JobMessagesResponse{
Response: &jobpb.JobMessagesResponse_StateResponse{
StateResponse: &jobpb.JobStateEvent{
State: state,
},
},
})
job.streamCond.L.Lock()
}
}
}
// GetJobMetrics Fetch metrics for a given job.
func (s *Server) GetJobMetrics(ctx context.Context, req *jobpb.GetJobMetricsRequest) (*jobpb.GetJobMetricsResponse, error) {
j := s.getJob(req.GetJobId())
if j == nil {
return nil, fmt.Errorf("GetJobMetrics: unknown jobID: %v", req.GetJobId())
}
return &jobpb.GetJobMetricsResponse{
Metrics: &jobpb.MetricResults{
Attempted: j.metrics.Results(tentative),
Committed: j.metrics.Results(committed),
},
}, nil
}
// GetJobs returns the set of active jobs and associated metadata.
func (s *Server) GetJobs(context.Context, *jobpb.GetJobsRequest) (*jobpb.GetJobsResponse, error) {
s.mu.Lock()
defer s.mu.Unlock()
resp := &jobpb.GetJobsResponse{}
for key, job := range s.jobs {
resp.JobInfo = append(resp.JobInfo, &jobpb.JobInfo{
JobId: key,
JobName: job.jobName,
State: job.state.Load().(jobpb.JobState_Enum),
PipelineOptions: job.options,
})
}
return resp, nil
}
// GetPipeline returns pipeline proto of the requested job id.
func (s *Server) GetPipeline(_ context.Context, req *jobpb.GetJobPipelineRequest) (*jobpb.GetJobPipelineResponse, error) {
s.mu.Lock()
defer s.mu.Unlock()
j, ok := s.jobs[req.GetJobId()]
if !ok {
return nil, fmt.Errorf("job with id %v not found", req.GetJobId())
}
return &jobpb.GetJobPipelineResponse{
Pipeline: j.Pipeline,
}, nil
}
// GetState returns the current state of the job with the requested id.
func (s *Server) GetState(_ context.Context, req *jobpb.GetJobStateRequest) (*jobpb.JobStateEvent, error) {
s.mu.Lock()
defer s.mu.Unlock()
j, ok := s.jobs[req.GetJobId()]
if !ok {
return nil, fmt.Errorf("job with id %v not found", req.GetJobId())
}
return &jobpb.JobStateEvent{
State: j.state.Load().(jobpb.JobState_Enum),
Timestamp: timestamppb.New(j.stateTime),
}, nil
}