github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/io/avroio.py (about) 1 # 2 # Licensed to the Apache Software Foundation (ASF) under one or more 3 # contributor license agreements. See the NOTICE file distributed with 4 # this work for additional information regarding copyright ownership. 5 # The ASF licenses this file to You under the Apache License, Version 2.0 6 # (the "License"); you may not use this file except in compliance with 7 # the License. You may obtain a copy of the License at 8 # 9 # http://www.apache.org/licenses/LICENSE-2.0 10 # 11 # Unless required by applicable law or agreed to in writing, software 12 # distributed under the License is distributed on an "AS IS" BASIS, 13 # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 # See the License for the specific language governing permissions and 15 # limitations under the License. 16 # 17 18 """``PTransforms`` for reading from and writing to Avro files. 19 20 Provides two read ``PTransform``s, ``ReadFromAvro`` and ``ReadAllFromAvro``, 21 that produces a ``PCollection`` of records. 22 Each record of this ``PCollection`` will contain a single record read from 23 an Avro file. Records that are of simple types will be mapped into 24 corresponding Python types. Records that are of Avro type 'RECORD' will be 25 mapped to Python dictionaries that comply with the schema contained in the 26 Avro file that contains those records. In this case, keys of each dictionary 27 will contain the corresponding field names and will be of type ``string`` 28 while the values of the dictionary will be of the type defined in the 29 corresponding Avro schema. 30 31 For example, if schema of the Avro file is the following. 32 {"namespace": "example.avro","type": "record","name": "User","fields": 33 [{"name": "name", "type": "string"}, 34 {"name": "favorite_number", "type": ["int", "null"]}, 35 {"name": "favorite_color", "type": ["string", "null"]}]} 36 37 Then records generated by read transforms will be dictionaries of the 38 following form. 39 {'name': 'Alyssa', 'favorite_number': 256, 'favorite_color': None}). 40 41 Additionally, this module provides a write ``PTransform`` ``WriteToAvro`` 42 that can be used to write a given ``PCollection`` of Python objects to an 43 Avro file. 44 """ 45 # pytype: skip-file 46 import os 47 from functools import partial 48 49 from fastavro.read import block_reader 50 from fastavro.write import Writer 51 52 import apache_beam as beam 53 from apache_beam.io import filebasedsink 54 from apache_beam.io import filebasedsource 55 from apache_beam.io import iobase 56 from apache_beam.io.filesystem import CompressionTypes 57 from apache_beam.io.iobase import Read 58 from apache_beam.transforms import PTransform 59 60 __all__ = [ 61 'ReadFromAvro', 62 'ReadAllFromAvro', 63 'ReadAllFromAvroContinuously', 64 'WriteToAvro' 65 ] 66 67 68 class ReadFromAvro(PTransform): 69 """A :class:`~apache_beam.transforms.ptransform.PTransform` for reading avro 70 files.""" 71 def __init__( 72 self, 73 file_pattern=None, 74 min_bundle_size=0, 75 validate=True, 76 use_fastavro=True): 77 """Initializes :class:`ReadFromAvro`. 78 79 Uses source :class:`~apache_beam.io._AvroSource` to read a set of Avro 80 files defined by a given file pattern. 81 82 If ``/mypath/myavrofiles*`` is a file-pattern that points to a set of Avro 83 files, a :class:`~apache_beam.pvalue.PCollection` for the records in 84 these Avro files can be created in the following manner. 85 86 .. testcode:: 87 88 with beam.Pipeline() as p: 89 records = p | 'Read' >> beam.io.ReadFromAvro('/mypath/myavrofiles*') 90 91 .. NOTE: We're not actually interested in this error; but if we get here, 92 it means that the way of calling this transform hasn't changed. 93 94 .. testoutput:: 95 :hide: 96 97 Traceback (most recent call last): 98 ... 99 OSError: No files found based on the file pattern 100 101 Each record of this :class:`~apache_beam.pvalue.PCollection` will contain 102 a single record read from a source. Records that are of simple types will be 103 mapped into corresponding Python types. Records that are of Avro type 104 ``RECORD`` will be mapped to Python dictionaries that comply with the schema 105 contained in the Avro file that contains those records. In this case, keys 106 of each dictionary will contain the corresponding field names and will be of 107 type :class:`str` while the values of the dictionary will be of the type 108 defined in the corresponding Avro schema. 109 110 For example, if schema of the Avro file is the following. :: 111 112 { 113 "namespace": "example.avro", 114 "type": "record", 115 "name": "User", 116 "fields": [ 117 118 {"name": "name", 119 "type": "string"}, 120 121 {"name": "favorite_number", 122 "type": ["int", "null"]}, 123 124 {"name": "favorite_color", 125 "type": ["string", "null"]} 126 127 ] 128 } 129 130 Then records generated by :class:`~apache_beam.io._AvroSource` will be 131 dictionaries of the following form. :: 132 133 {'name': 'Alyssa', 'favorite_number': 256, 'favorite_color': None}). 134 135 Args: 136 file_pattern (str): the file glob to read 137 min_bundle_size (int): the minimum size in bytes, to be considered when 138 splitting the input into bundles. 139 validate (bool): flag to verify that the files exist during the pipeline 140 creation time. 141 use_fastavro (bool): This flag is left for API backwards compatibility 142 and no longer has an effect. Do not use. 143 """ 144 super().__init__() 145 self._source = _create_avro_source( 146 file_pattern, min_bundle_size, validate=validate) 147 148 def expand(self, pvalue): 149 return pvalue.pipeline | Read(self._source) 150 151 def display_data(self): 152 return {'source_dd': self._source} 153 154 155 class ReadAllFromAvro(PTransform): 156 """A ``PTransform`` for reading ``PCollection`` of Avro files. 157 158 Uses source '_AvroSource' to read a ``PCollection`` of Avro files or file 159 patterns and produce a ``PCollection`` of Avro records. 160 161 This implementation is only tested with batch pipeline. In streaming, 162 reading may happen with delay due to the limitation in ReShuffle involved. 163 """ 164 165 DEFAULT_DESIRED_BUNDLE_SIZE = 64 * 1024 * 1024 # 64MB 166 167 def __init__( 168 self, 169 min_bundle_size=0, 170 desired_bundle_size=DEFAULT_DESIRED_BUNDLE_SIZE, 171 use_fastavro=True, 172 with_filename=False, 173 label='ReadAllFiles'): 174 """Initializes ``ReadAllFromAvro``. 175 176 Args: 177 min_bundle_size: the minimum size in bytes, to be considered when 178 splitting the input into bundles. 179 desired_bundle_size: the desired size in bytes, to be considered when 180 splitting the input into bundles. 181 use_fastavro (bool): This flag is left for API backwards compatibility 182 and no longer has an effect. Do not use. 183 with_filename: If True, returns a Key Value with the key being the file 184 name and the value being the actual data. If False, it only returns 185 the data. 186 """ 187 source_from_file = partial( 188 _create_avro_source, min_bundle_size=min_bundle_size) 189 self._read_all_files = filebasedsource.ReadAllFiles( 190 True, 191 CompressionTypes.AUTO, 192 desired_bundle_size, 193 min_bundle_size, 194 source_from_file, 195 with_filename) 196 197 self.label = label 198 199 def expand(self, pvalue): 200 return pvalue | self.label >> self._read_all_files 201 202 203 class ReadAllFromAvroContinuously(ReadAllFromAvro): 204 """A ``PTransform`` for reading avro files in given file patterns. 205 This PTransform acts as a Source and produces continuously a ``PCollection`` 206 of Avro records. 207 208 For more details, see ``ReadAllFromAvro`` for avro parsing settings; 209 see ``apache_beam.io.fileio.MatchContinuously`` for watching settings. 210 211 ReadAllFromAvroContinuously is experimental. No backwards-compatibility 212 guarantees. Due to the limitation on Reshuffle, current implementation does 213 not scale. 214 """ 215 _ARGS_FOR_MATCH = ( 216 'interval', 217 'has_deduplication', 218 'start_timestamp', 219 'stop_timestamp', 220 'match_updated_files', 221 'apply_windowing') 222 _ARGS_FOR_READ = ( 223 'min_bundle_size', 'desired_bundle_size', 'use_fastavro', 'with_filename') 224 225 def __init__(self, file_pattern, label='ReadAllFilesContinuously', **kwargs): 226 """Initialize the ``ReadAllFromAvroContinuously`` transform. 227 228 Accepts args for constructor args of both :class:`ReadAllFromAvro` and 229 :class:`~apache_beam.io.fileio.MatchContinuously`. 230 """ 231 kwargs_for_match = { 232 k: v 233 for (k, v) in kwargs.items() if k in self._ARGS_FOR_MATCH 234 } 235 kwargs_for_read = { 236 k: v 237 for (k, v) in kwargs.items() if k in self._ARGS_FOR_READ 238 } 239 kwargs_additinal = { 240 k: v 241 for (k, v) in kwargs.items() 242 if k not in self._ARGS_FOR_MATCH and k not in self._ARGS_FOR_READ 243 } 244 super().__init__(label=label, **kwargs_for_read, **kwargs_additinal) 245 self._file_pattern = file_pattern 246 self._kwargs_for_match = kwargs_for_match 247 248 def expand(self, pbegin): 249 # Importing locally to prevent circular dependency issues. 250 from apache_beam.io.fileio import MatchContinuously 251 252 # TODO(BEAM-14497) always reshuffle once gbk always trigger works. 253 return ( 254 pbegin 255 | MatchContinuously(self._file_pattern, **self._kwargs_for_match) 256 | 'ReadAllFiles' >> self._read_all_files._disable_reshuffle()) 257 258 259 class _AvroUtils(object): 260 @staticmethod 261 def advance_file_past_next_sync_marker(f, sync_marker): 262 buf_size = 10000 263 264 data = f.read(buf_size) 265 while data: 266 pos = data.find(sync_marker) 267 if pos >= 0: 268 # Adjusting the current position to the ending position of the sync 269 # marker. 270 backtrack = len(data) - pos - len(sync_marker) 271 f.seek(-1 * backtrack, os.SEEK_CUR) 272 return True 273 else: 274 if f.tell() >= len(sync_marker): 275 # Backtracking in case we partially read the sync marker during the 276 # previous read. We only have to backtrack if there are at least 277 # len(sync_marker) bytes before current position. We only have to 278 # backtrack (len(sync_marker) - 1) bytes. 279 f.seek(-1 * (len(sync_marker) - 1), os.SEEK_CUR) 280 data = f.read(buf_size) 281 282 283 def _create_avro_source(file_pattern=None, min_bundle_size=0, validate=False): 284 return \ 285 _FastAvroSource( 286 file_pattern=file_pattern, 287 min_bundle_size=min_bundle_size, 288 validate=validate 289 ) 290 291 292 class _FastAvroSource(filebasedsource.FileBasedSource): 293 """A source for reading Avro files using the `fastavro` library. 294 295 ``_FastAvroSource`` is implemented using the file-based source framework 296 available in module 'filebasedsource'. Hence please refer to module 297 'filebasedsource' to fully understand how this source implements operations 298 common to all file-based sources such as file-pattern expansion and splitting 299 into bundles for parallel processing. 300 301 TODO: remove ``_AvroSource`` in favor of using ``_FastAvroSource`` 302 everywhere once it has been more widely tested 303 """ 304 def read_records(self, file_name, range_tracker): 305 next_block_start = -1 306 307 def split_points_unclaimed(stop_position): 308 if next_block_start >= stop_position: 309 # Next block starts at or after the suggested stop position. Hence 310 # there will not be split points to be claimed for the range ending at 311 # suggested stop position. 312 return 0 313 314 return iobase.RangeTracker.SPLIT_POINTS_UNKNOWN 315 316 range_tracker.set_split_points_unclaimed_callback(split_points_unclaimed) 317 318 start_offset = range_tracker.start_position() 319 if start_offset is None: 320 start_offset = 0 321 322 with self.open_file(file_name) as f: 323 blocks = block_reader(f) 324 sync_marker = blocks._header['sync'] 325 326 # We have to start at current position if previous bundle ended at the 327 # end of a sync marker. 328 start_offset = max(0, start_offset - len(sync_marker)) 329 f.seek(start_offset) 330 _AvroUtils.advance_file_past_next_sync_marker(f, sync_marker) 331 332 next_block_start = f.tell() 333 334 while range_tracker.try_claim(next_block_start): 335 block = next(blocks) 336 next_block_start = block.offset + block.size 337 for record in block: 338 yield record 339 340 341 class WriteToAvro(beam.transforms.PTransform): 342 """A ``PTransform`` for writing avro files.""" 343 def __init__( 344 self, 345 file_path_prefix, 346 schema, 347 codec='deflate', 348 file_name_suffix='', 349 num_shards=0, 350 shard_name_template=None, 351 mime_type='application/x-avro', 352 use_fastavro=True): 353 """Initialize a WriteToAvro transform. 354 355 Args: 356 file_path_prefix: The file path to write to. The files written will begin 357 with this prefix, followed by a shard identifier (see num_shards), and 358 end in a common extension, if given by file_name_suffix. In most cases, 359 only this argument is specified and num_shards, shard_name_template, and 360 file_name_suffix use default values. 361 schema: The schema to use (dict). 362 codec: The codec to use for block-level compression. Any string supported 363 by the Avro specification is accepted (for example 'null'). 364 file_name_suffix: Suffix for the files written. 365 num_shards: The number of files (shards) used for output. If not set, the 366 service will decide on the optimal number of shards. 367 Constraining the number of shards is likely to reduce 368 the performance of a pipeline. Setting this value is not recommended 369 unless you require a specific number of output files. 370 shard_name_template: A template string containing placeholders for 371 the shard number and shard count. When constructing a filename for a 372 particular shard number, the upper-case letters 'S' and 'N' are 373 replaced with the 0-padded shard number and shard count respectively. 374 This argument can be '' in which case it behaves as if num_shards was 375 set to 1 and only one file will be generated. The default pattern used 376 is '-SSSSS-of-NNNNN' if None is passed as the shard_name_template. 377 mime_type: The MIME type to use for the produced files, if the filesystem 378 supports specifying MIME types. 379 use_fastavro (bool): This flag is left for API backwards compatibility 380 and no longer has an effect. Do not use. 381 382 Returns: 383 A WriteToAvro transform usable for writing. 384 """ 385 self._sink = _create_avro_sink( 386 file_path_prefix, 387 schema, 388 codec, 389 file_name_suffix, 390 num_shards, 391 shard_name_template, 392 mime_type) 393 394 def expand(self, pcoll): 395 return pcoll | beam.io.iobase.Write(self._sink) 396 397 def display_data(self): 398 return {'sink_dd': self._sink} 399 400 401 def _create_avro_sink( 402 file_path_prefix, 403 schema, 404 codec, 405 file_name_suffix, 406 num_shards, 407 shard_name_template, 408 mime_type): 409 if "class \'avro.schema" in str(type(schema)): 410 raise ValueError( 411 'You are using Avro IO with fastavro (default with Beam on ' 412 'Python 3), but supplying a schema parsed by avro-python3. ' 413 'Please change the schema to a dict.') 414 return _FastAvroSink( 415 file_path_prefix, 416 schema, 417 codec, 418 file_name_suffix, 419 num_shards, 420 shard_name_template, 421 mime_type) 422 423 424 class _BaseAvroSink(filebasedsink.FileBasedSink): 425 """A base for a sink for avro files. """ 426 def __init__( 427 self, 428 file_path_prefix, 429 schema, 430 codec, 431 file_name_suffix, 432 num_shards, 433 shard_name_template, 434 mime_type): 435 super().__init__( 436 file_path_prefix, 437 file_name_suffix=file_name_suffix, 438 num_shards=num_shards, 439 shard_name_template=shard_name_template, 440 coder=None, 441 mime_type=mime_type, 442 # Compression happens at the block level using the supplied codec, and 443 # not at the file level. 444 compression_type=CompressionTypes.UNCOMPRESSED) 445 self._schema = schema 446 self._codec = codec 447 448 def display_data(self): 449 res = super().display_data() 450 res['codec'] = str(self._codec) 451 res['schema'] = str(self._schema) 452 return res 453 454 455 class _FastAvroSink(_BaseAvroSink): 456 """A sink for avro files using FastAvro. """ 457 def __init__( 458 self, 459 file_path_prefix, 460 schema, 461 codec, 462 file_name_suffix, 463 num_shards, 464 shard_name_template, 465 mime_type): 466 super().__init__( 467 file_path_prefix, 468 schema, 469 codec, 470 file_name_suffix, 471 num_shards, 472 shard_name_template, 473 mime_type) 474 self.file_handle = None 475 476 def open(self, temp_path): 477 self.file_handle = super().open(temp_path) 478 return Writer(self.file_handle, self._schema, self._codec) 479 480 def write_record(self, writer, value): 481 writer.write(value) 482 483 def close(self, writer): 484 writer.flush() 485 self.file_handle.close()