[
https://issues.apache.org/jira/browse/BEAM-13189?focusedWorklogId=679336&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-679336
]
ASF GitHub Bot logged work on BEAM-13189:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 09/Nov/21 21:36
Start Date: 09/Nov/21 21:36
Worklog Time Spent: 10m
Work Description: nikie commented on a change in pull request #15901:
URL: https://github.com/apache/beam/pull/15901#discussion_r746062495
##########
File path: sdks/python/apache_beam/io/textio.py
##########
@@ -362,6 +391,15 @@ def _is_self_overlapping(delimiter):
return True
return False
+ def _is_escaped(self, read_buffer, position):
+ # Returns True if byte at position is preceded with an odd number
+ # of escapechar bytes or False if preceded by 0 or even escapes
+ # (the even number means that all the escapes are escaped themselves).
+ for current_pos in reversed(range(-1, position)):
+ if read_buffer.data[current_pos:current_pos + 1] != self._escapechar:
Review comment:
Totally agree about the trickiness. But it is beautiful and works :)
I will look for more explicit solution if you insist, though.
1. -1 is needed to check if the 1st character is escaped without additional
check if it is the first or not and be able to return from the same if block if
we reached the start of buffer, -1:0 range returns empty bytes which cannot be
equal to escapechar as we do not allow empty one:
```
>>> b'\\'[-1:0]
b''
>>> b'\\'[-1:0] == b'\\'
False
```
2. No, we cannot compare `byte` to `bytes` (I was a bit surprised as well,
but we are doing similar comparisons in `_find_separator_bounds`):
```
>>> b'a'[0] == b'a'
False
>>> b'a'[0:1] == b'a'
True
```
"why not to count preceding_escape_chars" - we are kind of counting, just
using the loop variable for this (by the time we reached non-escapechar, it
contains the number of consecutive escapechars, after adjusting for the
position)
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 679336)
Time Spent: 3h 10m (was: 3h)
> Add escapechar to Python TextIO reads
> -------------------------------------
>
> Key: BEAM-13189
> URL: https://issues.apache.org/jira/browse/BEAM-13189
> Project: Beam
> Issue Type: New Feature
> Components: io-py-common, io-py-files
> Reporter: Eugene Nikolaiev
> Assignee: Eugene Nikolaiev
> Priority: P2
> Time Spent: 3h 10m
> Remaining Estimate: 0h
>
> 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|https://docs.python.org/3/library/csv.html#csv.Dialect.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|https://experienceleague.adobe.com/docs/analytics/export/analytics-data-feed/data-feed-contents/datafeeds-spec-chars.html?lang=en]
> It would be nice if TextIO transforms {{ReadFromText}} and
> {{ReadAllFromText}} had support for {{escapechar}} as follows:
>
> {code:java}
> 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)))
> )
> {code}
> This would print:
> {code:java}
> 'a\\\na\taa'
> 'bb\tbb'
> ['a\na', 'aa']
> ['bb', 'bb']
> {code}
>
>
>
--
This message was sent by Atlassian Jira
(v8.20.1#820001)