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

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

                Author: ASF GitHub Bot
            Created on: 09/Jan/20 21:22
            Start Date: 09/Jan/20 21:22
    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_r364965191
 
 

 ##########
 File path: 
runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/status/WorkerStatusClient.java
 ##########
 @@ -0,0 +1,165 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Consumer;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.WorkerStatusRequest;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.WorkerStatusResponse;
+import org.apache.beam.sdk.fn.IdGenerator;
+import org.apache.beam.sdk.fn.IdGenerators;
+import org.apache.beam.sdk.fn.stream.SynchronizedStreamObserver;
+import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Client for handling requests and responses over Fn Worker Status Api 
between runner and SDK
+ * Harness.
+ */
+class WorkerStatusClient implements Closeable {
+
+  public static final Logger LOG = 
LoggerFactory.getLogger(WorkerStatusClient.class);
+  private final IdGenerator idGenerator = IdGenerators.incrementingLongs();
+  private final StreamObserver<WorkerStatusRequest> requestReceiver;
+  private final Map<String, CompletableFuture<WorkerStatusResponse>> 
responseQueue =
+      new ConcurrentHashMap<>();
+  private final String workerId;
+  private Consumer<String> deregisterCallback;
+  private AtomicBoolean isClosed = new AtomicBoolean(false);
+
+  private WorkerStatusClient(String workerId, 
StreamObserver<WorkerStatusRequest> requestReceiver) {
+    this.requestReceiver = 
SynchronizedStreamObserver.wrapping(requestReceiver);
+    this.workerId = workerId;
+  }
+
+  /**
+   * Create new status api client with SDK Harness worker id and request 
observer.
+   *
+   * @param workerId SDK Harness worker id.
+   * @param requestObserver The outbound request observer this client uses to 
send new status
+   *     requests to its corresponding SDK Harness.
+   * @return {@link WorkerStatusClient}
+   */
+  public static WorkerStatusClient forRequestObserver(
+      String workerId, StreamObserver<WorkerStatusRequest> requestObserver) {
+    return new WorkerStatusClient(workerId, requestObserver);
+  }
+
+  /**
+   * Get the latest sdk worker status from the client's corresponding SDK 
Harness. A random id will
+   * be used to specify the request_id field.
+   *
+   * @return {@link CompletableFuture} of the SDK Harness status response.
+   */
+  public CompletableFuture<WorkerStatusResponse> getWorkerStatus() {
+    WorkerStatusRequest request =
+        WorkerStatusRequest.newBuilder().setId(idGenerator.getId()).build();
+    return getWorkerStatus(request);
+  }
+
+  /**
+   * Get the latest sdk worker status from the client's corresponding SDK 
Harness with request.
+   *
+   * @param request WorkerStatusRequest to be sent to SDK Harness.
+   * @return {@link CompletableFuture} of the SDK Harness status response.
+   */
+  CompletableFuture<WorkerStatusResponse> getWorkerStatus(WorkerStatusRequest 
request) {
+    CompletableFuture<WorkerStatusResponse> future = new CompletableFuture<>();
 
 Review comment:
   This race still exists since adding to `pendingResponses` isn't synchronized 
with the iteration in `close()`
   
   You want to wrap pending responses mutation with synchronization:
   ```
   synchronized (pendingResponses) {
     this.pendingResponses.put(...);
   }
   ```
 
----------------------------------------------------------------
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: 369374)
    Time Spent: 14h 50m  (was: 14h 40m)

> 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: 14h 50m
>  Remaining Estimate: 0h
>




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

Reply via email to