lostluck commented on a change in pull request #16957:
URL: https://github.com/apache/beam/pull/16957#discussion_r837997728
##########
File path: sdks/go/pkg/beam/core/runtime/harness/harness.go
##########
@@ -98,6 +111,18 @@ func Main(ctx context.Context, loggingEndpoint,
controlEndpoint string) error {
log.Debugf(ctx, "control response channel closed")
}()
+ // if the runner supports worker status api then expose SDK harness
status
+ if statusEndpoint != "" {
+ statusHandler, err := newWorkerStatusHandler(ctx,
statusEndpoint)
+ if err != nil {
+ log.Error(ctx, err)
+ }
+ var swg sync.WaitGroup
+ swg.Add(1)
Review comment:
Since this WaitGroup is only used inside the status handler, and never
called elsewhere, could we move it to be a field in workerStatusHandler
instead? That way the WG doesn't need to be a pointer, and it `Add(1)` can be
called on the statusHandler's field in `handleRequests`
As a rule of thumb, whenever one is passing around a side value everywhere a
struct you own goes, it's worth thinking about putting it into the struct.
##########
File path: sdks/go/pkg/beam/core/runtime/harness/harness.go
##########
@@ -98,6 +111,18 @@ func Main(ctx context.Context, loggingEndpoint,
controlEndpoint string) error {
log.Debugf(ctx, "control response channel closed")
}()
+ // if the runner supports worker status api then expose SDK harness
status
+ if statusEndpoint != "" {
+ statusHandler, err := newWorkerStatusHandler(ctx,
statusEndpoint)
+ if err != nil {
+ log.Error(ctx, err)
+ }
+ var swg sync.WaitGroup
+ swg.Add(1)
+ statusHandler.handleRequest(ctx, &swg)
+ defer statusHandler.close(ctx, &swg)
Review comment:
It feels like this block (after the error `if`) is one of those cases
where this should be wrapped in an `else` after the error processing. `
log.Error` won't abort worker set up, so as written, if the set up returns
an error, we still do these allocations and set ups, but those will fail
anyway. So in my assessment, why go ahead with them on failure?
The defer for the `close` call is good, and will ensure things get cleaned
up when the main function returns.
##########
File path: sdks/go/pkg/beam/core/runtime/harness/harness.go
##########
@@ -98,6 +111,18 @@ func Main(ctx context.Context, loggingEndpoint,
controlEndpoint string) error {
log.Debugf(ctx, "control response channel closed")
}()
+ // if the runner supports worker status api then expose SDK harness
status
+ if statusEndpoint != "" {
+ statusHandler, err := newWorkerStatusHandler(ctx,
statusEndpoint)
+ if err != nil {
+ log.Error(ctx, err)
Review comment:
Please add context about what this is logging for, right now all that is
known is "that it fails to connect", but nothing says it's trying to connect
for a worker status handler.
Similar
##########
File path: sdks/go/pkg/beam/core/runtime/harness/worker_status_test.go
##########
@@ -0,0 +1,87 @@
+// 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 harness
+
+import (
+ "context"
+ "fmt"
+ "log"
+ "net"
+ "sync"
+ "testing"
+
+ fnpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/fnexecution_v1"
+ "google.golang.org/grpc"
+ "google.golang.org/grpc/test/bufconn"
+)
+
+type BeamFnWorkerStatusServicer struct {
+ fnpb.UnimplementedBeamFnWorkerStatusServer
+ response chan string
+ lis *bufconn.Listener
+}
+
+func (w *BeamFnWorkerStatusServicer) WorkerStatus(b
fnpb.BeamFnWorkerStatus_WorkerStatusServer) error {
+ b.Send(&fnpb.WorkerStatusRequest{Id: "1"})
+ resp, err := b.Recv()
+ if err != nil {
+ return fmt.Errorf("error receiving response b.recv: %v", err)
+ }
+ w.response <- resp.GetStatusInfo()
+ return nil
+}
+
+const buffsize = 1024 * 1024
+
+var lis *bufconn.Listener
+
+func setup(srv *BeamFnWorkerStatusServicer) {
+
Review comment:
nit: rm blank line
##########
File path: sdks/go/pkg/beam/core/runtime/harness/worker_status.go
##########
@@ -0,0 +1,91 @@
+// 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 harness
+
+import (
+ "context"
+ "io"
+ "runtime"
+ "sync"
+ "sync/atomic"
+ "time"
+
+ "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors"
+ "github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+ fnpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/fnexecution_v1"
+ "google.golang.org/grpc"
+)
+
+// workerStatusHandler stores the communication information of WorkerStatus
API.
+type workerStatusHandler struct {
+ conn *grpc.ClientConn
+ shutdown int32
+}
+
+func newWorkerStatusHandler(ctx context.Context, endpoint string)
(*workerStatusHandler, error) {
+ sconn, err := dial(ctx, endpoint, 60*time.Second)
+ if err != nil {
+ return &workerStatusHandler{}, errors.Wrapf(err, "failed to
connect: %v\n", endpoint)
+ }
+ return &workerStatusHandler{conn: sconn, shutdown: 0}, nil
+}
+
+func (w *workerStatusHandler) isAlive() bool {
+ return atomic.LoadInt32(&w.shutdown) == 0
+}
+
+func (w *workerStatusHandler) stop() {
+ atomic.StoreInt32(&w.shutdown, 1)
+}
+
+// handleRequest manages the WorkerStatus API.
+func (w *workerStatusHandler) handleRequest(ctx context.Context, wg
*sync.WaitGroup) {
+ statusClient := fnpb.NewBeamFnWorkerStatusClient(w.conn)
+ stub, err := statusClient.WorkerStatus(ctx)
+ if err != nil {
+ log.Errorf(ctx, "status client not established: %v", err)
+ return
+ }
+ go w.reader(ctx, stub, wg)
+}
+
+// reader reads the WorkerStatusRequest from the stream and sends a processed
WorkerStatusResponse to
+// a response channel.
+func (w *workerStatusHandler) reader(ctx context.Context, stub
fnpb.BeamFnWorkerStatus_WorkerStatusClient, wg *sync.WaitGroup) {
+ defer wg.Done()
+ buf := make([]byte, 1<<16)
+ for w.isAlive() {
+ req, err := stub.Recv()
+ if err != nil {
+ log.Debugf(ctx, "exiting workerStatusHandler.Reader():
%v", err)
+ return
+ }
+ log.Debugf(ctx, "RECV-status: %v", req.GetId())
+ runtime.Stack(buf, true)
+ response := &fnpb.WorkerStatusResponse{Id: req.GetId(),
StatusInfo: string(buf)}
+ if err := stub.Send(response); err != nil && err != io.EOF {
+ log.Errorf(ctx, "workerStatus.Writer: Failed to
respond: %v", err)
+ }
+ }
+}
+
+// close stops the reader first, closes the response channel thereby stopping
writer and finally closes the gRPC connection.
Review comment:
Please update the documentation, writer no longer exists.
##########
File path: sdks/go/pkg/beam/core/runtime/harness/worker_status.go
##########
@@ -0,0 +1,91 @@
+// 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 harness
+
+import (
+ "context"
+ "io"
+ "runtime"
+ "sync"
+ "sync/atomic"
+ "time"
+
+ "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors"
+ "github.com/apache/beam/sdks/v2/go/pkg/beam/log"
+ fnpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/fnexecution_v1"
+ "google.golang.org/grpc"
+)
+
+// workerStatusHandler stores the communication information of WorkerStatus
API.
+type workerStatusHandler struct {
+ conn *grpc.ClientConn
+ shutdown int32
+}
+
+func newWorkerStatusHandler(ctx context.Context, endpoint string)
(*workerStatusHandler, error) {
+ sconn, err := dial(ctx, endpoint, 60*time.Second)
+ if err != nil {
+ return &workerStatusHandler{}, errors.Wrapf(err, "failed to
connect: %v\n", endpoint)
+ }
+ return &workerStatusHandler{conn: sconn, shutdown: 0}, nil
+}
+
+func (w *workerStatusHandler) isAlive() bool {
+ return atomic.LoadInt32(&w.shutdown) == 0
+}
+
+func (w *workerStatusHandler) stop() {
+ atomic.StoreInt32(&w.shutdown, 1)
+}
+
+// handleRequest manages the WorkerStatus API.
+func (w *workerStatusHandler) handleRequest(ctx context.Context, wg
*sync.WaitGroup) {
Review comment:
naming nit: This doesn't handle a single request. `start` or
`startHandling` are better names for this, which describes what it does.
The only nit is then it doesn't pair with `close` seamlessly.
I do like `stop()` for the atomic wrapper method, and you're already using
`shutdown` for the atomic field.
What do you think about:
Name this method `start`
Rename the `shutdown` field to `shouldShutdown`
Keep `stop` for the atomic wrapper.
Use `shutdown` instead of `close`.
?
##########
File path: sdks/go/pkg/beam/core/runtime/harness/worker_status_test.go
##########
@@ -0,0 +1,87 @@
+// 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 harness
+
+import (
+ "context"
+ "fmt"
+ "log"
+ "net"
+ "sync"
+ "testing"
+
+ fnpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/fnexecution_v1"
+ "google.golang.org/grpc"
+ "google.golang.org/grpc/test/bufconn"
+)
+
+type BeamFnWorkerStatusServicer struct {
+ fnpb.UnimplementedBeamFnWorkerStatusServer
+ response chan string
+ lis *bufconn.Listener
+}
+
+func (w *BeamFnWorkerStatusServicer) WorkerStatus(b
fnpb.BeamFnWorkerStatus_WorkerStatusServer) error {
+ b.Send(&fnpb.WorkerStatusRequest{Id: "1"})
+ resp, err := b.Recv()
+ if err != nil {
+ return fmt.Errorf("error receiving response b.recv: %v", err)
+ }
+ w.response <- resp.GetStatusInfo()
+ return nil
+}
+
+const buffsize = 1024 * 1024
+
+var lis *bufconn.Listener
+
+func setup(srv *BeamFnWorkerStatusServicer) {
Review comment:
Please pass in `t` here and also have `t.Cleanup` be called so the grpc
server can be Stopped at the end of the test, and avoid a leak.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]