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

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

                Author: ASF GitHub Bot
            Created on: 13/Sep/18 08:36
            Start Date: 13/Sep/18 08:36
    Worklog Time Spent: 10m 
      Work Description: asfgit closed pull request #6301: [BEAM-5262] Add 
Reference runner support for add state stream
URL: https://github.com/apache/beam/pull/6301
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/ReferenceRunnerJobService.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/ReferenceRunnerJobService.java
index 9627152c100..b58959b338c 100644
--- 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/ReferenceRunnerJobService.java
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/ReferenceRunnerJobService.java
@@ -36,6 +36,8 @@
 import org.apache.beam.model.jobmanagement.v1.JobApi.CancelJobResponse;
 import org.apache.beam.model.jobmanagement.v1.JobApi.GetJobStateRequest;
 import org.apache.beam.model.jobmanagement.v1.JobApi.GetJobStateResponse;
+import org.apache.beam.model.jobmanagement.v1.JobApi.JobMessagesRequest;
+import org.apache.beam.model.jobmanagement.v1.JobApi.JobMessagesResponse;
 import org.apache.beam.model.jobmanagement.v1.JobApi.JobState;
 import org.apache.beam.model.jobmanagement.v1.JobApi.JobState.Enum;
 import org.apache.beam.model.jobmanagement.v1.JobApi.PrepareJobResponse;
@@ -56,6 +58,7 @@
 /** The ReferenceRunner uses the portability framework to execute a Pipeline 
on a single machine. */
 public class ReferenceRunnerJobService extends JobServiceImplBase implements 
FnService {
   private static final Logger LOG = 
LoggerFactory.getLogger(ReferenceRunnerJobService.class);
+  private static final int WAIT_MS = 1000;
 
   public static ReferenceRunnerJobService create(final ServerFactory 
serverFactory) {
     LOG.info("Starting {}", ReferenceRunnerJobService.class);
@@ -201,6 +204,36 @@ public void getState(
     responseObserver.onCompleted();
   }
 
+  @Override
+  public void getStateStream(
+      GetJobStateRequest request, StreamObserver<GetJobStateResponse> 
responseObserver) {
+    LOG.trace("{} {}", GetJobStateRequest.class.getSimpleName(), request);
+    String invocationId = request.getJobId();
+    try {
+      Thread.sleep(WAIT_MS);
+      Enum state = jobStates.getOrDefault(request.getJobId(), 
Enum.UNRECOGNIZED);
+      
responseObserver.onNext(GetJobStateResponse.newBuilder().setState(state).build());
+      while (Enum.RUNNING.equals(state)) {
+        Thread.sleep(WAIT_MS);
+        state = jobStates.getOrDefault(request.getJobId(), Enum.UNRECOGNIZED);
+      }
+      
responseObserver.onNext(GetJobStateResponse.newBuilder().setState(state).build());
+    } catch (Exception e) {
+      String errMessage =
+          String.format("Encountered Unexpected Exception for Invocation %s", 
invocationId);
+      LOG.error(errMessage, e);
+      responseObserver.onError(Status.INTERNAL.withCause(e).asException());
+    }
+    responseObserver.onCompleted();
+  }
+
+  @Override
+  public void getMessageStream(
+      JobMessagesRequest request, StreamObserver<JobMessagesResponse> 
responseObserver) {
+    // Not implemented
+    LOG.trace("{} {}", JobMessagesRequest.class.getSimpleName(), request);
+  }
+
   @Override
   public void cancel(CancelJobRequest request, 
StreamObserver<CancelJobResponse> responseObserver) {
     LOG.trace("{} {}", CancelJobRequest.class.getSimpleName(), request);


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on 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: 143888)
    Time Spent: 1h 20m  (was: 1h 10m)

> JobState support for Reference Runner
> -------------------------------------
>
>                 Key: BEAM-5262
>                 URL: https://issues.apache.org/jira/browse/BEAM-5262
>             Project: Beam
>          Issue Type: Bug
>          Components: runner-direct
>            Reporter: Ankur Goenka
>            Assignee: Ankur Goenka
>            Priority: Minor
>          Time Spent: 1h 20m
>  Remaining Estimate: 0h
>
> Reference runner does not support getStateStream which is needed by portable 
> SDK



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to