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

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_r364962334
 
 

 ##########
 File path: 
runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/status/BeamWorkerStatusGrpcService.java
 ##########
 @@ -92,90 +92,97 @@ public void close() throws Exception {
 
     WorkerStatusClient fnApiStatusClient =
         WorkerStatusClient.forRequestObserver(workerId, requestObserver);
-    fnApiStatusClient.setDeregisterCallback(this.connectedClient::remove);
-    if (connectedClient.containsKey(workerId) && 
connectedClient.get(workerId).isDone()) {
-      LOG.info(
-          "SDK Worker {} was connected to status server previously, 
disconnecting the old client",
-          workerId);
-      try {
-        WorkerStatusClient oldClient = connectedClient.get(workerId).get();
-        oldClient.close();
-      } catch (IOException | InterruptedException | ExecutionException e) {
-        LOG.warn("Error closing worker status client", e);
-      }
-    }
-    connectedClient
-        .computeIfAbsent(workerId, k -> new CompletableFuture<>())
-        .complete(fnApiStatusClient);
+    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.
+   * 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 CompletableFuture<WorkerStatusResponse> getWorkerStatus(String 
workerId) {
+  public String getSingleWorkerStatus(String workerId, long timeout, TimeUnit 
timeUnit) {
+    if (!connectedClient.containsKey(workerId)) {
+      return "Error: Not connected.";
+    }
     try {
-      return getStatusClient(workerId).getWorkerStatus();
+      return getWorkerStatus(workerId).get(timeout, timeUnit);
     } catch (InterruptedException | ExecutionException | TimeoutException e) {
-      if (e instanceof InterruptedException) {
-        Thread.currentThread().interrupt();
-      }
-      CompletableFuture<WorkerStatusResponse> error = new 
CompletableFuture<>();
-      error.completeExceptionally(e);
-      return error;
+      return handleAndReturnExceptionResponse(e);
     }
   }
 
   /**
-   * Get all the statuses from all connected SDK harnesses within specified 
timeout.
+   * 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 all SDK harness.
+   * @param timeout max time waiting for the response from each SDK harness.
    * @param timeUnit timeout time unit.
-   * @return All the statuses in a Map which key is the SDK harness id and 
value is the status
-   *     content.
+   * @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());
 
-    List<CompletableFuture<WorkerStatusResponse>> responses = new 
ArrayList<>();
-    for (String id : connectedClient.keySet()) {
-      responses.add(
-          getWorkerStatus(id)
-              .whenComplete(
-                  (response, ex) -> {
-                    allStatuses.put(
-                        id, ex != null ? DEFAULT_ERROR_RESPONSE : 
getStatusErrorOrInfo(response));
-                  }));
-    }
+    connectedClient
+        .keySet()
+        .parallelStream()
 
 Review comment:
   This can throw a 
[ConcurrentModificationException](https://docs.oracle.com/javase/7/docs/api/java/util/ConcurrentModificationException.html)
 if the parallel stream is over a connectedClient that is being mutated while 
your iterating over it.
 
----------------------------------------------------------------
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: 369373)
    Time Spent: 14h 40m  (was: 14.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: 14h 40m
>  Remaining Estimate: 0h
>




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

Reply via email to