def test_file_sink_multi_shards(self): temp_path = os.path.join(self._new_tempdir(), 'multishard') sink = MyFileBasedSink(temp_path, file_name_suffix='.output', coder=coders.ToBytesCoder()) # Manually invoke the generic Sink API. init_token = sink.initialize_write() num_shards = 1000 writer_results = [] for i in range(num_shards): uuid = 'uuid-%05d' % i writer = sink.open_writer(init_token, uuid) writer.write('a') writer.write('b') writer.write(uuid) writer_results.append(writer.close()) pre_finalize_results = sink.pre_finalize(init_token, writer_results) res = sorted( sink.finalize_write(init_token, writer_results, pre_finalize_results)) for i in range(num_shards): shard_name = '%s-%05d-of-%05d.output' % (temp_path, i, num_shards) uuid = 'uuid-%05d' % i self.assertEqual(res[i], shard_name) self.assertEqual( open(shard_name).read(), ('[start][a][b][%s][end]' % uuid)) # Check that any temp files are deleted. self.assertCountEqual(res, glob.glob(temp_path + '*'))
def test_file_sink_dst_matches_src(self): temp_path = os.path.join(self._new_tempdir(), 'dst_matches_src') sink = MyFileBasedSink(temp_path, file_name_suffix='.output', coder=coders.ToBytesCoder()) init_token, [res1, res2] = self._common_init(sink) pre_finalize_results = sink.pre_finalize(init_token, [res1, res2]) list( sink.finalize_write(init_token, [res1, res2], pre_finalize_results)) self.assertFalse(os.path.exists(res1)) self.assertFalse(os.path.exists(res2)) shard1 = temp_path + '-00000-of-00002.output' shard2 = temp_path + '-00001-of-00002.output' self.assertEqual(open(shard1).read(), '[start][a][b][end]') self.assertEqual(open(shard2).read(), '[start][x][y][z][end]') os.makedirs(os.path.dirname(res1)) shutil.copyfile(shard1, res1) shutil.copyfile(shard2, res2) list( sink.finalize_write(init_token, [res1, res2], pre_finalize_results))
def test_temp_dir_uniqueness(self): temp_path = os.path.join(self._new_tempdir(), 'unique') sink = MyFileBasedSink(temp_path, coder=coders.ToBytesCoder()) init_list = [''] * 1000 temp_dir_list = [sink._create_temp_dir(temp_path) for _ in init_list] temp_dir_set = set(temp_dir_list) self.assertEqual(len(temp_dir_list), len(temp_dir_set))
def test_file_sink_writing(self): temp_path = os.path.join(self._new_tempdir(), 'FileBasedSink') sink = MyFileBasedSink(temp_path, file_name_suffix='.output', coder=coders.ToBytesCoder()) init_token, writer_results = self._common_init(sink) pre_finalize_results = sink.pre_finalize(init_token, writer_results) finalize_res1 = list( sink.finalize_write(init_token, writer_results, pre_finalize_results)) # Retry the finalize operation (as if the first attempt was lost). finalize_res2 = list( sink.finalize_write(init_token, writer_results, pre_finalize_results)) # Check the results. shard1 = temp_path + '-00000-of-00002.output' shard2 = temp_path + '-00001-of-00002.output' self.assertEqual(finalize_res1, [shard1, shard2]) self.assertEqual(finalize_res2, []) self.assertEqual(open(shard1).read(), '[start][a][b][end]') self.assertEqual(open(shard2).read(), '[start][x][y][z][end]') # Check that any temp files are deleted. self.assertCountEqual([shard1, shard2], glob.glob(temp_path + '*'))
def test_pre_finalize(self): temp_path = os.path.join(self._new_tempdir(), 'pre_finalize') sink = MyFileBasedSink(temp_path, file_name_suffix='.output', coder=coders.ToBytesCoder()) init_token, [res1, res2] = self._common_init(sink) # no-op sink.pre_finalize(init_token, [res1, res2]) # Create finalized outputs from a previous run, which pre_finalize should # delete. shard1 = temp_path + '-00000-of-00002.output' shard2 = temp_path + '-00001-of-00002.output' with open(shard1, 'w') as f: f.write('foo') with open(shard2, 'w') as f: f.write('foo') self.assertTrue(os.path.exists(res1)) self.assertTrue(os.path.exists(res2)) self.assertTrue(os.path.exists(shard1)) self.assertTrue(os.path.exists(shard2)) sink.pre_finalize(init_token, [res1, res2]) self.assertTrue(os.path.exists(res1)) self.assertTrue(os.path.exists(res2)) self.assertFalse(os.path.exists(shard1)) self.assertFalse(os.path.exists(shard2))
def open(self, temp_path): file_handle = super(_TextSink, self).open(temp_path) if self._header is not None: file_handle.write(coders.ToBytesCoder().encode(self._header)) if self._append_trailing_newlines: file_handle.write(b'\n') return file_handle
def __init__( self, file_path_prefix, file_name_suffix='', append_trailing_newlines=True, num_shards=0, shard_name_template=None, coder=coders.ToBytesCoder(), # type: coders.Coder compression_type=CompressionTypes.AUTO, header=None, footer=None): """Initialize a _TextSink. Args: file_path_prefix: The file path to write to. The files written will begin with this prefix, followed by a shard identifier (see num_shards), and end in a common extension, if given by file_name_suffix. In most cases, only this argument is specified and num_shards, shard_name_template, and file_name_suffix use default values. file_name_suffix: Suffix for the files written. append_trailing_newlines: indicate whether this sink should write an additional newline char after writing each element. num_shards: The number of files (shards) used for output. If not set, the service will decide on the optimal number of shards. Constraining the number of shards is likely to reduce the performance of a pipeline. Setting this value is not recommended unless you require a specific number of output files. shard_name_template: A template string containing placeholders for the shard number and shard count. When constructing a filename for a particular shard number, the upper-case letters 'S' and 'N' are replaced with the 0-padded shard number and shard count respectively. This argument can be '' in which case it behaves as if num_shards was set to 1 and only one file will be generated. The default pattern used is '-SSSSS-of-NNNNN' if None is passed as the shard_name_template. coder: Coder used to encode each line. compression_type: Used to handle compressed output files. Typical value is CompressionTypes.AUTO, in which case the final file path's extension (as determined by file_path_prefix, file_name_suffix, num_shards and shard_name_template) will be used to detect the compression. header: String to write at beginning of file as a header. If not None and append_trailing_newlines is set, '\n' will be added. footer: String to write at the end of file as a footer. If not None and append_trailing_newlines is set, '\n' will be added. Returns: A _TextSink object usable for writing. """ super(_TextSink, self).__init__(file_path_prefix, file_name_suffix=file_name_suffix, num_shards=num_shards, shard_name_template=shard_name_template, coder=coder, mime_type='text/plain', compression_type=compression_type) self._append_trailing_newlines = append_trailing_newlines self._header = header self._footer = footer
def test_empty_write(self): temp_path = tempfile.NamedTemporaryFile().name sink = MyFileBasedSink(temp_path, file_name_suffix='.output', coder=coders.ToBytesCoder()) with TestPipeline() as p: p | beam.Create([]) | beam.io.Write(sink) # pylint: disable=expression-not-assigned self.assertEqual( open(temp_path + '-00000-of-00001.output').read(), '[start][end]')
def __init__( self, file_path_prefix, # type: str file_name_suffix='', append_trailing_newlines=True, num_shards=0, shard_name_template=None, # type: Optional[str] coder=coders.ToBytesCoder(), # type: coders.Coder compression_type=CompressionTypes.AUTO, header=None, footer=None, skip_if_empty=False): r"""Initialize a :class:`WriteToText` transform. Args: file_path_prefix (str): The file path to write to. The files written will begin with this prefix, followed by a shard identifier (see **num_shards**), and end in a common extension, if given by **file_name_suffix**. In most cases, only this argument is specified and **num_shards**, **shard_name_template**, and **file_name_suffix** use default values. file_name_suffix (str): Suffix for the files written. append_trailing_newlines (bool): indicate whether this sink should write an additional newline char after writing each element. num_shards (int): The number of files (shards) used for output. If not set, the service will decide on the optimal number of shards. Constraining the number of shards is likely to reduce the performance of a pipeline. Setting this value is not recommended unless you require a specific number of output files. shard_name_template (str): A template string containing placeholders for the shard number and shard count. Currently only ``''`` and ``'-SSSSS-of-NNNNN'`` are patterns accepted by the service. When constructing a filename for a particular shard number, the upper-case letters ``S`` and ``N`` are replaced with the ``0``-padded shard number and shard count respectively. This argument can be ``''`` in which case it behaves as if num_shards was set to 1 and only one file will be generated. The default pattern used is ``'-SSSSS-of-NNNNN'``. coder (~apache_beam.coders.coders.Coder): Coder used to encode each line. compression_type (str): Used to handle compressed output files. Typical value is :class:`CompressionTypes.AUTO <apache_beam.io.filesystem.CompressionTypes.AUTO>`, in which case the final file path's extension (as determined by **file_path_prefix**, **file_name_suffix**, **num_shards** and **shard_name_template**) will be used to detect the compression. header (str): String to write at beginning of file as a header. If not :data:`None` and **append_trailing_newlines** is set, ``\n`` will be added. footer (str): String to write at the end of file as a footer. If not :data:`None` and **append_trailing_newlines** is set, ``\n`` will be added. skip_if_empty: Don't write any shards if the PCollection is empty. """ self._sink = _TextSink(file_path_prefix, file_name_suffix, append_trailing_newlines, num_shards, shard_name_template, coder, compression_type, header, footer, skip_if_empty)
def test_static_value_provider_empty_write(self): temp_path = StaticValueProvider( value_type=str, value=tempfile.NamedTemporaryFile().name) sink = MyFileBasedSink(temp_path, file_name_suffix=StaticValueProvider( value_type=str, value='.output'), coder=coders.ToBytesCoder()) with TestPipeline() as p: p | beam.Create([]) | beam.io.Write(sink) # pylint: disable=expression-not-assigned self.assertEqual( open(temp_path.get() + '-00000-of-00001.output').read(), '[start][end]')
def test_file_sink_src_missing(self): temp_path = os.path.join(self._new_tempdir(), 'src_missing') sink = MyFileBasedSink(temp_path, file_name_suffix='.output', coder=coders.ToBytesCoder()) init_token, writer_results = self._common_init(sink) pre_finalize_results = sink.pre_finalize(init_token, writer_results) os.remove(writer_results[0]) with self.assertRaisesRegex(Exception, r'not exist'): list( sink.finalize_write(init_token, writer_results, pre_finalize_results))
def test_file_sink_display_data(self): temp_path = os.path.join(self._new_tempdir(), 'display') sink = MyFileBasedSink(temp_path, file_name_suffix='.output', coder=coders.ToBytesCoder()) dd = DisplayData.create_from(sink) expected_items = [ DisplayDataItemMatcher('compression', 'auto'), DisplayDataItemMatcher( 'file_pattern', '{}{}'.format(temp_path, '-%(shard_num)05d-of-%(num_shards)05d.output')) ] hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
def test_file_sink_rename_error(self, rename_mock): temp_path = os.path.join(self._new_tempdir(), 'rename_error') sink = MyFileBasedSink(temp_path, file_name_suffix='.output', coder=coders.ToBytesCoder()) init_token, writer_results = self._common_init(sink) pre_finalize_results = sink.pre_finalize(init_token, writer_results) error_str = 'mock rename error description' rename_mock.side_effect = BeamIOError('mock rename error', {('src', 'dst'): error_str}) with self.assertRaisesRegex(Exception, error_str): list( sink.finalize_write(init_token, writer_results, pre_finalize_results))
def test_fixed_shard_write(self): temp_path = os.path.join(self._new_tempdir(), 'empty') sink = MyFileBasedSink(temp_path, file_name_suffix='.output', num_shards=3, shard_name_template='_NN_SSS_', coder=coders.ToBytesCoder()) with TestPipeline() as p: p | beam.Create(['a', 'b']) | beam.io.Write(sink) # pylint: disable=expression-not-assigned concat = ''.join( open(temp_path + '_03_%03d_.output' % shard_num).read() for shard_num in range(3)) self.assertTrue('][a][' in concat, concat) self.assertTrue('][b][' in concat, concat)
def test_pre_finalize_error(self, delete_mock): temp_path = os.path.join(self._new_tempdir(), 'pre_finalize') sink = MyFileBasedSink(temp_path, file_name_suffix='.output', coder=coders.ToBytesCoder()) init_token, [res1, res2] = self._common_init(sink) # no-op sink.pre_finalize(init_token, [res1, res2]) # Create finalized outputs from a previous run, which pre_finalize should # delete. shard1 = temp_path + '-00000-of-00002.output' shard2 = temp_path + '-00001-of-00002.output' with open(shard1, 'w') as f: f.write('foo') with open(shard2, 'w') as f: f.write('foo') error_str = 'mock rename error description' delete_mock.side_effect = BeamIOError('mock rename error', {shard2: error_str}) with self.assertRaisesRegex(Exception, error_str): sink.pre_finalize(init_token, [res1, res2])
def close(self, file_handle): if self._footer is not None: file_handle.write(coders.ToBytesCoder().encode(self._footer)) if self._append_trailing_newlines: file_handle.write(b'\n') super().close(file_handle)
def _get_temp_dir(file_path_prefix): sink = MyFileBasedSink(file_path_prefix, file_name_suffix='.output', coder=coders.ToBytesCoder()) return sink.initialize_write()