[ https://issues.apache.org/jira/browse/BEAM-8624?focusedWorklogId=369371&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-369371 ]
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_r364961777 ########## 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)) { Review comment: This check here is not needed since getWorkerStatus will return a completed future and getWorkerStatus needs to do that work otherwise you expose a race where connectedClient is mutated after this check but before getWorkerStatus is invoked. ---------------------------------------------------------------- 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: 369371) Time Spent: 14.5h (was: 14h 20m) > 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: 14.5h > Remaining Estimate: 0h > -- This message was sent by Atlassian Jira (v8.3.4#803005)