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

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

                Author: ASF GitHub Bot
            Created on: 04/Mar/20 00:23
            Start Date: 04/Mar/20 00:23
    Worklog Time Spent: 10m 
      Work Description: robertwb commented on pull request #11005: [BEAM-8335] 
Modify the StreamingCache to subclass the CacheManager 
URL: https://github.com/apache/beam/pull/11005#discussion_r387373091
 
 

 ##########
 File path: 
sdks/python/apache_beam/runners/interactive/caching/streaming_cache.py
 ##########
 @@ -19,15 +19,298 @@
 
 from __future__ import absolute_import
 
+import itertools
+import os
+import shutil
+import tempfile
+import time
+
+import apache_beam as beam
+from apache_beam.portability.api.beam_interactive_api_pb2 import 
TestStreamFileHeader
+from apache_beam.portability.api.beam_interactive_api_pb2 import 
TestStreamFileRecord
 from apache_beam.portability.api.beam_runner_api_pb2 import TestStreamPayload
+from apache_beam.runners.interactive.cache_manager import CacheManager
+from apache_beam.runners.interactive.cache_manager import 
SafeFastPrimitivesCoder
+from apache_beam.testing.test_stream import ReverseTestStream
 from apache_beam.utils import timestamp
 
 
-class StreamingCache(object):
+class StreamingCacheSink(beam.PTransform):
+  """A PTransform that writes TestStreamFile(Header|Records)s to file.
+
+  This transform takes in an arbitrary element stream and writes the 
best-effort
+  list of TestStream events (as TestStreamFileRecords) to file.
+
+  Note that this PTransform is assumed to be only run on a single machine where
+  the following assumptions are correct: elements come in ordered, no two
+  transforms are writing to the same file. This PTransform is assumed to only
+  run correctly with the DirectRunner.
+  """
+  def __init__(
+      self,
+      cache_dir,
+      filename,
+      sample_resolution_sec,
+      coder=SafeFastPrimitivesCoder()):
+    self._cache_dir = cache_dir
+    self._filename = filename
+    self._sample_resolution_sec = sample_resolution_sec
+    self._coder = coder
+    self._path = os.path.join(self._cache_dir, self._filename)
+
+  @property
+  def path(self):
+    """Returns the path the sink leads to."""
+    return self._path
+
+  def expand(self, pcoll):
+    class StreamingWriteToText(beam.DoFn):
+      """DoFn that performs the writing.
+
+      Note that the other file writing methods cannot be used in streaming
+      contexts.
+      """
+      def __init__(self, full_path, coder=SafeFastPrimitivesCoder()):
+        self._full_path = full_path
+        self._coder = coder
+
+        # Try and make the given path.
+        os.makedirs(os.path.dirname(full_path), exist_ok=True)
+
+      def start_bundle(self):
+        # Open the file for 'append-mode' and writing 'bytes'.
+        self._fh = open(self._full_path, 'ab')
+
+      def finish_bundle(self):
+        self._fh.close()
+
+      def process(self, e):
+        """Appends the given element to the file.
+        """
+        self._fh.write(self._coder.encode(e))
+        self._fh.write(b'\n')
+
+    return (
+        pcoll
+        | ReverseTestStream(
+            output_tag=self._filename,
+            sample_resolution_sec=self._sample_resolution_sec,
+            output_format=ReverseTestStream.Format.
+            SERIALIZED_TEST_STREAM_FILE_RECORDS,
+            coder=self._coder)
+        | beam.ParDo(
+            StreamingWriteToText(full_path=self._path, coder=self._coder)))
+
+
+class StreamingCacheSource:
+  """A class that reads and parses TestStreamFile(Header|Reader)s.
+
+  This source operates in the following way:
+
+    1. Wait for up to `timeout_secs` for the file to be available.
+    2. Read, parse, and emit the entire contents of the file
+    3. Wait for more events to come or until `is_cache_complete` returns True
+    4. If there are more events, then go to 2
+    5. Otherwise, stop emitting.
+
+  This class is used to read from file and send its to the TestStream via the
+  StreamingCacheManager.Reader.
+  """
+  def __init__(
+      self,
+      cache_dir,
+      labels,
+      is_cache_complete=None,
+      coder=SafeFastPrimitivesCoder()):
+    self._cache_dir = cache_dir
+    self._coder = coder
+    self._labels = labels
+    self._is_cache_complete = (
+        is_cache_complete if is_cache_complete else lambda: True)
+
+  def _wait_until_file_exists(self, timeout_secs=30):
+    """Blocks until the file exists for a maximum of timeout_secs.
+    """
+    f = None
+    now_secs = time.time()
+    timeout_timestamp_secs = now_secs + timeout_secs
+
+    # Wait for up to `timeout_secs` for the file to be available.
+    while f is None and now_secs < timeout_timestamp_secs:
+      now_secs = time.time()
+      try:
+        path = os.path.join(self._cache_dir, *self._labels)
+        f = open(path, mode='r')
+      except EnvironmentError as e:
+        # For Python2 and Python3 compatibility, this checks the
+        # EnvironmentError to see if the file exists.
+        # TODO: Change this to a FileNotFoundError when Python3 migration is
+        # complete.
+        import errno
+        if e.errno != errno.ENOENT:
+          # Raise the exception if it is not a FileNotFoundError.
+          raise
+        time.sleep(1)
+    if now_secs >= timeout_timestamp_secs:
+      raise RuntimeError(
+          "Timed out waiting for file '{}' to be available".format(path))
+    return f
+
+  def _emit_from_file(self, fh, tail):
+    """Emits the TestStreamFile(Header|Record)s from file.
+
+    This returns a generator to be able to read all lines from the given file.
+    If `tail` is True, then it will wait until the cache is complete to exit.
+    Otherwise, it will read the file only once.
+    """
+    # Always read at least once to read the whole file.
+    while True:
+      pos = fh.tell()
+      line = fh.readline()
+
+      # Check if we are at EOF.
+      if not line:
+        # Complete reading only when the cache is complete.
+        if self._is_cache_complete():
+          break
+
+        if not tail:
+          break
+
+        # Otherwise wait for new data in the file to be written.
+        time.sleep(0.5)
+        fh.seek(pos)
+      else:
+        # The first line at pos = 0 is always the header. Read the line without
+        # the new line.
+        if pos == 0:
+          header = TestStreamFileHeader()
+          header.ParseFromString(self._coder.decode(line[:-1]))
+          yield header
+        else:
+          record = TestStreamFileRecord()
+          record.ParseFromString(self._coder.decode(line[:-1]))
+          yield record
+
+  def read(self, tail):
+    """Reads all TestStreamFile(Header|TestStreamFileRecord)s from file.
+
+    This returns a generator to be able to read all lines from the given file.
+    If `tail` is True, then it will wait until the cache is complete to exit.
+    Otherwise, it will read the file only once.
+    """
+    with self._wait_until_file_exists() as f:
+      for e in self._emit_from_file(f, tail):
+        yield e
+
+
+class StreamingCache(CacheManager):
   """Abstraction that holds the logic for reading and writing to cache.
   """
