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

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

                Author: ASF GitHub Bot
            Created on: 28/Aug/19 09:05
            Start Date: 28/Aug/19 09:05
    Worklog Time Spent: 10m 
      Work Description: mxm commented on pull request #9440: [BEAM-5428] Modify 
cache token Proto design to only include tokens in ProcessBundleRequest
URL: https://github.com/apache/beam/pull/9440#discussion_r318471119
 
 

 ##########
 File path: model/fn-execution/src/main/proto/beam_fn_api.proto
 ##########
 @@ -232,9 +232,35 @@ message ProcessBundleRequest {
   // instantiated and executed by the SDK harness.
   string process_bundle_descriptor_reference = 1;
 
+  // A cache token which can be used by an SDK to check for the validity
+  // of cached elements which have a cache token associated.
+  message CacheToken {
+
+    // A flag to indicate a cache token is valid for user state.
+    message UserState {}
+
+    // A flag to indicate a cache token is valid for a side input.
+    message SideInput {
+      // The id of a side input.
+      string side_input = 1;
+    }
+
+    // The scope of a cache token.
+    oneof type {
+      UserState user_state = 1;
+      SideInput side_input = 2;
+    }
+
+    // The cache token identifier which should be globally unique.
+    bytes token = 10;
+  }
+
   // (Optional) A list of cache tokens that can be used by an SDK to reuse
   // cached data returned by the State API across multiple bundles.
-  repeated bytes cache_tokens = 2;
+  repeated CacheToken cache_tokens = 3;
+
+  // Old version of cache_tokens field
+  reserved 2;
 
 Review comment:
   Not sure we need to do this since cache tokens were never set by the 
Runner/handled on the SDK side.
 
----------------------------------------------------------------
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: 302690)
    Time Spent: 6h 50m  (was: 6h 40m)

> Implement cross-bundle state caching.
> -------------------------------------
>
>                 Key: BEAM-5428
>                 URL: https://issues.apache.org/jira/browse/BEAM-5428
>             Project: Beam
>          Issue Type: Improvement
>          Components: sdk-py-harness
>            Reporter: Robert Bradshaw
>            Assignee: Rakesh Kumar
>            Priority: Major
>          Time Spent: 6h 50m
>  Remaining Estimate: 0h
>
> Tech spec: 
> [https://docs.google.com/document/d/1BOozW0bzBuz4oHJEuZNDOHdzaV5Y56ix58Ozrqm2jFg/edit#heading=h.7ghoih5aig5m]
> Relevant document: 
> [https://docs.google.com/document/d/1ltVqIW0XxUXI6grp17TgeyIybk3-nDF8a0-Nqw-s9mY/edit#|https://docs.google.com/document/d/1ltVqIW0XxUXI6grp17TgeyIybk3-nDF8a0-Nqw-s9mY/edit]
> Mailing list link: 
> [https://lists.apache.org/thread.html/caa8d9bc6ca871d13de2c5e6ba07fdc76f85d26497d95d90893aa1f6@%3Cdev.beam.apache.org%3E]



--
This message was sent by Atlassian Jira
(v8.3.2#803003)

Reply via email to