Skip to content

Commit c0a8ff6

Browse files
authored
[Go SDK] Container Worker pool functionality. (#33572)
1 parent cf26a42 commit c0a8ff6

File tree

6 files changed

+368
-9
lines changed

6 files changed

+368
-9
lines changed

sdks/go/container/boot.go

+39-1
Original file line numberDiff line numberDiff line change
@@ -28,6 +28,7 @@ import (
2828
"strings"
2929
"time"
3030

31+
"github.com/apache/beam/sdks/v2/go/container/pool"
3132
"github.com/apache/beam/sdks/v2/go/container/tools"
3233
"github.com/apache/beam/sdks/v2/go/pkg/beam/artifact"
3334
"github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime"
@@ -44,6 +45,7 @@ import (
4445
var (
4546
// Contract: https://s.apache.org/beam-fn-api-container-contract.
4647

48+
workerPool = flag.Bool("worker_pool", false, "Run as worker pool (optional).")
4749
id = flag.String("id", "", "Local identifier (required).")
4850
loggingEndpoint = flag.String("logging_endpoint", "", "Local logging endpoint for FnHarness (required).")
4951
artifactEndpoint = flag.String("artifact_endpoint", "", "Local artifact endpoint for FnHarness (required).")
@@ -56,6 +58,7 @@ const (
5658
cloudProfilingJobName = "CLOUD_PROF_JOB_NAME"
5759
cloudProfilingJobID = "CLOUD_PROF_JOB_ID"
5860
enableGoogleCloudProfilerOption = "enable_google_cloud_profiler"
61+
workerPoolIdEnv = "BEAM_GO_WORKER_POOL_ID"
5962
)
6063

6164
func configureGoogleCloudProfilerEnvVars(ctx context.Context, logger *tools.Logger, metadata map[string]string) error {
@@ -78,6 +81,30 @@ func configureGoogleCloudProfilerEnvVars(ctx context.Context, logger *tools.Logg
7881

7982
func main() {
8083
flag.Parse()
84+
85+
if *workerPool {
86+
workerPoolId := fmt.Sprintf("%d", os.Getpid())
87+
bin, err := os.Executable()
88+
if err != nil {
89+
log.Fatalf("Error starting worker pool, couldn't find boot loader path: %v", err)
90+
}
91+
92+
os.Setenv(workerPoolIdEnv, workerPoolId)
93+
log.Printf("Starting worker pool %v: Go %v binary: %vv", workerPoolId, ":50000", bin)
94+
95+
ctx := context.Background()
96+
server, err := pool.New(ctx, 50000, bin)
97+
if err != nil {
98+
log.Fatalf("Error starting worker pool: %v", err)
99+
}
100+
defer server.Stop(ctx)
101+
if err := server.ServeAndWait(); err != nil {
102+
log.Fatalf("Error with worker pool: %v", err)
103+
}
104+
log.Print("Go SDK worker pool exited.")
105+
os.Exit(0)
106+
}
107+
81108
if *id == "" {
82109
log.Fatal("No id provided.")
83110
}
@@ -126,7 +153,13 @@ func main() {
126153

127154
// (3) The persist dir may be on a noexec volume, so we must
128155
// copy the binary to a different location to execute.
129-
const prog = "/bin/worker"
156+
tmpPrefix, err := os.MkdirTemp("/tmp/", "bin*")
157+
if err != nil {
158+
logger.Fatalf(ctx, "Failed to copy worker binary: %v", err)
159+
}
160+
161+
prog := tmpPrefix + "/worker"
162+
logger.Printf(ctx, "From: %q To:%q", filepath.Join(dir, name), prog)
130163
if err := copyExe(filepath.Join(dir, name), prog); err != nil {
131164
logger.Fatalf(ctx, "Failed to copy worker binary: %v", err)
132165
}
@@ -233,6 +266,11 @@ func copyExe(from, to string) error {
233266
}
234267
defer src.Close()
235268

269+
// Ensure that the folder path exists locally.
270+
if err := os.MkdirAll(filepath.Dir(to), 0755); err != nil {
271+
return err
272+
}
273+
236274
dst, err := os.OpenFile(to, os.O_WRONLY|os.O_CREATE, 0755)
237275
if err != nil {
238276
return err

sdks/go/container/pool/workerpool.go

+160
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,160 @@
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+
}
+150
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,150 @@
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
17+
18+
import (
19+
"context"
20+
"os/exec"
21+
"testing"
22+
23+
fnpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/fnexecution_v1"
24+
pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1"
25+
)
26+
27+
func TestProcess(t *testing.T) {
28+
// Use the no-op true binary, if available, skip this test otherwise.
29+
dummyExec, err := exec.LookPath("true")
30+
if err != nil {
31+
t.Skip("Binary `true` doesn't exist, skipping tests.")
32+
}
33+
34+
endpoint := &pipepb.ApiServiceDescriptor{
35+
Url: "localhost:0",
36+
}
37+
secureEndpoint := &pipepb.ApiServiceDescriptor{
38+
Url: "localhost:0",
39+
Authentication: &pipepb.AuthenticationSpec{
40+
Urn: "beam:authentication:oauth2_client_credentials_grant:v1",
41+
},
42+
}
43+
44+
ctx, cancelFn := context.WithCancel(context.Background())
45+
t.Cleanup(cancelFn)
46+
server, err := New(ctx, 0, dummyExec)
47+
if err != nil {
48+
t.Fatalf("Unable to create server: %v", err)
49+
}
50+
go server.ServeAndWait()
51+
52+
startTests := []struct {
53+
req *fnpb.StartWorkerRequest
54+
errExpected bool
55+
}{
56+
{
57+
req: &fnpb.StartWorkerRequest{
58+
WorkerId: "Worker1",
59+
ControlEndpoint: endpoint,
60+
LoggingEndpoint: endpoint,
61+
},
62+
}, {
63+
req: &fnpb.StartWorkerRequest{
64+
WorkerId: "Worker2",
65+
ControlEndpoint: endpoint,
66+
LoggingEndpoint: endpoint,
67+
},
68+
}, {
69+
req: &fnpb.StartWorkerRequest{
70+
WorkerId: "Worker1",
71+
ControlEndpoint: endpoint,
72+
LoggingEndpoint: endpoint,
73+
},
74+
errExpected: true, // Repeated start
75+
}, {
76+
req: &fnpb.StartWorkerRequest{
77+
WorkerId: "missingControl",
78+
LoggingEndpoint: endpoint,
79+
},
80+
errExpected: true,
81+
}, {
82+
req: &fnpb.StartWorkerRequest{
83+
WorkerId: "missingLogging",
84+
ControlEndpoint: endpoint,
85+
},
86+
errExpected: true,
87+
}, {
88+
req: &fnpb.StartWorkerRequest{
89+
WorkerId: "secureLogging",
90+
LoggingEndpoint: secureEndpoint,
91+
ControlEndpoint: endpoint,
92+
},
93+
errExpected: true,
94+
}, {
95+
req: &fnpb.StartWorkerRequest{
96+
WorkerId: "secureControl",
97+
LoggingEndpoint: endpoint,
98+
ControlEndpoint: secureEndpoint,
99+
},
100+
errExpected: true,
101+
},
102+
}
103+
for _, test := range startTests {
104+
resp, err := server.StartWorker(ctx, test.req)
105+
if test.errExpected {
106+
if err != nil || resp.Error == "" {
107+
t.Errorf("Expected error starting %v: err: %v, resp: %v", test.req.GetWorkerId(), err, resp)
108+
}
109+
} else {
110+
if err != nil || resp.Error != "" {
111+
t.Errorf("Unexpected error starting %v: err: %v, resp: %v", test.req.GetWorkerId(), err, resp)
112+
}
113+
}
114+
}
115+
stopTests := []struct {
116+
req *fnpb.StopWorkerRequest
117+
errExpected bool
118+
}{
119+
{
120+
req: &fnpb.StopWorkerRequest{
121+
WorkerId: "Worker1",
122+
},
123+
}, {
124+
req: &fnpb.StopWorkerRequest{
125+
WorkerId: "Worker1",
126+
},
127+
errExpected: true,
128+
}, {
129+
req: &fnpb.StopWorkerRequest{
130+
WorkerId: "NonExistent",
131+
},
132+
errExpected: true,
133+
},
134+
}
135+
for _, test := range stopTests {
136+
resp, err := server.StopWorker(ctx, test.req)
137+
if test.errExpected {
138+
if err != nil || resp.Error == "" {
139+
t.Errorf("Expected error starting %v: err: %v, resp: %v", test.req.GetWorkerId(), err, resp)
140+
}
141+
} else {
142+
if err != nil || resp.Error != "" {
143+
t.Errorf("Unexpected error starting %v: err: %v, resp: %v", test.req.GetWorkerId(), err, resp)
144+
}
145+
}
146+
}
147+
if err := server.Stop(ctx); err != nil {
148+
t.Fatalf("error stopping server: err: %v", err)
149+
}
150+
}

0 commit comments

Comments
 (0)