[ https://issues.apache.org/jira/browse/BEAM-8624?focusedWorklogId=369469&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-369469 ]
ASF GitHub Bot logged work on BEAM-8624: ---------------------------------------- Author: ASF GitHub Bot Created on: 10/Jan/20 00:03 Start Date: 10/Jan/20 00:03 Worklog Time Spent: 10m Work Description: lukecwik commented on pull request #10115: [BEAM-8624] Implement Worker Status FnService in Dataflow runner URL: https://github.com/apache/beam/pull/10115#discussion_r365017047 ########## File path: runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/status/BeamWorkerStatusGrpcService.java ########## @@ -0,0 +1,207 @@ +/* + * 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 org.apache.beam.runners.fnexecution.status; + +import java.io.IOException; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import org.apache.beam.model.fnexecution.v1.BeamFnApi.WorkerStatusRequest; +import org.apache.beam.model.fnexecution.v1.BeamFnApi.WorkerStatusResponse; +import org.apache.beam.model.fnexecution.v1.BeamFnWorkerStatusGrpc.BeamFnWorkerStatusImplBase; +import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor; +import org.apache.beam.runners.fnexecution.FnService; +import org.apache.beam.runners.fnexecution.HeaderAccessor; +import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A Fn Status service which can collect run-time status information from SDK harnesses for + * debugging purpose. + */ +public class BeamWorkerStatusGrpcService extends BeamFnWorkerStatusImplBase implements FnService { + + private static final Logger LOG = LoggerFactory.getLogger(BeamWorkerStatusGrpcService.class); + private static final String DEFAULT_EXCEPTION_RESPONSE = + "Error: exception encountered getting status from SDK harness"; + + private final HeaderAccessor headerAccessor; + private final Map<String, CompletableFuture<WorkerStatusClient>> connectedClient = + Collections.synchronizedMap(new HashMap<>()); + + private BeamWorkerStatusGrpcService( + ApiServiceDescriptor apiServiceDescriptor, HeaderAccessor headerAccessor) { + this.headerAccessor = headerAccessor; + LOG.info("Launched Beam Fn Status service at {}", apiServiceDescriptor); + } + + /** + * Create new instance of {@link BeamWorkerStatusGrpcService}. + * + * @param apiServiceDescriptor describes the configuration for the endpoint the server will + * expose. + * @param headerAccessor headerAccessor gRPC header accessor used to obtain SDK harness worker id. + * @return {@link BeamWorkerStatusGrpcService} + */ + public static BeamWorkerStatusGrpcService create( + ApiServiceDescriptor apiServiceDescriptor, HeaderAccessor headerAccessor) { + return new BeamWorkerStatusGrpcService(apiServiceDescriptor, headerAccessor); + } + + @Override + public void close() throws Exception { + synchronized (connectedClient) { + for (CompletableFuture<WorkerStatusClient> clientFuture : connectedClient.values()) { + if (clientFuture.isDone()) { + clientFuture.get().close(); + } + } + connectedClient.clear(); + } + } + + @Override + public StreamObserver<WorkerStatusResponse> workerStatus( + StreamObserver<WorkerStatusRequest> requestObserver) { + String workerId = headerAccessor.getSdkWorkerId(); + LOG.info("Beam Fn Status client connected with id {}", workerId); + + WorkerStatusClient fnApiStatusClient = + WorkerStatusClient.forRequestObserver(workerId, requestObserver); + connectedClient.compute( + workerId, + (k, existingClientFuture) -> { + if (existingClientFuture != null) { + try { + if (existingClientFuture.isDone()) { + LOG.info( + "SDK Worker {} was connected to status server previously, disconnecting old client", + workerId); + existingClientFuture.get().close(); + } else { + existingClientFuture.complete(fnApiStatusClient); + return existingClientFuture; + } + } catch (IOException | InterruptedException | ExecutionException e) { + LOG.warn("Error closing worker status client", e); + } + } + return CompletableFuture.completedFuture(fnApiStatusClient); + }); + return fnApiStatusClient.getResponseObserver(); + } + + /** + * Get the latest SDK worker status from the client's corresponding SDK harness. + * + * @param workerId worker id of the SDK harness. + * @return {@link CompletableFuture} of WorkerStatusResponse from SDK harness. + */ + public String getSingleWorkerStatus(String workerId, long timeout, TimeUnit timeUnit) { + try { + return getWorkerStatus(workerId).get(timeout, timeUnit); + } catch (InterruptedException | ExecutionException | TimeoutException e) { + return handleAndReturnExceptionResponse(e); + } + } + + /** + * Get all the statuses from all connected SDK harnesses within specified timeout. Any errors + * getting status from the SDK harnesses will be returned in the map. + * + * @param timeout max time waiting for the response from each SDK harness. + * @param timeUnit timeout time unit. + * @return All the statuses in a map keyed by the SDK harness id. + */ + public Map<String, String> getAllWorkerStatuses(long timeout, TimeUnit timeUnit) { + // return result in worker id sorted map. + Map<String, String> allStatuses = new ConcurrentSkipListMap<>(Comparator.naturalOrder()); + + Set<String> connectedClientIdsCopy = ImmutableSet.copyOf(connectedClient.keySet()); Review comment: Even though connectedClient is a synchronous map, iteration over it still needs to be synchronized. See: https://docs.oracle.com/javase/7/docs/api/java/util/Collections.html#synchronizedMap(java.util.Map) Internally within `ImmutableSet.copyOf`, it iterates over the collection of values. ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 369469) Time Spent: 15h 40m (was: 15.5h) > Implement FnService for status api in Dataflow runner > ----------------------------------------------------- > > Key: BEAM-8624 > URL: https://issues.apache.org/jira/browse/BEAM-8624 > Project: Beam > Issue Type: Sub-task > Components: runner-dataflow > Reporter: Yichi Zhang > Assignee: Yichi Zhang > Priority: Major > Time Spent: 15h 40m > Remaining Estimate: 0h > -- This message was sent by Atlassian Jira (v8.3.4#803005)