[
https://issues.apache.org/jira/browse/BEAM-12730?focusedWorklogId=665769&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-665769
]
ASF GitHub Bot logged work on BEAM-12730:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 17/Oct/21 00:31
Start Date: 17/Oct/21 00:31
Worklog Time Spent: 10m
Work Description: nikie commented on a change in pull request #15667:
URL: https://github.com/apache/beam/pull/15667#discussion_r730046514
##########
File path: sdks/python/apache_beam/io/textio.py
##########
@@ -561,6 +570,7 @@ def __init__(
skipped from each source file. Must be 0 or higher. Large number of
skipped lines might impact performance.
coder (~apache_beam.coders.coders.Coder): Coder used to decode each line.
+ delimiter (str or bytes): delimiter to split records
Review comment:
@dmitriikuzinepam
What happens if `readbuffer` ends in the middle of a multi-byte delimiter?
This line `current_pos = len(read_buffer.data)` together with `next_lf =
read_buffer.data.find(self._delimiter, current_pos)` on next iteration might
skip such delimiter.
Also, it looks like Java SDK's TextIO behaves differently in case if
delimiter is not set: it uses both `\n` and `\r\n` delimiters only in case if
delimiter is not provided. If it is provided - it looks only for the explicit
delimiter, see:
https://github.com/apache/beam/blob/52a178f0a66829bbd1d99fcaf70921a8bd9300f6/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java#L189,
https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java#L208
##########
File path: sdks/python/apache_beam/io/textio.py
##########
@@ -561,6 +570,7 @@ def __init__(
skipped from each source file. Must be 0 or higher. Large number of
skipped lines might impact performance.
coder (~apache_beam.coders.coders.Coder): Coder used to decode each line.
+ delimiter (str or bytes): delimiter to split records
Review comment:
@dmitriikuzinepam
What happens if `readbuffer` ends in the middle of a multi-byte delimiter?
This line `current_pos = len(read_buffer.data)` together with `next_lf =
read_buffer.data.find(self._delimiter, current_pos)` on next iteration might
skip such delimiter.
Also, it looks like Java SDK's TextIO behaves differently in case if
delimiter is not set: it uses both `\n` and `\r\n` delimiters only in case if
delimiter is not provided. If it is provided - it looks only for the explicit
delimiter, see:
https://github.com/apache/beam/blob/52a178f0a66829bbd1d99fcaf70921a8bd9300f6/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java#L189,
https://github.com/apache/beam/blob/52a178f0a66829bbd1d99fcaf70921a8bd9300f6/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java#L208
##########
File path: sdks/python/apache_beam/io/textio.py
##########
@@ -561,6 +570,7 @@ def __init__(
skipped from each source file. Must be 0 or higher. Large number of
skipped lines might impact performance.
coder (~apache_beam.coders.coders.Coder): Coder used to decode each line.
+ delimiter (str or bytes): delimiter to split records
Review comment:
@dmitriikuzinepam
What happens if `readbuffer` ends in the middle of a multi-byte delimiter?
This line `current_pos = len(read_buffer.data)` together with `next_lf =
read_buffer.data.find(self._delimiter, current_pos)` on next iteration might
skip such delimiter.
Also, it looks like Java SDK's TextIO behaves differently in case if
delimiter is not set: it uses both `\n` and `\r\n` delimiters only in case if
delimiter is not provided. If it is provided - it looks only for the explicit
delimiter, see:
https://github.com/apache/beam/blob/52a178f0a66829bbd1d99fcaf70921a8bd9300f6/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java#L189,
https://github.com/apache/beam/blob/52a178f0a66829bbd1d99fcaf70921a8bd9300f6/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java#L208
Actually, the default for the delimiter `delimiter=b'\n',` is a bit
misleading, since under the hood it continues to split on both `\n` and `\r\n`
currently.
--
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: 665769)
Time Spent: 7h 10m (was: 7h)
> Add custom delimiters to Python TextIO reads
> --------------------------------------------
>
> Key: BEAM-12730
> URL: https://issues.apache.org/jira/browse/BEAM-12730
> Project: Beam
> Issue Type: New Feature
> Components: io-py-common, io-py-files
> Reporter: Daniel Oliveira
> Assignee: Dmitrii Kuzin
> Priority: P2
> Labels: beginner, newbie, starter
> Time Spent: 7h 10m
> Remaining Estimate: 0h
>
> A common request by users is to be able to separate a text files read by
> TextIO with delimiters other than newline. The Java SDK already supports this
> feature.
> The current delimiter code is [located
> here|https://github.com/apache/beam/blob/v2.31.0/sdks/python/apache_beam/io/textio.py#L236]
> and defaults to newlines. This function could easily be modified to also
> handle custom delimiters. Changing this would also necessitate changing the
> API for the various TextIO.Read methods and adding documentation.
> This seems like a good starter bug for making more in-depth contributions to
> Beam Python.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)