Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Commit 19aaad2

Browse files
author
KevinGG
committed
Fixed the cache_manager write implementation.
Addtionally: 1. Disabled writing arbitrary typed elements in streaming cache. 2. Temporarily use SafeFastPrimitivesCoder for WindowedValueHolder wrapped elements (i.e., all types of elements). 3. Fixed forward the screenshot integration test dependency version.
1 parent e95b18a commit 19aaad2

6 files changed

Lines changed: 44 additions & 57 deletions

File tree

sdks/python/apache_beam/runners/interactive/cache_manager.py

Lines changed: 24 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@
2929
from apache_beam.io import textio
3030
from apache_beam.io import tfrecordio
3131
from apache_beam.transforms import combiners
32+
from apache_beam.runners.interactive.utils import obfuscate
3233

3334

3435
class CacheManager(object):
@@ -165,7 +166,7 @@ def __init__(self, cache_dir=None, cache_format='text'):
165166
self._cache_dir = cache_dir
166167
else:
167168
self._cache_dir = tempfile.mkdtemp(
168-
prefix='it-', dir=os.environ.get('TEST_TMPDIR', None))
169+
prefix='ib-', dir=os.environ.get('TEST_TMPDIR', None))
169170
self._versions = collections.defaultdict(lambda: self._CacheVersion())
170171
self.cache_format = cache_format
171172

@@ -208,7 +209,12 @@ def save_pcoder(self, pcoder, *labels):
208209

209210
def load_pcoder(self, *labels):
210211
saved_pcoder = self._saved_pcoders.get(self._path(*labels), None)
211-
return self._default_pcoder if saved_pcoder is None else saved_pcoder
212+
# TODO(BEAM-12506): Get rid of the SafeFastPrimitivesCoder for
213+
# WindowedValueHolder.
214+
if saved_pcoder is None or isinstance(saved_pcoder,
215+
coders.FastPrimitivesCoder):
216+
return self._default_pcoder
217+
return saved_pcoder
212218

213219
def read(self, *labels, **args):
214220
# Return an iterator to an empty list if it doesn't exist.
@@ -224,11 +230,22 @@ def read(self, *labels, **args):
224230
return reader, version
225231

226232
def write(self, values, *labels):
227-
sink = self.sink(labels)._sink
228-
path = self._path(*labels)
233+
"""Imitates how a WriteCache tranform works without running a pipeline.
229234
230-
init_result = sink.initialize_write()
231-
writer = sink.open_writer(init_result, path)
235+
For testing and cache manager development, not for production usage because
236+
the write is not sharded and does not use Beam execution model.
237+
"""
238+
pcoder = coders.registry.get_coder(type(values[0]))
239+
# Save the pcoder for the actual labels.
240+
self.save_pcoder(pcoder, *labels)
241+
single_shard_labels = [*labels[:-1], '-00000-of-00001']
242+
# Save the pcoder for the labels that imitates the sharded cache file name
243+
# suffix.
244+
self.save_pcoder(pcoder, *single_shard_labels)
245+
# Put a '-%05d-of-%05d' suffix to the cache file.
246+
sink = self.sink(single_shard_labels)._sink
247+
path = self._path(*labels[:-1])
248+
writer = sink.open_writer(path, labels[-1])
232249
for v in values:
233250
writer.write(v)
234251
writer.close()
@@ -253,7 +270,7 @@ def cleanup(self):
253270
self._saved_pcoders = {}
254271

255272
def _glob_path(self, *labels):
256-
return self._path(*labels) + '-*-of-*'
273+
return self._path(*labels) + '*-*-of-*'
257274

258275
def _path(self, *labels):
259276
return filesystems.FileSystems.join(self._cache_dir, *labels)

sdks/python/apache_beam/runners/interactive/cache_manager_test.py

Lines changed: 4 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -43,36 +43,20 @@ class FileBasedCacheManagerTest(object):
4343
cache_format = None # type: str
4444

4545
def setUp(self):
46-
self.test_dir = tempfile.mkdtemp()
4746
self.cache_manager = cache.FileBasedCacheManager(
48-
self.test_dir, cache_format=self.cache_format)
47+
cache_format=self.cache_format)
4948

