github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/io/textio.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 """A source and a sink for reading from and writing to text files.""" 19 20 # pytype: skip-file 21 22 import logging 23 from functools import partial 24 from typing import TYPE_CHECKING 25 from typing import Any 26 from typing import Optional 27 28 from apache_beam import typehints 29 from apache_beam.coders import coders 30 from apache_beam.io import filebasedsink 31 from apache_beam.io import filebasedsource 32 from apache_beam.io import iobase 33 from apache_beam.io.filebasedsource import ReadAllFiles 34 from apache_beam.io.filesystem import CompressionTypes 35 from apache_beam.io.iobase import Read 36 from apache_beam.io.iobase import Write 37 from apache_beam.transforms import PTransform 38 from apache_beam.transforms.display import DisplayDataItem 39 40 if TYPE_CHECKING: 41 from apache_beam.io import fileio 42 43 __all__ = [ 44 'ReadFromText', 45 'ReadFromTextWithFilename', 46 'ReadAllFromText', 47 'ReadAllFromTextContinuously', 48 'WriteToText', 49 'ReadFromCsv', 50 'WriteToCsv', 51 'ReadFromJson', 52 'WriteToJson', 53 ] 54 55 _LOGGER = logging.getLogger(__name__) 56 57 58 class _TextSource(filebasedsource.FileBasedSource): 59 r"""A source for reading text files. 60 61 Parses a text file as newline-delimited elements. Supports newline delimiters 62 '\n' and '\r\n. 63 64 This implementation reads encoded text and uses the input coder's encoding to 65 decode from bytes to str. This does not support ``UTF-16`` or ``UTF-32`` 66 encodings. 67 """ 68 69 DEFAULT_READ_BUFFER_SIZE = 8192 70 71 class ReadBuffer(object): 72 # A buffer that gives the buffered data and next position in the 73 # buffer that should be read. 74 75 def __init__(self, data, position): 76 self._data = data 77 self._position = position 78 79 @property 80 def data(self): 81 return self._data 82 83 @data.setter 84 def data(self, value): 85 assert isinstance(value, bytes) 86 self._data = value 87 88 @property 89 def position(self): 90 return self._position 91 92 @position.setter 93 def position(self, value): 94 assert isinstance(value, int) 95 if value > len(self._data): 96 raise ValueError( 97 'Cannot set position to %d since it\'s larger than ' 98 'size of data %d.' % (value, len(self._data))) 99 self._position = value 100 101 def reset(self): 102 self.data = b'' 103 self.position = 0 104 105 def __init__(self, 106 file_pattern, 107 min_bundle_size, 108 compression_type, 109 strip_trailing_newlines, 110 coder, # type: coders.Coder 111 buffer_size=DEFAULT_READ_BUFFER_SIZE, 112 validate=True, 113 skip_header_lines=0, 114 header_processor_fns=(None, None), 115 delimiter=None, 116 escapechar=None): 117 """Initialize a _TextSource 118 119 Args: 120 header_processor_fns (tuple): a tuple of a `header_matcher` function 121 and a `header_processor` function. The `header_matcher` should 122 return `True` for all lines at the start of the file that are part 123 of the file header and `False` otherwise. These header lines will 124 not be yielded when reading records and instead passed into 125 `header_processor` to be handled. If `skip_header_lines` and a 126 `header_matcher` are both provided, the value of `skip_header_lines` 127 lines will be skipped and the header will be processed from 128 there. 129 delimiter (bytes) Optional: delimiter to split records. 130 Must not self-overlap, because self-overlapping delimiters cause 131 ambiguous parsing. 132 escapechar (bytes) Optional: a single byte to escape the records 133 delimiter, can also escape itself. 134 Raises: 135 ValueError: if skip_lines is negative. 136 137 Please refer to documentation in class `ReadFromText` for the rest 138 of the arguments. 139 """ 140 super().__init__( 141 file_pattern, 142 min_bundle_size, 143 compression_type=compression_type, 144 validate=validate) 145 146 self._strip_trailing_newlines = strip_trailing_newlines 147 self._compression_type = compression_type 148 self._coder = coder 149 self._buffer_size = buffer_size 150 if skip_header_lines < 0: 151 raise ValueError( 152 'Cannot skip negative number of header lines: %d' % skip_header_lines) 153 elif skip_header_lines > 10: 154 _LOGGER.warning( 155 'Skipping %d header lines. Skipping large number of header ' 156 'lines might significantly slow down processing.') 157 self._skip_header_lines = skip_header_lines 158 self._header_matcher, self._header_processor = header_processor_fns 159 if delimiter is not None: 160 if not isinstance(delimiter, bytes) or len(delimiter) == 0: 161 raise ValueError('Delimiter must be a non-empty bytes sequence.') 162 if self._is_self_overlapping(delimiter): 163 raise ValueError('Delimiter must not self-overlap.') 164 self._delimiter = delimiter 165 if escapechar is not None: 166 if not (isinstance(escapechar, bytes) and len(escapechar) == 1): 167 raise ValueError( 168 "escapechar must be bytes of size 1: '%s'" % escapechar) 169 self._escapechar = escapechar 170 171 def display_data(self): 172 parent_dd = super().display_data() 173 parent_dd['strip_newline'] = DisplayDataItem( 174 self._strip_trailing_newlines, label='Strip Trailing New Lines') 175 parent_dd['buffer_size'] = DisplayDataItem( 176 self._buffer_size, label='Buffer Size') 177 parent_dd['coder'] = DisplayDataItem(self._coder.__class__, label='Coder') 178 return parent_dd 179 180 def read_records(self, file_name, range_tracker): 181 start_offset = range_tracker.start_position() 182 read_buffer = _TextSource.ReadBuffer(b'', 0) 183 184 next_record_start_position = -1 185 186 def split_points_unclaimed(stop_position): 187 return ( 188 0 if stop_position <= next_record_start_position else 189 iobase.RangeTracker.SPLIT_POINTS_UNKNOWN) 190 191 range_tracker.set_split_points_unclaimed_callback(split_points_unclaimed) 192 193 with self.open_file(file_name) as file_to_read: 194 position_after_processing_header_lines = ( 195 self._process_header(file_to_read, read_buffer)) 196 start_offset = max(start_offset, position_after_processing_header_lines) 197 if start_offset > position_after_processing_header_lines: 198 # Seeking to one delimiter length before the start index and ignoring 199 # the current line. If start_position is at beginning of the line, that 200 # line belongs to the current bundle, hence ignoring that is incorrect. 201 # Seeking to one delimiter before prevents that. 202 203 if self._delimiter is not None and start_offset >= len(self._delimiter): 204 required_position = start_offset - len(self._delimiter) 205 else: 206 required_position = start_offset - 1 207 208 if self._escapechar is not None: 209 # Need more bytes to check if the delimiter is escaped. 210 # Seek until the first escapechar if any. 211 while required_position > 0: 212 file_to_read.seek(required_position - 1) 213 if file_to_read.read(1) == self._escapechar: 214 required_position -= 1 215 else: 216 break 217 218 file_to_read.seek(required_position) 219 read_buffer.reset() 220 sep_bounds = self._find_separator_bounds(file_to_read, read_buffer) 221 if not sep_bounds: 222 # Could not find a delimiter after required_position. This means that 223 # none of the records within the file belongs to the current source. 224 return 225 226 _, sep_end = sep_bounds 227 read_buffer.data = read_buffer.data[sep_end:] 228 next_record_start_position = required_position + sep_end 229 else: 230 next_record_start_position = position_after_processing_header_lines 231 232 while range_tracker.try_claim(next_record_start_position): 233 record, num_bytes_to_next_record = self._read_record(file_to_read, 234 read_buffer) 235 # For compressed text files that use an unsplittable OffsetRangeTracker 236 # with infinity as the end position, above 'try_claim()' invocation 237 # would pass for an empty record at the end of file that is not 238 # followed by a new line character. Since such a record is at the last 239 # position of a file, it should not be a part of the considered range. 240 # We do this check to ignore such records. 241 if len(record) == 0 and num_bytes_to_next_record < 0: # pylint: disable=len-as-condition 242 break 243 244 # Record delimiter must be larger than zero bytes. 245 assert num_bytes_to_next_record != 0 246 if num_bytes_to_next_record > 0: 247 next_record_start_position += num_bytes_to_next_record 248 249 yield self._coder.decode(record) 250 if num_bytes_to_next_record < 0: 251 break 252 253 def _process_header(self, file_to_read, read_buffer): 254 # Returns a tuple containing the position in file after processing header 255 # records and a list of decoded header lines that match 256 # 'header_matcher'. 257 header_lines = [] 258 position = self._skip_lines( 259 file_to_read, read_buffer, 260 self._skip_header_lines) if self._skip_header_lines else 0 261 if self._header_matcher: 262 while True: 263 record, num_bytes_to_next_record = self._read_record(file_to_read, 264 read_buffer) 265 decoded_line = self._coder.decode(record) 266 if not self._header_matcher(decoded_line): 267 # We've read past the header section at this point, so go back a line. 268 file_to_read.seek(position) 269 read_buffer.reset() 270 break 271 header_lines.append(decoded_line) 272 if num_bytes_to_next_record < 0: 273 break 274 position += num_bytes_to_next_record 275 276 if self._header_processor: 277 self._header_processor(header_lines) 278 279 return position 280 281 def _find_separator_bounds(self, file_to_read, read_buffer): 282 # Determines the start and end positions within 'read_buffer.data' of the 283 # next delimiter starting from position 'read_buffer.position'. 284 # Use the custom delimiter to be used in place of 285 # the default ones ('\n' or '\r\n')' 286 # This method may increase the size of buffer but it will not decrease the 287 # size of it. 288 289 current_pos = read_buffer.position 290 291 # b'\n' use as default 292 delimiter = self._delimiter or b'\n' 293 delimiter_len = len(delimiter) 294 295 while True: 296 if current_pos >= len(read_buffer.data) - delimiter_len + 1: 297 # Ensuring that there are enough bytes to determine 298 # at current_pos. 299 if not self._try_to_ensure_num_bytes_in_buffer( 300 file_to_read, read_buffer, current_pos + delimiter_len): 301 return 302 303 # Using find() here is more efficient than a linear scan 304 # of the byte array. 305 next_delim = read_buffer.data.find(delimiter, current_pos) 306 307 if next_delim >= 0: 308 if (self._delimiter is None and 309 read_buffer.data[next_delim - 1:next_delim] == b'\r'): 310 if self._escapechar is not None and self._is_escaped(read_buffer, 311 next_delim - 1): 312 # Accept '\n' as a default delimiter, because '\r' is escaped. 313 return (next_delim, next_delim + 1) 314 else: 315 # Accept both '\r\n' and '\n' as a default delimiter. 316 return (next_delim - 1, next_delim + 1) 317 else: 318 if self._escapechar is not None and self._is_escaped(read_buffer, 319 next_delim): 320 # Skip an escaped delimiter. 321 current_pos = next_delim + delimiter_len + 1 322 continue 323 else: 324 # Found a delimiter. Accepting that as the next delimiter. 325 return (next_delim, next_delim + delimiter_len) 326 327 elif self._delimiter is not None: 328 # Corner case: custom delimiter is truncated at the end of the buffer. 329 next_delim = read_buffer.data.find( 330 delimiter[0], len(read_buffer.data) - delimiter_len + 1) 331 if next_delim >= 0: 332 # Delimiters longer than 1 byte may cross the buffer boundary. 333 # Defer full matching till the next iteration. 334 current_pos = next_delim 335 continue 336 337 current_pos = len(read_buffer.data) 338 339 def _try_to_ensure_num_bytes_in_buffer( 340 self, file_to_read, read_buffer, num_bytes): 341 # Tries to ensure that there are at least num_bytes bytes in the buffer. 342 # Returns True if this can be fulfilled, returned False if this cannot be 343 # fulfilled due to reaching EOF. 344 while len(read_buffer.data) < num_bytes: 345 read_data = file_to_read.read(self._buffer_size) 346 if not read_data: 347 return False 348 349 read_buffer.data += read_data 350 351 return True 352 353 def _skip_lines(self, file_to_read, read_buffer, num_lines): 354 """Skip num_lines from file_to_read, return num_lines+1 start position.""" 355 if file_to_read.tell() > 0: 356 file_to_read.seek(0) 357 position = 0 358 for _ in range(num_lines): 359 _, num_bytes_to_next_record = self._read_record(file_to_read, read_buffer) 360 if num_bytes_to_next_record < 0: 361 # We reached end of file. It is OK to just break here 362 # because subsequent _read_record will return same result. 363 break 364 position += num_bytes_to_next_record 365 return position 366 367 def _read_record(self, file_to_read, read_buffer): 368 # Returns a tuple containing the current_record and number of bytes to the 369 # next record starting from 'read_buffer.position'. If EOF is 370 # reached, returns a tuple containing the current record and -1. 371 372 if read_buffer.position > self._buffer_size: 373 # read_buffer is too large. Truncating and adjusting it. 374 read_buffer.data = read_buffer.data[read_buffer.position:] 375 read_buffer.position = 0 376 377 record_start_position_in_buffer = read_buffer.position 378 sep_bounds = self._find_separator_bounds(file_to_read, read_buffer) 379 read_buffer.position = sep_bounds[1] if sep_bounds else len( 380 read_buffer.data) 381 382 if not sep_bounds: 383 # Reached EOF. Bytes up to the EOF is the next record. Returning '-1' for 384 # the starting position of the next record. 385 return (read_buffer.data[record_start_position_in_buffer:], -1) 386 387 if self._strip_trailing_newlines: 388 # Current record should not contain the delimiter. 389 return ( 390 read_buffer.data[record_start_position_in_buffer:sep_bounds[0]], 391 sep_bounds[1] - record_start_position_in_buffer) 392 else: 393 # Current record should contain the delimiter. 394 return ( 395 read_buffer.data[record_start_position_in_buffer:sep_bounds[1]], 396 sep_bounds[1] - record_start_position_in_buffer) 397 398 @staticmethod 399 def _is_self_overlapping(delimiter): 400 # A delimiter self-overlaps if it has a prefix that is also its suffix. 401 for i in range(1, len(delimiter)): 402 if delimiter[0:i] == delimiter[len(delimiter) - i:]: 403 return True 404 return False 405 406 def _is_escaped(self, read_buffer, position): 407 # Returns True if byte at position is preceded with an odd number 408 # of escapechar bytes or False if preceded by 0 or even escapes 409 # (the even number means that all the escapes are escaped themselves). 410 escape_count = 0 411 for current_pos in reversed(range(0, position)): 412 if read_buffer.data[current_pos:current_pos + 1] != self._escapechar: 413 break 414 escape_count += 1 415 return escape_count % 2 == 1 416 417 def output_type_hint(self): 418 try: 419 return self._coder.to_type_hint() 420 except NotImplementedError: 421 return Any 422 423 424 class _TextSourceWithFilename(_TextSource): 425 def read_records(self, file_name, range_tracker): 426 records = super().read_records(file_name, range_tracker) 427 for record in records: 428 yield (file_name, record) 429 430 def output_type_hint(self): 431 return typehints.KV[str, super().output_type_hint()] 432 433 434 class _TextSink(filebasedsink.FileBasedSink): 435 """A sink to a GCS or local text file or files.""" 436 437 def __init__(self, 438 file_path_prefix, 439 file_name_suffix='', 440 append_trailing_newlines=True, 441 num_shards=0, 442 shard_name_template=None, 443 coder=coders.ToBytesCoder(), # type: coders.Coder 444 compression_type=CompressionTypes.AUTO, 445 header=None, 446 footer=None, 447 *, 448 max_records_per_shard=None, 449 max_bytes_per_shard=None, 450 skip_if_empty=False): 451 """Initialize a _TextSink. 452 453 Args: 454 file_path_prefix: The file path to write to. The files written will begin 455 with this prefix, followed by a shard identifier (see num_shards), and 456 end in a common extension, if given by file_name_suffix. In most cases, 457 only this argument is specified and num_shards, shard_name_template, and 458 file_name_suffix use default values. 459 file_name_suffix: Suffix for the files written. 460 append_trailing_newlines: indicate whether this sink should write an 461 additional newline char after writing each element. 462 num_shards: The number of files (shards) used for output. If not set, the 463 service will decide on the optimal number of shards. 464 Constraining the number of shards is likely to reduce 465 the performance of a pipeline. Setting this value is not recommended 466 unless you require a specific number of output files. 467 shard_name_template: A template string containing placeholders for 468 the shard number and shard count. When constructing a filename for a 469 particular shard number, the upper-case letters 'S' and 'N' are 470 replaced with the 0-padded shard number and shard count respectively. 471 This argument can be '' in which case it behaves as if num_shards was 472 set to 1 and only one file will be generated. The default pattern used 473 is '-SSSSS-of-NNNNN' if None is passed as the shard_name_template. 474 coder: Coder used to encode each line. 475 compression_type: Used to handle compressed output files. Typical value 476 is CompressionTypes.AUTO, in which case the final file path's 477 extension (as determined by file_path_prefix, file_name_suffix, 478 num_shards and shard_name_template) will be used to detect the 479 compression. 480 header: String to write at beginning of file as a header. If not None and 481 append_trailing_newlines is set, '\n' will be added. 482 footer: String to write at the end of file as a footer. If not None and 483 append_trailing_newlines is set, '\n' will be added. 484 max_records_per_shard: Maximum number of records to write to any 485 individual shard. 486 max_bytes_per_shard: Target maximum number of bytes to write to any 487 individual shard. This may be exceeded slightly, as a new shard is 488 created once this limit is hit, but the remainder of a given record, a 489 subsequent newline, and a footer may cause the actual shard size 490 to exceed this value. This also tracks the uncompressed, 491 not compressed, size of the shard. 492 skip_if_empty: Don't write any shards if the PCollection is empty. 493 494 Returns: 495 A _TextSink object usable for writing. 496 """ 497 super().__init__( 498 file_path_prefix, 499 file_name_suffix=file_name_suffix, 500 num_shards=num_shards, 501 shard_name_template=shard_name_template, 502 coder=coder, 503 mime_type='text/plain', 504 compression_type=compression_type, 505 max_records_per_shard=max_records_per_shard, 506 max_bytes_per_shard=max_bytes_per_shard, 507 skip_if_empty=skip_if_empty) 508 self._append_trailing_newlines = append_trailing_newlines 509 self._header = header 510 self._footer = footer 511 512 def open(self, temp_path): 513 file_handle = super().open(temp_path) 514 if self._header is not None: 515 file_handle.write(coders.ToBytesCoder().encode(self._header)) 516 if self._append_trailing_newlines: 517 file_handle.write(b'\n') 518 return file_handle 519 520 def close(self, file_handle): 521 if self._footer is not None: 522 file_handle.write(coders.ToBytesCoder().encode(self._footer)) 523 if self._append_trailing_newlines: 524 file_handle.write(b'\n') 525 super().close(file_handle) 526 527 def display_data(self): 528 dd_parent = super().display_data() 529 dd_parent['append_newline'] = DisplayDataItem( 530 self._append_trailing_newlines, label='Append Trailing New Lines') 531 return dd_parent 532 533 def write_encoded_record(self, file_handle, encoded_value): 534 """Writes a single encoded record.""" 535 file_handle.write(encoded_value) 536 if self._append_trailing_newlines: 537 file_handle.write(b'\n') 538 539 540 def _create_text_source( 541 file_pattern=None, 542 min_bundle_size=None, 543 compression_type=None, 544 strip_trailing_newlines=None, 545 coder=None, 546 validate=False, 547 skip_header_lines=None, 548 delimiter=None, 549 escapechar=None): 550 return _TextSource( 551 file_pattern=file_pattern, 552 min_bundle_size=min_bundle_size, 553 compression_type=compression_type, 554 strip_trailing_newlines=strip_trailing_newlines, 555 coder=coder, 556 validate=validate, 557 skip_header_lines=skip_header_lines, 558 delimiter=delimiter, 559 escapechar=escapechar) 560 561 562 class ReadAllFromText(PTransform): 563 """A ``PTransform`` for reading a ``PCollection`` of text files. 564 565 Reads a ``PCollection`` of text files or file patterns and produces a 566 ``PCollection`` of strings. 567 568 Parses a text file as newline-delimited elements, by default assuming 569 UTF-8 encoding. Supports newline delimiters '\\n' and '\\r\\n'. 570 571 If `with_filename` is ``True`` the output will include the file name. This is 572 similar to ``ReadFromTextWithFilename`` but this ``PTransform`` can be placed 573 anywhere in the pipeline. 574 575 If reading from a text file that that requires a different encoding, you may 576 provide a custom :class:`~apache_beam.coders.coders.Coder` that encodes and 577 decodes with the appropriate codec. For example, see the implementation of 578 :class:`~apache_beam.coders.coders.StrUtf8Coder`. 579 580 This does not support ``UTF-16`` or ``UTF-32`` encodings. 581 582 This implementation is only tested with batch pipeline. In streaming, 583 reading may happen with delay due to the limitation in ReShuffle involved. 584 """ 585 DEFAULT_DESIRED_BUNDLE_SIZE = 64 * 1024 * 1024 # 64MB 586 587 def __init__( 588 self, 589 min_bundle_size=0, 590 desired_bundle_size=DEFAULT_DESIRED_BUNDLE_SIZE, 591 compression_type=CompressionTypes.AUTO, 592 strip_trailing_newlines=True, 593 validate=False, 594 coder=coders.StrUtf8Coder(), # type: coders.Coder 595 skip_header_lines=0, 596 with_filename=False, 597 delimiter=None, 598 escapechar=None, 599 **kwargs): 600 """Initialize the ``ReadAllFromText`` transform. 601 602 Args: 603 min_bundle_size: Minimum size of bundles that should be generated when 604 splitting this source into bundles. See ``FileBasedSource`` for more 605 details. 606 desired_bundle_size: Desired size of bundles that should be generated when 607 splitting this source into bundles. See ``FileBasedSource`` for more 608 details. 609 compression_type: Used to handle compressed input files. Typical value 610 is ``CompressionTypes.AUTO``, in which case the underlying file_path's 611 extension will be used to detect the compression. 612 strip_trailing_newlines: Indicates whether this source should remove 613 the newline char in each line it reads before decoding that line. 614 validate: flag to verify that the files exist during the pipeline 615 creation time. 616 skip_header_lines: Number of header lines to skip. Same number is skipped 617 from each source file. Must be 0 or higher. Large number of skipped 618 lines might impact performance. 619 coder: Coder used to decode each line. 620 with_filename: If True, returns a Key Value with the key being the file 621 name and the value being the actual data. If False, it only returns 622 the data. 623 delimiter (bytes) Optional: delimiter to split records. 624 Must not self-overlap, because self-overlapping delimiters cause 625 ambiguous parsing. 626 escapechar (bytes) Optional: a single byte to escape the records 627 delimiter, can also escape itself. 628 """ 629 super().__init__(**kwargs) 630 self._source_from_file = partial( 631 _create_text_source, 632 min_bundle_size=min_bundle_size, 633 compression_type=compression_type, 634 strip_trailing_newlines=strip_trailing_newlines, 635 validate=validate, 636 coder=coder, 637 skip_header_lines=skip_header_lines, 638 delimiter=delimiter, 639 escapechar=escapechar) 640 self._desired_bundle_size = desired_bundle_size 641 self._min_bundle_size = min_bundle_size 642 self._compression_type = compression_type 643 self._with_filename = with_filename 644 self._read_all_files = ReadAllFiles( 645 True, 646 self._compression_type, 647 self._desired_bundle_size, 648 self._min_bundle_size, 649 self._source_from_file, 650 self._with_filename) 651 652 def expand(self, pvalue): 653 return pvalue | 'ReadAllFiles' >> self._read_all_files 654 655 656 class ReadAllFromTextContinuously(ReadAllFromText): 657 """A ``PTransform`` for reading text files in given file patterns. 658 This PTransform acts as a Source and produces continuously a ``PCollection`` 659 of strings. 660 661 For more details, see ``ReadAllFromText`` for text parsing settings; 662 see ``apache_beam.io.fileio.MatchContinuously`` for watching settings. 663 664 ReadAllFromTextContinuously is experimental. No backwards-compatibility 665 guarantees. Due to the limitation on Reshuffle, current implementation does 666 not scale. 667 """ 668 _ARGS_FOR_MATCH = ( 669 'interval', 670 'has_deduplication', 671 'start_timestamp', 672 'stop_timestamp', 673 'match_updated_files', 674 'apply_windowing') 675 _ARGS_FOR_READ = ( 676 'min_bundle_size', 677 'desired_bundle_size', 678 'compression_type', 679 'strip_trailing_newlines', 680 'validate', 681 'coder', 682 'skip_header_lines', 683 'with_filename', 684 'delimiter', 685 'escapechar') 686 687 def __init__(self, file_pattern, **kwargs): 688 """Initialize the ``ReadAllFromTextContinuously`` transform. 689 690 Accepts args for constructor args of both :class:`ReadAllFromText` and 691 :class:`~apache_beam.io.fileio.MatchContinuously`. 692 """ 693 kwargs_for_match = { 694 k: v 695 for (k, v) in kwargs.items() if k in self._ARGS_FOR_MATCH 696 } 697 kwargs_for_read = { 698 k: v 699 for (k, v) in kwargs.items() if k in self._ARGS_FOR_READ 700 } 701 kwargs_additinal = { 702 k: v 703 for (k, v) in kwargs.items() 704 if k not in self._ARGS_FOR_MATCH and k not in self._ARGS_FOR_READ 705 } 706 super().__init__(**kwargs_for_read, **kwargs_additinal) 707 self._file_pattern = file_pattern 708 self._kwargs_for_match = kwargs_for_match 709 710 def expand(self, pbegin): 711 # Importing locally to prevent circular dependency issues. 712 from apache_beam.io.fileio import MatchContinuously 713 714 # TODO(BEAM-14497) always reshuffle once gbk always trigger works. 715 return ( 716 pbegin 717 | MatchContinuously(self._file_pattern, **self._kwargs_for_match) 718 | 'ReadAllFiles' >> self._read_all_files._disable_reshuffle()) 719 720 721 class ReadFromText(PTransform): 722 r"""A :class:`~apache_beam.transforms.ptransform.PTransform` for reading text 723 files. 724 725 Parses a text file as newline-delimited elements, by default assuming 726 ``UTF-8`` encoding. Supports newline delimiters ``\n`` and ``\r\n`` 727 or specified delimiter. 728 729 If reading from a text file that that requires a different encoding, you may 730 provide a custom :class:`~apache_beam.coders.coders.Coder` that encodes and 731 decodes with the appropriate codec. For example, see the implementation of 732 :class:`~apache_beam.coders.coders.StrUtf8Coder`. 733 734 This does not support ``UTF-16`` or ``UTF-32`` encodings. 735 """ 736 737 _source_class = _TextSource 738 739 def __init__( 740 self, 741 file_pattern=None, 742 min_bundle_size=0, 743 compression_type=CompressionTypes.AUTO, 744 strip_trailing_newlines=True, 745 coder=coders.StrUtf8Coder(), # type: coders.Coder 746 validate=True, 747 skip_header_lines=0, 748 delimiter=None, 749 escapechar=None, 750 **kwargs): 751 """Initialize the :class:`ReadFromText` transform. 752 753 Args: 754 file_pattern (str): The file path to read from as a local file path or a 755 GCS ``gs://`` path. The path can contain glob characters 756 (``*``, ``?``, and ``[...]`` sets). 757 min_bundle_size (int): Minimum size of bundles that should be generated 758 when splitting this source into bundles. See 759 :class:`~apache_beam.io.filebasedsource.FileBasedSource` for more 760 details. 761 compression_type (str): Used to handle compressed input files. 762 Typical value is :attr:`CompressionTypes.AUTO 763 <apache_beam.io.filesystem.CompressionTypes.AUTO>`, in which case the 764 underlying file_path's extension will be used to detect the compression. 765 strip_trailing_newlines (bool): Indicates whether this source should 766 remove the newline char in each line it reads before decoding that line. 767 validate (bool): flag to verify that the files exist during the pipeline 768 creation time. 769 skip_header_lines (int): Number of header lines to skip. Same number is 770 skipped from each source file. Must be 0 or higher. Large number of 771 skipped lines might impact performance. 772 coder (~apache_beam.coders.coders.Coder): Coder used to decode each line. 773 delimiter (bytes) Optional: delimiter to split records. 774 Must not self-overlap, because self-overlapping delimiters cause 775 ambiguous parsing. 776 escapechar (bytes) Optional: a single byte to escape the records 777 delimiter, can also escape itself. 778 """ 779 780 super().__init__(**kwargs) 781 self._source = self._source_class( 782 file_pattern, 783 min_bundle_size, 784 compression_type, 785 strip_trailing_newlines, 786 coder, 787 validate=validate, 788 skip_header_lines=skip_header_lines, 789 delimiter=delimiter, 790 escapechar=escapechar) 791 792 def expand(self, pvalue): 793 return pvalue.pipeline | Read(self._source).with_output_types( 794 self._source.output_type_hint()) 795 796 797 class ReadFromTextWithFilename(ReadFromText): 798 r"""A :class:`~apache_beam.io.textio.ReadFromText` for reading text 799 files returning the name of the file and the content of the file. 800 801 This class extend ReadFromText class just setting a different 802 _source_class attribute. 803 """ 804 805 _source_class = _TextSourceWithFilename 806 807 808 class WriteToText(PTransform): 809 """A :class:`~apache_beam.transforms.ptransform.PTransform` for writing to 810 text files.""" 811 812 def __init__( 813 self, 814 file_path_prefix, # type: str 815 file_name_suffix='', 816 append_trailing_newlines=True, 817 num_shards=0, 818 shard_name_template=None, # type: Optional[str] 819 coder=coders.ToBytesCoder(), # type: coders.Coder 820 compression_type=CompressionTypes.AUTO, 821 header=None, 822 footer=None, 823 *, 824 max_records_per_shard=None, 825 max_bytes_per_shard=None, 826 skip_if_empty=False): 827 r"""Initialize a :class:`WriteToText` transform. 828 829 Args: 830 file_path_prefix (str): The file path to write to. The files written will 831 begin with this prefix, followed by a shard identifier (see 832 **num_shards**), and end in a common extension, if given by 833 **file_name_suffix**. In most cases, only this argument is specified and 834 **num_shards**, **shard_name_template**, and **file_name_suffix** use 835 default values. 836 file_name_suffix (str): Suffix for the files written. 837 append_trailing_newlines (bool): indicate whether this sink should write 838 an additional newline char after writing each element. 839 num_shards (int): The number of files (shards) used for output. 840 If not set, the service will decide on the optimal number of shards. 841 Constraining the number of shards is likely to reduce 842 the performance of a pipeline. Setting this value is not recommended 843 unless you require a specific number of output files. 844 shard_name_template (str): A template string containing placeholders for 845 the shard number and shard count. Currently only ``''`` and 846 ``'-SSSSS-of-NNNNN'`` are patterns accepted by the service. 847 When constructing a filename for a particular shard number, the 848 upper-case letters ``S`` and ``N`` are replaced with the ``0``-padded 849 shard number and shard count respectively. This argument can be ``''`` 850 in which case it behaves as if num_shards was set to 1 and only one file 851 will be generated. The default pattern used is ``'-SSSSS-of-NNNNN'``. 852 coder (~apache_beam.coders.coders.Coder): Coder used to encode each line. 853 compression_type (str): Used to handle compressed output files. 854 Typical value is :class:`CompressionTypes.AUTO 855 <apache_beam.io.filesystem.CompressionTypes.AUTO>`, in which case the 856 final file path's extension (as determined by **file_path_prefix**, 857 **file_name_suffix**, **num_shards** and **shard_name_template**) will 858 be used to detect the compression. 859 header (str): String to write at beginning of file as a header. 860 If not :data:`None` and **append_trailing_newlines** is set, ``\n`` will 861 be added. 862 footer (str): String to write at the end of file as a footer. 863 If not :data:`None` and **append_trailing_newlines** is set, ``\n`` will 864 be added. 865 max_records_per_shard: Maximum number of records to write to any 866 individual shard. 867 max_bytes_per_shard: Target maximum number of bytes to write to any 868 individual shard. This may be exceeded slightly, as a new shard is 869 created once this limit is hit, but the remainder of a given record, a 870 subsequent newline, and a footer may cause the actual shard size 871 to exceed this value. This also tracks the uncompressed, 872 not compressed, size of the shard. 873 skip_if_empty: Don't write any shards if the PCollection is empty. 874 """ 875 876 self._sink = _TextSink( 877 file_path_prefix, 878 file_name_suffix, 879 append_trailing_newlines, 880 num_shards, 881 shard_name_template, 882 coder, 883 compression_type, 884 header, 885 footer, 886 max_records_per_shard=max_records_per_shard, 887 max_bytes_per_shard=max_bytes_per_shard, 888 skip_if_empty=skip_if_empty) 889 890 def expand(self, pcoll): 891 return pcoll | Write(self._sink) 892 893 894 try: 895 import pandas 896 897 def append_pandas_args(src, exclude): 898 def append(dest): 899 state = None 900 skip = False 901 extra_lines = [] 902 for line in src.__doc__.split('\n'): 903 if line.strip() == 'Parameters': 904 indent = len(line) - len(line.lstrip()) 905 extra_lines = ['\n\nPandas Parameters'] 906 state = 'append' 907 continue 908 elif line.strip().startswith('Returns'): 909 break 910 911 if state == 'append': 912 if skip: 913 if line and not line[indent:].startswith(' '): 914 skip = False 915 if any(line.strip().startswith(arg + ' : ') for arg in exclude): 916 skip = True 917 if not skip: 918 extra_lines.append(line[indent:]) 919 # Expand title underline due to Parameters -> Pandas Parameters. 920 extra_lines[1] += '-------' 921 dest.__doc__ += '\n'.join(extra_lines) 922 return dest 923 924 return append 925 926 @append_pandas_args( 927 pandas.read_csv, exclude=['filepath_or_buffer', 'iterator']) 928 def ReadFromCsv(path: str, *, splittable: bool = True, **kwargs): 929 """A PTransform for reading comma-separated values (csv) files into a 930 PCollection. 931 932 Args: 933 path (str): The file path to read from. The path can contain glob 934 characters such as ``*`` and ``?``. 935 splittable (bool): Whether the csv files are splittable at line 936 boundaries, i.e. each line of this file represents a complete record. 937 This should be set to False if single records span multiple lines (e.g. 938 a quoted field has a newline inside of it). Setting this to false may 939 disable liquid sharding. 940 **kwargs: Extra arguments passed to `pandas.read_csv` (see below). 941 """ 942 from apache_beam.dataframe.io import ReadViaPandas 943 return 'ReadFromCsv' >> ReadViaPandas( 944 'csv', path, splittable=splittable, **kwargs) 945 946 @append_pandas_args( 947 pandas.DataFrame.to_csv, exclude=['path_or_buf', 'index', 'index_label']) 948 def WriteToCsv( 949 path: str, 950 num_shards: Optional[int] = None, 951 file_naming: Optional['fileio.FileNaming'] = None, 952 **kwargs): 953 # pylint: disable=line-too-long 954 955 """A PTransform for writing a schema'd PCollection as a (set of) 956 comma-separated values (csv) files. 957 958 Args: 959 path (str): The file path to write to. The files written will 960 begin with this prefix, followed by a shard identifier (see 961 `num_shards`) according to the `file_naming` parameter. 962 num_shards (optional int): The number of shards to use in the distributed 963 write. Defaults to None, letting the system choose an optimal value. 964 file_naming (optional callable): A file-naming strategy, determining the 965 actual shard names given their shard number, etc. 966 See the section on `file naming 967 <https://beam.apache.org/releases/pydoc/current/apache_beam.io.fileio.html#file-naming>`_ 968 Defaults to `fileio.default_file_naming`, which names files as 969 `path-XXXXX-of-NNNNN`. 970 **kwargs: Extra arguments passed to `pandas.Dataframe.to_csv` (see below). 971 """ 972 from apache_beam.dataframe.io import WriteViaPandas 973 if num_shards is not None: 974 kwargs['num_shards'] = num_shards 975 if file_naming is not None: 976 kwargs['file_naming'] = file_naming 977 return 'WriteToCsv' >> WriteViaPandas('csv', path, index=False, **kwargs) 978 979 @append_pandas_args(pandas.read_json, exclude=['path_or_buf']) 980 def ReadFromJson( 981 path: str, *, orient: str = 'records', lines: bool = True, **kwargs): 982 """A PTransform for reading json values from files into a PCollection. 983 984 Args: 985 path (str): The file path to read from. The path can contain glob 986 characters such as ``*`` and ``?``. 987 orient (str): Format of the json elements in the file. 988 Default to 'records', meaning the file is expected to contain a list 989 of json objects like `{field1: value1, field2: value2, ...}`. 990 lines (bool): Whether each line should be considered a separate record, 991 as opposed to the entire file being a valid JSON object or list. 992 Defaults to True (unlike Pandas). 993 **kwargs: Extra arguments passed to `pandas.read_json` (see below). 994 """ 995 from apache_beam.dataframe.io import ReadViaPandas 996 return 'ReadFromJson' >> ReadViaPandas( 997 'json', path, orient=orient, lines=lines, **kwargs) 998 999 @append_pandas_args( 1000 pandas.DataFrame.to_json, exclude=['path_or_buf', 'index']) 1001 def WriteToJson( 1002 path: str, 1003 *, 1004 num_shards: Optional[int] = None, 1005 file_naming: Optional['fileio.FileNaming'] = None, 1006 orient: str = 'records', 1007 lines: Optional[bool] = None, 1008 **kwargs): 1009 # pylint: disable=line-too-long 1010 1011 """A PTransform for writing a PCollection as json values to files. 1012 1013 Args: 1014 path (str): The file path to write to. The files written will 1015 begin with this prefix, followed by a shard identifier (see 1016 `num_shards`) according to the `file_naming` parameter. 1017 num_shards (optional int): The number of shards to use in the distributed 1018 write. Defaults to None, letting the system choose an optimal value. 1019 file_naming (optional callable): A file-naming strategy, determining the 1020 actual shard names given their shard number, etc. 1021 See the section on `file naming 1022 <https://beam.apache.org/releases/pydoc/current/apache_beam.io.fileio.html#file-naming>`_ 1023 Defaults to `fileio.default_file_naming`, which names files as 1024 `path-XXXXX-of-NNNNN`. 1025 orient (str): Format of the json elements in the file. 1026 Default to 'records', meaning the file will to contain a list 1027 of json objects like `{field1: value1, field2: value2, ...}`. 1028 lines (bool): Whether each line should be considered a separate record, 1029 as opposed to the entire file being a valid JSON object or list. 1030 Defaults to True if orient is 'records' (unlike Pandas). 1031 **kwargs: Extra arguments passed to `pandas.Dataframe.to_json` 1032 (see below). 1033 """ 1034 from apache_beam.dataframe.io import WriteViaPandas 1035 if num_shards is not None: 1036 kwargs['num_shards'] = num_shards 1037 if file_naming is not None: 1038 kwargs['file_naming'] = file_naming 1039 if lines is None: 1040 lines = orient == 'records' 1041 return 'WriteToJson' >> WriteViaPandas( 1042 'json', path, orient=orient, lines=lines, **kwargs) 1043 1044 except ImportError: 1045 1046 def no_pandas(*args, **kwargs): 1047 raise ImportError('Please install apache_beam[dataframe]') 1048 1049 for transform in ('ReadFromCsv', 'WriteToCsv', 'ReadFromJson', 'WriteToJson'): 1050 globals()[transform] = no_pandas