[jira] [Comment Edited] (BEAM-5180) Broken FileResultCoder via parseSchema change

2018-08-24 Thread Jozef Vilcek (JIRA)


[ 
https://issues.apache.org/jira/browse/BEAM-5180?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16592486#comment-16592486
 ] 

Jozef Vilcek edited comment on BEAM-5180 at 8/25/18 6:11 AM:
-

Sorry, I am unable to see how // is enforced. I see 
{noformat}
[scheme:][//authority][path][?query][#fragment]{noformat}
which I interpret as if authority is not present, `//` does not have to be 
there either.

In reality, this is the behaviour:
{code:java}
System.out.println(new java.net.URI("hdfs:/path/to/file"));
System.out.println(new java.net.URI("hdfs:///path/to/file"));
System.out.println(new 
java.net.URI("hdfs:///path/to/file").resolve("something-else"));
{code}
prints
{noformat}
hdfs:/path/to/file
hdfs:///path/to/file
hdfs:/path/to/something-else
{noformat}
So even if user specifies hdfs://  paths as entry, it will be due to operations 
within the beam translated to hdfs:/


was (Author: jozovilcek):
Sorry, I am unable to see how // is enforced. I see 
{noformat}
[scheme:][//authority][path][?query][#fragment]{noformat}
which I interpret as if authority is not present, `//` does not have to be 
there either.

In reality, this is the behaviour:
{code:java}
System.out.println(new java.net.URI("hdfs:/path/to/file"));
System.out.println(new java.net.URI("hdfs:///path/to/file"));
System.out.println(new 
java.net.URI("hdfs:///path/to/file").resolve("something-else"));
{code}
prints
{code:java}
hdfs:/path/to/file
hdfs:///path/to/file
hdfs:/path/to/something-else
{code}
So even if user specifies hdfs://  paths as entry, it will be due to operations 
within the beam translated to hdfs:/

> Broken FileResultCoder via parseSchema change
> -
>
> Key: BEAM-5180
> URL: https://issues.apache.org/jira/browse/BEAM-5180
> Project: Beam
>  Issue Type: Bug
>  Components: sdk-java-core
>Affects Versions: 2.6.0
>Reporter: Jozef Vilcek
>Assignee: Ankur Goenka
>Priority: Blocker
> Fix For: 2.7.0
>
>  Time Spent: 2.5h
>  Remaining Estimate: 0h
>
> Recently this commit
> [https://github.com/apache/beam/commit/3fff58c21f94415f3397e185377e36d3df662384]
> introduced more strict schema parsing which is breaking the contract between 
> _FileResultCoder_ and _FileSystems.matchNewResource()_.
> Coder takes _ResourceId_ and serialize it via `_toString_` methods and then 
> relies on filesystem being able to parse it back again. Having strict 
> _scheme://_ breaks this at least for Hadoop filesystem which use _URI_ for 
> _ResourceId_ and produce _toString()_ in form of `_hdfs:/some/path_`
> I guess the _ResourceIdCoder_ is suffering the same problem.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Comment Edited] (BEAM-5180) Broken FileResultCoder via parseSchema change

2018-08-25 Thread Ankur Goenka (JIRA)


[ 
https://issues.apache.org/jira/browse/BEAM-5180?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16591973#comment-16591973
 ] 

Ankur Goenka edited comment on BEAM-5180 at 8/25/18 10:06 PM:
--

>From the java.net.URI docs,

A hierarchical URI is subject to further parsing according to the syntax
{quote}[_scheme_:][//_authority_][_path_][?_query_][#_fragment_|#*}}_fragment_]
{quote}
Which enforces  //

But to support HDFS and unblock our selves we should go with the rollback.


was (Author: angoenka):
>From the java.net.URI docs,

A hierarchical URI is subject to further parsing according to the syntax
{quote}[_scheme_{{*:*}}][{{*//*}}_authority_][_path_][{{*?*}}_query_][{{*#*}}_fragment_]{quote}
Which enforces  //

But to support HDFS and unblock our selves we should go with the rollback.

> Broken FileResultCoder via parseSchema change
> -
>
> Key: BEAM-5180
> URL: https://issues.apache.org/jira/browse/BEAM-5180
> Project: Beam
>  Issue Type: Bug
>  Components: sdk-java-core
>Affects Versions: 2.6.0
>Reporter: Jozef Vilcek
>Assignee: Ankur Goenka
>Priority: Blocker
> Fix For: 2.7.0
>
>  Time Spent: 2h 40m
>  Remaining Estimate: 0h
>
> Recently this commit
> [https://github.com/apache/beam/commit/3fff58c21f94415f3397e185377e36d3df662384]
> introduced more strict schema parsing which is breaking the contract between 
> _FileResultCoder_ and _FileSystems.matchNewResource()_.
> Coder takes _ResourceId_ and serialize it via `_toString_` methods and then 
> relies on filesystem being able to parse it back again. Having strict 
> _scheme://_ breaks this at least for Hadoop filesystem which use _URI_ for 
> _ResourceId_ and produce _toString()_ in form of `_hdfs:/some/path_`
> I guess the _ResourceIdCoder_ is suffering the same problem.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)