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.ToStringCoder()) 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_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.ToStringCoder() ) 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]')
class WriteToFiles(beam.PTransform): """Write the incoming PCollection to a set of output files. The incoming ``PCollection`` may be bounded or unbounded. **Note:** For unbounded ``PCollection``s, this transform does not support multiple firings per Window (due to the fact that files are named only by their destination, and window, at the moment). """ # We allow up to 20 different destinations to be written in a single bundle. # Too many files will add memory pressure to the worker, so we let it be 20. MAX_NUM_WRITERS_PER_BUNDLE = 20 DEFAULT_SHARDING = 5 def __init__( self, path, file_naming=None, destination=None, temp_directory=None, sink=None, shards=None, output_fn=None, max_writers_per_bundle=MAX_NUM_WRITERS_PER_BUNDLE): """Initializes a WriteToFiles transform. Args: path (str, ValueProvider): The directory to write files into. file_naming (callable): A callable that takes in a window, pane, shard_index, total_shards and compression; and returns a file name. destination (callable): If this argument is provided, the sink parameter must also be a callable. temp_directory (str, ValueProvider): To ensure atomicity in the transform, the output is written into temporary files, which are written to a directory that is meant to be temporary as well. Once the whole output has been written, the files are moved into their final destination, and given their final names. By default, the temporary directory will be within the temp_location of your pipeline. sink (callable, FileSink): The sink to use to write into a file. It should implement the methods of a ``FileSink``. If none is provided, a ``TextSink`` is used. shards (int): The number of shards per destination and trigger firing. max_writers_per_bundle (int): The number of writers that can be open concurrently in a single worker that's processing one bundle. """ self.path = ( path if isinstance(path, ValueProvider) else StaticValueProvider( str, path)) self.file_naming_fn = file_naming or default_file_naming('output') self.destination_fn = self._get_destination_fn(destination) self._temp_directory = temp_directory self.sink_fn = self._get_sink_fn(sink) self.shards = shards or WriteToFiles.DEFAULT_SHARDING self.output_fn = output_fn or (lambda x: x) self._max_num_writers_per_bundle = max_writers_per_bundle @staticmethod def _get_sink_fn(input_sink): # type: (...) -> Callable[[Any], FileSink] if isinstance(input_sink, FileSink): return lambda x: input_sink elif callable(input_sink): return input_sink else: return lambda x: TextSink() @staticmethod def _get_destination_fn(destination): # type: (...) -> Callable[[Any], str] if isinstance(destination, ValueProvider): return lambda elm: destination.get() elif callable(destination): return destination else: return lambda elm: destination def expand(self, pcoll): p = pcoll.pipeline if not self._temp_directory: temp_location = ( p.options.view_as(GoogleCloudOptions).temp_location or self.path.get()) dir_uid = str(uuid.uuid4()) self._temp_directory = StaticValueProvider( str, filesystems.FileSystems.join(temp_location, '.temp%s' % dir_uid)) _LOGGER.info('Added temporary directory %s', self._temp_directory.get()) output = ( pcoll | beam.ParDo( _WriteUnshardedRecordsFn( base_path=self._temp_directory, destination_fn=self.destination_fn, sink_fn=self.sink_fn, max_writers_per_bundle=self._max_num_writers_per_bundle)). with_outputs( _WriteUnshardedRecordsFn.SPILLED_RECORDS, _WriteUnshardedRecordsFn.WRITTEN_FILES)) written_files_pc = output[_WriteUnshardedRecordsFn.WRITTEN_FILES] spilled_records_pc = output[_WriteUnshardedRecordsFn.SPILLED_RECORDS] more_written_files_pc = ( spilled_records_pc | beam.ParDo( _AppendShardedDestination(self.destination_fn, self.shards)) | "GroupRecordsByDestinationAndShard" >> beam.GroupByKey() | beam.ParDo( _WriteShardedRecordsFn( self._temp_directory, self.sink_fn, self.shards))) files_by_destination_pc = ( (written_files_pc, more_written_files_pc) | beam.Flatten() | beam.Map(lambda file_result: (file_result.destination, file_result)) | "GroupTempFilesByDestination" >> beam.GroupByKey()) # Now we should take the temporary files, and write them to the final # destination, with their proper names. file_results = ( files_by_destination_pc | beam.ParDo( _MoveTempFilesIntoFinalDestinationFn( self.path, self.file_naming_fn, self._temp_directory))) return file_results