-
Notifications
You must be signed in to change notification settings - Fork 4.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[#28187][prism] Basic cross language support. #28545
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -67,7 +67,7 @@ type W struct { | |
server *grpc.Server | ||
|
||
// These are the ID sources | ||
inst, bund uint64 | ||
inst uint64 | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is now being pulled from the ProcessBundle instruction abstraction directly, right? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Just the bundle descriptor IDs. The shorter lived RPCs like Progress and Split still use the Instruction IDs from the worker side, rather than the "global" one in execute used by the ElementManager. I haven't yet nailed down the best way to have "global" state accessed by the worker abstraction, vs dedicated state (like environment protos). It's getting there though. |
||
connected, stopped atomic.Bool | ||
|
||
InstReqs chan *fnpb.InstructionRequest | ||
|
@@ -76,8 +76,6 @@ type W struct { | |
mu sync.Mutex | ||
activeInstructions map[string]controlResponder // Active instructions keyed by InstructionID | ||
Descriptors map[string]*fnpb.ProcessBundleDescriptor // Stages keyed by PBDID | ||
|
||
D *DataService | ||
} | ||
|
||
type controlResponder interface { | ||
|
@@ -104,8 +102,6 @@ func New(id, env string) *W { | |
|
||
activeInstructions: make(map[string]controlResponder), | ||
Descriptors: make(map[string]*fnpb.ProcessBundleDescriptor), | ||
|
||
D: &DataService{}, | ||
} | ||
slog.Debug("Serving Worker components", slog.String("endpoint", wk.Endpoint())) | ||
fnpb.RegisterBeamFnControlServer(wk.server, wk) | ||
|
@@ -149,11 +145,7 @@ func (wk *W) Stop() { | |
} | ||
|
||
func (wk *W) NextInst() string { | ||
return fmt.Sprintf("inst%03d", atomic.AddUint64(&wk.inst, 1)) | ||
} | ||
|
||
func (wk *W) NextStage() string { | ||
return fmt.Sprintf("stage%03d", atomic.AddUint64(&wk.bund, 1)) | ||
return fmt.Sprintf("inst-%v-%03d", wk.Env, atomic.AddUint64(&wk.inst, 1)) | ||
} | ||
|
||
// TODO set logging level. | ||
|
@@ -263,6 +255,11 @@ func (wk *W) Connected() bool { | |
return wk.connected.Load() | ||
} | ||
|
||
// Stopped indicates that the worker has stopped. | ||
func (wk *W) Stopped() bool { | ||
return wk.stopped.Load() | ||
} | ||
|
||
// Control relays instructions to SDKs and back again, coordinated via unique instructionIDs. | ||
// | ||
// Requests come from the runner, and are sent to the client in the SDK. | ||
|
@@ -312,10 +309,12 @@ func (wk *W) Control(ctrl fnpb.BeamFnControl_ControlServer) error { | |
wk.mu.Lock() | ||
// Fail extant instructions | ||
slog.Debug("SDK Disconnected", "worker", wk, "ctx_error", ctrl.Context().Err(), "outstanding_instructions", len(wk.activeInstructions)) | ||
|
||
msg := fmt.Sprintf("SDK worker disconnected: %v, %v active instructions", wk.String(), len(wk.activeInstructions)) | ||
for instID, b := range wk.activeInstructions { | ||
b.Respond(&fnpb.InstructionResponse{ | ||
InstructionId: instID, | ||
Error: "SDK Disconnected", | ||
Error: msg, | ||
}) | ||
} | ||
wk.mu.Unlock() | ||
|
@@ -536,7 +535,7 @@ func (wk *W) sendInstruction(ctx context.Context, req *fnpb.InstructionRequest) | |
|
||
req.InstructionId = progInst | ||
|
||
if wk.stopped.Load() { | ||
if wk.Stopped() { | ||
return nil | ||
} | ||
wk.InstReqs <- req | ||
|
@@ -566,6 +565,7 @@ func (wk *W) MonitoringMetadata(ctx context.Context, unknownIDs []string) *fnpb. | |
|
||
// DataService is slated to be deleted in favour of stage based state | ||
// management for side inputs. | ||
// TODO(https://github.com/apache/beam/issues/28543), remove this concept. | ||
type DataService struct { | ||
mu sync.Mutex | ||
// TODO actually quick process the data to windows here as well. | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is there a case where Stopped() could be indicative of an error in worker start-up?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No. The stopped bit is only set when Stop is called, which only happens when a context has cancelled, which only happens on post Job clean up.
Other errors would be reported earlier (ideally).
This approach largely prevents this ("worker didn't connect") function from failing successful jobs after the job completed, if the job never needed the environment at all. I intend to change worker environments to start on demand, and move worker startup to after preprocessing. As it stands, a minute+ job using one of these will just die. Won't happen for real xlang transforms though.