5049
def tearDown(self):
51-
# The test_dir might have already been removed by cache_manager.cleanup().
52-
if os.path.exists(self.test_dir):
53-
shutil.rmtree(self.test_dir)
50+
self.cache_manager.cleanup()
5451

55-
def mock_write_cache(self, pcoll_list, prefix, cache_label):
52+
def mock_write_cache(self, values, prefix, cache_label):
5653
"""Cache the PCollection where cache.WriteCache would write to."""
57-
cache_path = filesystems.FileSystems.join(
58-
self.cache_manager._cache_dir, prefix)
59-
if not filesystems.FileSystems.exists(cache_path):
60-
filesystems.FileSystems.mkdirs(cache_path)
61-
6254
# Pause for 0.1 sec, because the Jenkins test runs so fast that the file
6355
# writes happen at the same timestamp.
6456
time.sleep(0.1)
6557

66-
cache_file = cache_label + '-1-of-2'
6758
labels = [prefix, cache_label]
68-
69-
# Usually, the pcoder will be inferred from `pcoll.element_type`
70-
pcoder = coders.registry.get_coder(object)
71-
# Save a pcoder for reading.
72-
self.cache_manager.save_pcoder(pcoder, *labels)
73-
# Save a pcoder for the fake write to the file.
74-
self.cache_manager.save_pcoder(pcoder, prefix, cache_file)
75-
self.cache_manager.write(pcoll_list, prefix, cache_file)
59+
self.cache_manager.write(values, *labels)
7660

7761
def test_exists(self):
7862
"""Test that CacheManager can correctly tell if the cache exists or not."""

sdks/python/apache_beam/runners/interactive/caching/streaming_cache.py

Lines changed: 10 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -251,7 +251,7 @@ def __init__(
251251
self._cache_dir = cache_dir
252252
else:
253253
self._cache_dir = tempfile.mkdtemp(
254-
prefix='interactive-temp-', dir=os.environ.get('TEST_TMPDIR', None))
254+
prefix='ib-', dir=os.environ.get('TEST_TMPDIR', None))
255255

256256
# List of saved pcoders keyed by PCollection path. It is OK to keep this
257257
# list in memory because once FileBasedCacheManager object is
@@ -343,8 +343,9 @@ def write(self, values, *labels):
343343
if isinstance(v, (TestStreamFileHeader, TestStreamFileRecord)):
344344
val = v.SerializeToString()
345345
else:
346-
val = v
347-
self.save_pcoder(coders.registry.get_coder(type(val)), *labels)
346+
raise TypeError(
347+
'Values given to streaming cache should be either '
348+
'TestStreamFileHeader or TestStreamFileRecord.')
348349
f.write(self.load_pcoder(*labels).encode(val) + b'\n')
349350

350351
def clear(self, *labels):
@@ -389,7 +390,12 @@ def save_pcoder(self, pcoder, *labels):
389390
def load_pcoder(self, *labels):
390391
saved_pcoder = self._saved_pcoders.get(
391392
os.path.join(self._cache_dir, *labels), None)
392-
return self._default_pcoder if saved_pcoder is None else saved_pcoder
393+
# TODO(BEAM-12506): Get rid of the SafeFastPrimitivesCoder for
394+
# WindowedValueHolder.
395+
if saved_pcoder is None or isinstance(saved_pcoder,
396+
coders.FastPrimitivesCoder):
397+
return self._default_pcoder
398+
return saved_pcoder
393399

394400
def cleanup(self):
395401

sdks/python/apache_beam/runners/interactive/caching/streaming_cache_test.py

