[ 
https://issues.apache.org/jira/browse/BEAM-8624?focusedWorklogId=367618&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-367618
 ]

ASF GitHub Bot logged work on BEAM-8624:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 07/Jan/20 18:03
            Start Date: 07/Jan/20 18:03
    Worklog Time Spent: 10m 
      Work Description: y1chi commented on pull request #10115: [BEAM-8624] 
Implement Worker Status FnService in Dataflow runner
URL: https://github.com/apache/beam/pull/10115#discussion_r363876575
 
 

 ##########
 File path: 
runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/status/BeamWorkerStatusGrpcService.java
 ##########
 @@ -0,0 +1,190 @@
+/*
+ * 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.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+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.sdk.util.MoreFutures;
+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.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 {
+
+  static final String DEFAULT_ERROR_RESPONSE = "Error getting status from SDK 
harness";
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(BeamWorkerStatusGrpcService.class);
+  private static final long DEFAULT_CLIENT_CONNECTION_WAIT_TIME_SECONDS = 5;
+  private final HeaderAccessor headerAccessor;
+  private final Map<String, CompletableFuture<WorkerStatusClient>> 
connectedClient =
+      new ConcurrentHashMap<>();
+
+  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 {
+    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);
+    fnApiStatusClient.setDeregisterCallback(this.connectedClient::remove);
+    if (connectedClient.containsKey(workerId) && 
connectedClient.get(workerId).isDone()) {
 
 Review comment:
   applied change.
 
----------------------------------------------------------------
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: 367618)
    Time Spent: 13h 10m  (was: 13h)

> 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: 13h 10m
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to