-  def __init__(self, readers):
-    self._readers = readers
+  def __init__(
+      self, cache_dir, is_cache_complete=None, sample_resolution_sec=0.1):
+    self._sample_resolution_sec = sample_resolution_sec
+    self._is_cache_complete = is_cache_complete
+
+    if cache_dir:
+      self._cache_dir = cache_dir
+    else:
+      self._cache_dir = tempfile.mkdtemp(
+          prefix='interactive-temp-', dir=os.environ.get('TEST_TMPDIR', None))
+
+    # List of saved pcoders keyed by PCollection path. It is OK to keep this
+    # list in memory because once FileBasedCacheManager object is
+    # destroyed/re-created it loses the access to previously written cache
+    # objects anyways even if cache_dir already exists. In other words,
+    # it is not possible to resume execution of Beam pipeline from the
+    # saved cache if FileBasedCacheManager has been reset.
+    #
+    # However, if we are to implement better cache persistence, one needs
+    # to take care of keeping consistency between the cached PCollection
+    # and its PCoder type.
+    self._saved_pcoders = {}
+    self._default_pcoder = SafeFastPrimitivesCoder()
+
+  def exists(self, *labels):
+    path = os.path.join(self._cache_dir, *labels)
+    return os.path.exists(path)
+
+  # TODO(srohde): Modify this to return the correct version.
+  def read(self, *labels):
+    """Returns a generator to read all records from file.
+
+    Does not tail.
+    """
+    if not self.exists(*labels):
+      return [].__iter__(), -1
+
+    reader = StreamingCacheSource(
+        self._cache_dir, labels,
+        is_cache_complete=self._is_cache_complete).read(tail=False)
+    header = next(reader)
+    return StreamingCache.Reader([header], [reader]).read(), 1
+
+  def read_multiple(self, labels):
+    """Returns a generator to read all records from file.
+
+    Does tail until the cache is complete. This is because it is used in the
+    TestStreamServiceController to read from file which is only used during
+    pipeline runtime which needs to block.
+    """
+    readers = [
+        StreamingCacheSource(
+            self._cache_dir, l,
+            is_cache_complete=self._is_cache_complete).read(tail=True)
+        for l in labels
+    ]
+    headers = [next(r) for r in readers]
+    return StreamingCache.Reader(headers, readers).read()
+
+  def write(self, values, *labels):
+    """Writes the given values to cache.
+    """
+    to_write = [v.SerializeToString() for v in values]
 
 Review comment:
   Why is this not done in the loop below?
 
----------------------------------------------------------------
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: 397147)
    Time Spent: 86h  (was: 85h 50m)

> 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: 86h
>  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