[
https://issues.apache.org/jira/browse/BEAM-5428?focusedWorklogId=300859&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-300859
]
ASF GitHub Bot logged work on BEAM-5428:
----------------------------------------
Author: ASF GitHub Bot
Created on: 25/Aug/19 14:49
Start Date: 25/Aug/19 14:49
Worklog Time Spent: 10m
Work Description: rakeshcusat commented on pull request #9418:
[BEAM-5428][WIP] State caching in the Python SDK
URL: https://github.com/apache/beam/pull/9418#discussion_r317401826
##########
File path: sdks/python/apache_beam/runners/worker/sdk_worker.py
##########
@@ -613,13 +633,62 @@ def _blocking_request(self, request):
if response.error:
raise RuntimeError(response.error)
else:
+ # TODO mxm here we could update the cache token
return response
def _next_id(self):
self._last_id += 1
return str(self._last_id)
+class CachingGrpcStateHandler(GrpcStateHandler):
+ """ TODO mxm """
+
+ def __init__(self, state_stub, global_state_cache):
+ super(CachingGrpcStateHandler, self).__init__(state_stub)
+ self._state_cache = global_state_cache
+
+ def blocking_get(self, state_key, coder):
+ cache_tokens = self._context.cache_tokens
+ if not cache_tokens:
+ # no cache tokens, can't do a lookup in the cache
+ return super(CachingGrpcStateHandler, self).blocking_get(
+ state_key, coder)
+ cache_state_key = CachingGrpcStateHandler.convert_to_cache_key(state_key)
+ value = self._state_cache.get(cache_state_key, cache_tokens)
+ if not value:
+ # Cache miss, need to retrieve from the Runner
+ value = super(CachingGrpcStateHandler, self).blocking_get(
+ state_key, coder)
+ # TODO mxm uses always the first cache token for now
+ self._state_cache.put(
+ CachingGrpcStateHandler.convert_to_cache_key(state_key),
+ cache_tokens[0], value)
+ return value
+
+ def blocking_append(self, state_key, data):
+ cache_tokens = self._context.cache_tokens
+ self._state_cache.put(
Review comment:
if we are considering the current protocol then put method should return the
new token, in that case, we should caching it after updating the runner.
----------------------------------------------------------------
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: 300859)
Time Spent: 3h 50m (was: 3h 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: 3h 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)