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

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

                Author: ASF GitHub Bot
            Created on: 22/Oct/19 18:52
            Start Date: 22/Oct/19 18:52
    Worklog Time Spent: 10m 
      Work Description: davidyan74 commented on pull request #9720: [BEAM-8335] 
Add initial modules for interactive streaming support
URL: https://github.com/apache/beam/pull/9720#discussion_r337691444
 
 

 ##########
 File path: model/interactive/src/main/proto/beam_interactive_api.proto
 ##########
 @@ -0,0 +1,160 @@
+/*
+ * 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.
+ */
+
+/*
+ * Protocol Buffers describing a service that can be used in conjunction with
+ * the TestStream class in order to control a pipeline remotely.
+ */
+
+syntax = "proto3";
+
+package org.apache.beam.model.interactive.v1;
+
+option go_package = "interactive_v1";
+option java_package = "org.apache.beam.model.interactive.v1";
+option java_outer_classname = "BeamInteractiveApi";
+
+import "beam_runner_api.proto";
+import "google/protobuf/timestamp.proto";
+import "google/protobuf/duration.proto";
+
+
+service InteractiveService {
+  // A TestStream will request for events using this RPC.
+  rpc Events(EventsRequest) returns (stream EventsResponse) {}
+
+  // Starts the stream of events to the EventsRequest.
+  rpc Start (StartRequest) returns (StartResponse) {}
+
+  // Advances the stream to the specified offset then pauses the stream. This
+  // starts the stream if it is not RUNNING.
+  rpc Advance (AdvanceRequest) returns (AdvanceResponse) {}
+
+  // Stops and resets the stream to the beginning.
+  rpc Stop (StopRequest) returns (StopResponse) {}
+
+  // Pauses the stream of events to the EventsRequest. If there is already an
+  // outstanding EventsRequest streaming events, then the stream will pause
+  // after the EventsResponse is completed.
+  // To un-pause, send either a Start or Advance request.
+  rpc Pause (PauseRequest) returns (PauseResponse) {}
+
+  // Sends a single element to the EventsRequest then closes the stream.
+  rpc Step (StepRequest) returns (StepResponse) {}
+
+  // Responds with debugging and other cache-specific metadata.
+  rpc Status (StatusRequest) returns (StatusResponse) {}
+}
+
+message StartRequest {
+  // (Optional) How quickly the stream will be played back, e.g. if
+  // playback_speed == 2, then the stream will replay events twice as fast as
+  // they were recorded. If unspecified, this will default to 1.
+  double playback_speed = 1;
+
+  // (Optional) if present, will start the stream at the specified timestamp.
+  google.protobuf.Timestamp start_at = 2;
 
 Review comment:
   I'm not sure whether we should allow this feature since that means 
re-execution with start_at specified means different input data and it kinda 
defeats the purpose of replaying. Also, if we use Start method for also 
unpausing and changing playback speed, start_at either should be ignored or 
should return an error.
 
----------------------------------------------------------------
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: 332149)
    Time Spent: 7h 40m  (was: 7.5h)

> Add streaming support to Interactive Beam
> -----------------------------------------
>
>                 Key: BEAM-8335
>                 URL: https://issues.apache.org/jira/browse/BEAM-8335
>             Project: Beam
>          Issue Type: Improvement
>          Components: runner-py-interactive
>            Reporter: Sam Rohde
>            Assignee: Sam Rohde
>            Priority: Major
>          Time Spent: 7h 40m
>  Remaining Estimate: 0h
>
> This issue tracks the work items to introduce streaming support to the 
> Interactive Beam experience. This will allow users to:
>  * Write and run a streaming job in IPython
>  * Automatically cache records from unbounded sources
>  * Add a replay experience that replays all cached records to simulate the 
> original pipeline execution
>  * Add controls to play/pause/stop/step individual elements from the cached 
> records
>  * Add ability to inspect/visualize unbounded PCollections



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

Reply via email to