[
https://issues.apache.org/jira/browse/FLINK-6652?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16042612#comment-16042612
]
ASF GitHub Bot commented on FLINK-6652:
---------------------------------------
GitHub user fhueske opened a pull request:
https://github.com/apache/flink/pull/4088
[FLINK-6652] [core] Fix handling of delimiters split by buffers in
DelimitedInputFormat
This PR fixes a bug introduced by fix FLINK-5771 / PR #3316. That fix
resets the read position if a character sequence matches a prefix of a multi
char delimiter. This will yield an `IndexOutOfBoundsException` if prefix spans
the boundaries of the read buffer.
This fix moves the prefix of the multi char delimiter to the beginning of
the read buffer if the read buffer was completely read. In case the read
position has to be reset, the prefix is still in the read buffer and can be
parsed again.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/fhueske/flink fixDelimIF
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/flink/pull/4088.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #4088
----
commit 740a2f42788e1cb957ab1e79b9b67dcfb06b875d
Author: Fabian Hueske <[email protected]>
Date: 2017-06-07T21:01:06Z
[FLINK-6652] [core] Fix handling of delimiters split by buffers in
DelimitedInputFormat
----
> Problem with DelimitedInputFormat
> ---------------------------------
>
> Key: FLINK-6652
> URL: https://issues.apache.org/jira/browse/FLINK-6652
> Project: Flink
> Issue Type: Bug
> Components: Batch Connectors and Input/Output Formats
> Affects Versions: 1.2.1
> Reporter: Moritz Schubotz
> Assignee: Fabian Hueske
> Fix For: 1.2.0
>
>
> After upgrading from Flink 1.2.0 to 1.2.1 I got the following error
> ```
> 07:54:52,395 ERROR org.apache.flink.api.common.io.DelimitedInputFormat
> - Unexpected problen while getting the file statistics for file
> 'mytestfile': -1
> java.lang.ArrayIndexOutOfBoundsException: -1
> at
> org.apache.flink.api.common.io.DelimitedInputFormat.readLine(DelimitedInputFormat.java:572)
> at
> org.apache.flink.api.common.io.DelimitedInputFormat.getStatistics(DelimitedInputFormat.java:423)
> at
> org.apache.flink.api.common.io.DelimitedInputFormat.getStatistics(DelimitedInputFormat.java:48)
> at
> org.apache.flink.optimizer.dag.DataSourceNode.computeOperatorSpecificDefaultEstimates(DataSourceNode.java:166)
> ```
> I have created a test repo to isolate the issue here
> https://github.com/physikerwelt/flinkReadTest
> and reproduced the bug using travis
> https://travis-ci.org/physikerwelt/flinkReadTest
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)