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

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

                Author: ASF GitHub Bot
            Created on: 03/Dec/19 00:41
            Start Date: 03/Dec/19 00:41
    Worklog Time Spent: 10m 
      Work Description: angoenka commented on pull request #10115: [BEAM-8624] 
Implement Worker Status FnService in Dataflow runner
URL: https://github.com/apache/beam/pull/10115#discussion_r352933561
 
 

 ##########
 File path: 
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/status/BeamWorkerStatusGrpcService.java
 ##########
 @@ -0,0 +1,140 @@
+/*
+ * 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.dataflow.worker.fn.status;
+
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+import javax.annotation.Nullable;
+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.runners.fnexecution.status.WorkerStatusClient;
+import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+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 {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(BeamWorkerStatusGrpcService.class);
+  private static final String UNKNOW_SDK_ID_PREFIX = "unknown_sdk";
+  private final AtomicLong idGenerator = new AtomicLong();
+  private final HeaderAccessor headerAccessor;
+  private final Map<String, 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 ApiServiceDescriptor used for hosting to the 
server.
+   * @param headerAccessor Grpc head accessor used to obtain SDK Harness 
worker id.
+   * @return {@link BeamWorkerStatusGrpcService} if apiServiceDescriptor is 
valid, otherwise returns
+   *     null.
+   */
+  @Nullable
+  public static BeamWorkerStatusGrpcService create(
+      ApiServiceDescriptor apiServiceDescriptor, HeaderAccessor 
headerAccessor) {
+    if (apiServiceDescriptor == null || 
Strings.isNullOrEmpty(apiServiceDescriptor.getUrl())) {
+      LOG.info(
+          "Received empty api service descriptor for status service,"
+              + " the sdk status report will be skipped");
+      return null;
+    }
+    return new BeamWorkerStatusGrpcService(apiServiceDescriptor, 
headerAccessor);
+  }
+
+  @Override
+  public void close() throws Exception {
+    for (WorkerStatusClient client : this.connectedClient.values()) {
+      client.close();
+    }
+    this.connectedClient.clear();
+  }
+
+  @Override
+  public StreamObserver<WorkerStatusResponse> workerStatus(
+      StreamObserver<WorkerStatusRequest> requestObserver) {
+    String workerId = headerAccessor.getSdkWorkerId();
+    if (Strings.isNullOrEmpty(workerId)) {
+      // SDK harness status report should be enforced with proper sdk worker 
id.
+      // If SDK harness connect without proper sdk worker id, generate one 
with name unknown_sdkXX.
+      // TODO: enforce proper worker id.
+      workerId = UNKNOW_SDK_ID_PREFIX + idGenerator.getAndIncrement();
+      LOG.info(
+          "No worker_id header provided in status response. Will use generated 
id: {}", workerId);
+    }
+
+    LOG.info("Beam Fn Status client connected with id {}", workerId);
+    WorkerStatusClient fnApiStatusClient =
 
 Review comment:
   Shall we setup a timer to periodically poll the status?
 
----------------------------------------------------------------
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:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 352344)
    Time Spent: 4h  (was: 3h 50m)

> 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: 4h
>  Remaining Estimate: 0h
>




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

Reply via email to