Lines changed: 2 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -26,9 +26,7 @@
2626
from apache_beam.portability.api.beam_interactive_api_pb2 import TestStreamFileHeader
2727
from apache_beam.portability.api.beam_interactive_api_pb2 import TestStreamFileRecord
2828
from apache_beam.portability.api.beam_runner_api_pb2 import TestStreamPayload
29-
from apache_beam.runners.interactive.cache_manager import FileBasedCacheManager
3029
from apache_beam.runners.interactive.cache_manager import SafeFastPrimitivesCoder
31-
from apache_beam.runners.interactive.cache_manager import WriteCache
3230
from apache_beam.runners.interactive.caching.cacheable import CacheKey
3331
from apache_beam.runners.interactive.caching.streaming_cache import StreamingCache
3432
from apache_beam.runners.interactive.testing.test_cache_manager import FileRecordsBuilder
@@ -446,27 +444,9 @@ def test_always_default_coder_for_test_stream_records(self):
446444
self.assertIs(
447445
type(cache.load_pcoder(CACHED_NUMBERS)), type(cache._default_pcoder))
448446

449-
def test_load_saved_coder_for_non_record_types(self):
447+
def test_streaming_cache_does_not_write_non_record_or_header_types(self):
450448
cache = StreamingCache(cache_dir=None)
451-
cache.write('some value', 'a key')
452-
self.assertIs(
453-
type(cache.load_pcoder('a key')), type(coders.registry.get_coder(str)))
454-
455-
def test_wrap_file_based_cache_respect_load_saved_coders(self):
456-
file_based_cache = FileBasedCacheManager(tempfile.mkdtemp())
457-
pipeline = beam.Pipeline()
458-
pcoll = pipeline | beam.Create([1, 2, 3])
459-
_ = pcoll | WriteCache(file_based_cache, 'a key')
460-
self.assertIs(
461-
type(file_based_cache.load_pcoder('full', 'a key')),
462-
type(coders.registry.get_coder(int)))
463-
streaming_cache = StreamingCache(
464-
cache_dir=file_based_cache._cache_dir,
465-
saved_pcoders=file_based_cache._saved_pcoders)
466-
streaming_cache.write('some value', 'another key')
467-
self.assertIs(
468-
type(streaming_cache.load_pcoder('full', 'a key')),
469-
type(coders.registry.get_coder(int)))
449+
self.assertRaises(TypeError, cache.write, 'some value', 'a key')
470450

471451

472452
if __name__ == '__main__':

sdks/python/apache_beam/runners/interactive/pipeline_instrument_test.py

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -342,7 +342,7 @@ def test_instrument_example_unbounded_pipeline_to_read_cache(self):
342342
if not isinstance(pcoll, beam.pvalue.PCollection):
343343
continue
344344
cache_key = self.cache_key_of(name, pcoll)
345-
self._mock_write_cache(p_original, [b''], cache_key)
345+
self._mock_write_cache(p_original, [], cache_key)
346346

347347
# Instrument the original pipeline to create the pipeline the user will see.
348348
instrumenter = instr.build_pipeline_instrument(p_original)
@@ -498,7 +498,7 @@ def test_instrument_mixed_streaming_batch(self):
498498
ib.watch(locals())
499499

500500
self._mock_write_cache(
501-
p_original, [b''], self.cache_key_of('source_2', source_2))
501+
p_original, [], self.cache_key_of('source_2', source_2))
502502
ie.current_env().mark_pcollection_computed([source_2])
503503

504504
# Instrument the original pipeline to create the pipeline the user will see.
@@ -710,7 +710,7 @@ def test_instrument_example_unbounded_pipeline_to_multiple_read_cache(self):
710710
if not isinstance(pcoll, beam.pvalue.PCollection):
711711
continue
712712
cache_key = self.cache_key_of(name, pcoll)
713-
self._mock_write_cache(p_original, [b''], cache_key)
713+
self._mock_write_cache(p_original, [], cache_key)
714714

715715
# Instrument the original pipeline to create the pipeline the user will see.
716716
instrumenter = instr.build_pipeline_instrument(p_original)

sdks/python/setup.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -218,7 +218,7 @@ def get_version():
218218
# headless chrome based integration tests
219219
'selenium>=3.141.0,<4',
220220
'needle>=0.5.0,<1',
221-
'chromedriver-binary>=88,<89',
221+
'chromedriver-binary>=91,<92',
222222
# use a fixed major version of PIL for different python versions
223223
'pillow>=7.1.1,<8',
224224
]

0 commit comments

Comments
 (0)