Details
-
New Feature
-
Status: Resolved
-
P2
-
Resolution: Fixed
-
None
-
None
Description
Existing TextIO connector can be used for splitting lines of CSV or tab-delimited files for its ability to read large files in parallel and rebalance the work. Each line then can be parsed with csv library separately. This works, if there are no line delimiters inside the lines. Otherwise the lines are split incorrectly.
One of tab-delimited dialects uses escape characters to escape the line and column delimiters (usually backslash) instead of quoting the columns. This can be parsed with Python csv library using escapechar dialect parameter.
The escapechar itself can also be escaped to allow having such character before the line delimiters.
Example of such file format usage: Adobe Analytics Data Feed
It would be nice if TextIO transforms ReadFromText and ReadAllFromText had support for escapechar as follows:
import csv import tempfile import apache_beam as beam with tempfile.NamedTemporaryFile('w') as temp_file: # Write CSV lines with escaped line terminator temp_file.write('a\\\na\taa\n') temp_file.write('bb\tbb\n') temp_file.flush() # Read and print lines with beam.Pipeline() as pipeline: ( pipeline | beam.io.ReadFromText(file_pattern=temp_file.name, escapechar=b'\\') | beam.Map(lambda x: print(repr(x))) ) # Read lines, parse and print TSV rows with beam.Pipeline() as pipeline: ( pipeline | beam.io.ReadFromText(file_pattern=temp_file.name, escapechar=b'\\') | beam.Map(lambda x: next(csv.reader([x], escapechar='\\', delimiter='\t'))) | beam.Map(lambda x: print(repr(x))) )
This would print:
'a\\\na\taa' 'bb\tbb' ['a\na', 'aa'] ['bb', 'bb']
Attachments
Issue Links
- links to