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

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

                Author: ASF GitHub Bot
            Created on: 26/Sep/19 18:02
            Start Date: 26/Sep/19 18:02
    Worklog Time Spent: 10m 
      Work Description: mxm commented on pull request #9418: [BEAM-5428] 
Implement cross-bundle user state caching in the Python SDK
URL: https://github.com/apache/beam/pull/9418#discussion_r328751006
 
 

 ##########
 File path: sdks/python/apache_beam/runners/worker/sdk_worker.py
 ##########
 @@ -620,6 +632,107 @@ def _next_id(self):
     return str(self._last_id)
 
 
+class CachingMaterializingStateHandler(object):
+  """ A State handler which retrieves and caches state. """
+
+  def __init__(self, global_state_cache, underlying_state):
+    self._underlying = underlying_state
+    self._state_cache = global_state_cache
+    self._context = threading.local()
+
+  @contextlib.contextmanager
+  def process_instruction_id(self, bundle_id, cache_tokens):
+    if getattr(self._context, 'cache_token', None) is not None:
+      raise RuntimeError(
+          'Cache tokens already set to %s' % self._context.cache_token)
+    # TODO Also handle cache tokens for side input, if present:
+    # https://issues.apache.org/jira/browse/BEAM-8298
+    user_state_cache_token = None
+    for cache_token_struct in cache_tokens:
+      if cache_token_struct.HasField("user_state"):
+        # There should only be one user state token present
+        assert not user_state_cache_token
+        user_state_cache_token = cache_token_struct.token
+    try:
+      self._context.cache_token = user_state_cache_token
+      with self._underlying.process_instruction_id(bundle_id):
+        yield
+    finally:
+      self._context.cache_token = None
+
+  def blocking_get(self, state_key, coder, is_cached=False):
+    if not self._should_be_cached(is_cached):
+      # no cache / tokens, can't do a lookup/store in the cache
+      return self._materialize_iter(state_key, coder)
+    # Cache lookup
+    cache_state_key = self._convert_to_cache_key(state_key)
+    cached_value = self._state_cache.get(cache_state_key,
+                                         self._context.cache_token)
+    if cached_value is None:
+      # Cache miss, need to retrieve from the Runner
+      materialized = cached_value = list(
+          self._materialize_iter(state_key, coder))
+      self._state_cache.put(
+          cache_state_key,
+          self._context.cache_token,
+          materialized)
+    return iter(cached_value)
+
+  def append(self, state_key, coder, elements, is_cached=False):
+    if self._should_be_cached(is_cached):
+      # Update the cache
+      cache_key = self._convert_to_cache_key(state_key)
+      self._state_cache.append(cache_key, self._context.cache_token, elements)
+    # Write to state handler
+    out = coder_impl.create_OutputStream()
+    for element in elements:
+      coder.encode_to_stream(element, out, True)
+    return self._underlying.append_raw(state_key, out.get())
+
+  def clear(self, state_key, is_cached=False):
+    if self._should_be_cached(is_cached):
+      cache_key = self._convert_to_cache_key(state_key)
+      self._state_cache.clear(cache_key, self._context.cache_token)
+    return self._underlying.clear(state_key)
+
+  # The following methods are for interaction with the FnApiRunner:
+
+  def get_raw(self, state_key, continuation_token=None):
+    return self._underlying.get_raw(state_key, continuation_token)
+
+  def append_raw(self, state_key, data):
+    return self._underlying.append_raw(state_key, data)
+
+  def restore(self):
+    self._underlying.restore()
+
+  def checkpoint(self):
 
 Review comment:
   Note, these and the above until the comment will go away (fix will come 
tomorrow), as they won't be necessary with the cache not being inserted for the 
state handler of the fn_api_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:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 319110)
    Time Spent: 23h 50m  (was: 23h 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: Maximilian Michels
>            Priority: Major
>          Time Spent: 23h 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.4#803005)

Reply via email to