|
| 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 pool facilitates a external worker service, as an alternate mode for |
| 17 | +// the standard Beam container. |
| 18 | +// |
| 19 | +// This is predeominantly to serve as a process spawner within a given container |
| 20 | +// VM for an arbitrary number of jobs, instead of for a single worker instance. |
| 21 | +// |
| 22 | +// Workers will be spawned as executed OS processes. |
| 23 | +package pool |
| 24 | + |
| 25 | +import ( |
| 26 | + "context" |
| 27 | + "fmt" |
| 28 | + "log/slog" |
| 29 | + "net" |
| 30 | + "os" |
| 31 | + "os/exec" |
| 32 | + "sync" |
| 33 | + |
| 34 | + fnpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/fnexecution_v1" |
| 35 | + "github.com/apache/beam/sdks/v2/go/pkg/beam/util/grpcx" |
| 36 | + "google.golang.org/grpc" |
| 37 | +) |
| 38 | + |
| 39 | +// New initializes a process based ExternalWorkerService, at the given |
| 40 | +// port. |
| 41 | +func New(ctx context.Context, port int, containerExecutable string) (*Process, error) { |
| 42 | + lis, err := net.Listen("tcp", fmt.Sprintf(":%d", port)) |
| 43 | + if err != nil { |
| 44 | + return nil, err |
| 45 | + } |
| 46 | + slog.Info("starting Process server", "addr", lis.Addr()) |
| 47 | + grpcServer := grpc.NewServer() |
| 48 | + root, cancel := context.WithCancel(ctx) |
| 49 | + s := &Process{lis: lis, root: root, rootCancel: cancel, workers: map[string]context.CancelFunc{}, |
| 50 | + grpcServer: grpcServer, containerExecutable: containerExecutable} |
| 51 | + fnpb.RegisterBeamFnExternalWorkerPoolServer(grpcServer, s) |
| 52 | + return s, nil |
| 53 | +} |
| 54 | + |
| 55 | +// ServeAndWait starts the ExternalWorkerService and blocks until exit. |
| 56 | +func (s *Process) ServeAndWait() error { |
| 57 | + return s.grpcServer.Serve(s.lis) |
| 58 | +} |
| 59 | + |
| 60 | +// Process implements fnpb.BeamFnExternalWorkerPoolServer, by starting external |
| 61 | +// processes. |
| 62 | +type Process struct { |
| 63 | + fnpb.UnimplementedBeamFnExternalWorkerPoolServer |
| 64 | + |
| 65 | + containerExecutable string // The host for the container executable. |
| 66 | + |
| 67 | + lis net.Listener |
| 68 | + root context.Context |
| 69 | + rootCancel context.CancelFunc |
| 70 | + |
| 71 | + mu sync.Mutex |
| 72 | + workers map[string]context.CancelFunc |
| 73 | + |
| 74 | + grpcServer *grpc.Server |
| 75 | +} |
| 76 | + |
| 77 | +// StartWorker initializes a new worker harness, implementing BeamFnExternalWorkerPoolServer.StartWorker. |
| 78 | +func (s *Process) StartWorker(_ context.Context, req *fnpb.StartWorkerRequest) (*fnpb.StartWorkerResponse, error) { |
| 79 | + slog.Info("starting worker", "id", req.GetWorkerId()) |
| 80 | + s.mu.Lock() |
| 81 | + defer s.mu.Unlock() |
| 82 | + if s.workers == nil { |
| 83 | + return &fnpb.StartWorkerResponse{ |
| 84 | + Error: "worker pool shutting down", |
| 85 | + }, nil |
| 86 | + } |
| 87 | + |
| 88 | + if _, ok := s.workers[req.GetWorkerId()]; ok { |
| 89 | + return &fnpb.StartWorkerResponse{ |
| 90 | + Error: fmt.Sprintf("worker with ID %q already exists", req.GetWorkerId()), |
| 91 | + }, nil |
| 92 | + } |
| 93 | + if req.GetLoggingEndpoint() == nil { |
| 94 | + return &fnpb.StartWorkerResponse{Error: fmt.Sprintf("Missing logging endpoint for worker %v", req.GetWorkerId())}, nil |
| 95 | + } |
| 96 | + if req.GetControlEndpoint() == nil { |
| 97 | + return &fnpb.StartWorkerResponse{Error: fmt.Sprintf("Missing control endpoint for worker %v", req.GetWorkerId())}, nil |
| 98 | + } |
| 99 | + if req.GetLoggingEndpoint().Authentication != nil || req.GetControlEndpoint().Authentication != nil { |
| 100 | + return &fnpb.StartWorkerResponse{Error: "[BEAM-10610] Secure endpoints not supported."}, nil |
| 101 | + } |
| 102 | + |
| 103 | + ctx := grpcx.WriteWorkerID(s.root, req.GetWorkerId()) |
| 104 | + ctx, s.workers[req.GetWorkerId()] = context.WithCancel(ctx) |
| 105 | + |
| 106 | + args := []string{ |
| 107 | + "--id=" + req.GetWorkerId(), |
| 108 | + "--control_endpoint=" + req.GetControlEndpoint().GetUrl(), |
| 109 | + "--artifact_endpoint=" + req.GetArtifactEndpoint().GetUrl(), |
| 110 | + "--provision_endpoint=" + req.GetProvisionEndpoint().GetUrl(), |
| 111 | + "--logging_endpoint=" + req.GetLoggingEndpoint().GetUrl(), |
| 112 | + } |
| 113 | + |
| 114 | + cmd := exec.CommandContext(ctx, s.containerExecutable, args...) |
| 115 | + cmd.Stdin = os.Stdin |
| 116 | + cmd.Stdout = os.Stdout |
| 117 | + cmd.Stderr = os.Stderr |
| 118 | + cmd.Env = nil // Use the current environment. |
| 119 | + |
| 120 | + if err := cmd.Start(); err != nil { |
| 121 | + return &fnpb.StartWorkerResponse{Error: fmt.Sprintf("Unable to start boot for worker %v: %v", req.GetWorkerId(), err)}, nil |
| 122 | + } |
| 123 | + return &fnpb.StartWorkerResponse{}, nil |
| 124 | +} |
| 125 | + |
| 126 | +// StopWorker terminates a worker harness, implementing BeamFnExternalWorkerPoolServer.StopWorker. |
| 127 | +func (s *Process) StopWorker(_ context.Context, req *fnpb.StopWorkerRequest) (*fnpb.StopWorkerResponse, error) { |
| 128 | + slog.Info("stopping worker", "id", req.GetWorkerId()) |
| 129 | + s.mu.Lock() |
| 130 | + defer s.mu.Unlock() |
| 131 | + if s.workers == nil { |
| 132 | + // Worker pool is already shutting down, so no action is needed. |
| 133 | + return &fnpb.StopWorkerResponse{}, nil |
| 134 | + } |
| 135 | + if cancelfn, ok := s.workers[req.GetWorkerId()]; ok { |
| 136 | + cancelfn() |
| 137 | + delete(s.workers, req.GetWorkerId()) |
| 138 | + return &fnpb.StopWorkerResponse{}, nil |
| 139 | + } |
| 140 | + return &fnpb.StopWorkerResponse{ |
| 141 | + Error: fmt.Sprintf("no worker with id %q running", req.GetWorkerId()), |
| 142 | + }, nil |
| 143 | + |
| 144 | +} |
| 145 | + |
| 146 | +// Stop terminates the service and stops all workers. |
| 147 | +func (s *Process) Stop(ctx context.Context) error { |
| 148 | + s.mu.Lock() |
| 149 | + |
| 150 | + slog.Debug("stopping Process", "worker_count", len(s.workers)) |
| 151 | + s.workers = nil |
| 152 | + s.rootCancel() |
| 153 | + |
| 154 | + // There can be a deadlock between the StopWorker RPC and GracefulStop |
| 155 | + // which waits for all RPCs to finish, so it must be outside the critical section. |
| 156 | + s.mu.Unlock() |
| 157 | + |
| 158 | + s.grpcServer.GracefulStop() |
| 159 | + return nil |
| 160 | +} |
0 commit comments