[jira] [Commented] (BEAM-2658) SerializableCoder has higher precedence over ProtoCoder in CoderRegistry#getCoder

2017-07-26 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on BEAM-2658:
--

Github user asfgit closed the pull request at:

https://github.com/apache/beam/pull/3617


> SerializableCoder has higher precedence over ProtoCoder in 
> CoderRegistry#getCoder
> -
>
> Key: BEAM-2658
> URL: https://issues.apache.org/jira/browse/BEAM-2658
> Project: Beam
>  Issue Type: Bug
>  Components: sdk-java-core
>Affects Versions: 2.0.0
>Reporter: Neville Li
>Assignee: Davor Bonaci
>Priority: Minor
>
> {code}
> import com.google.protobuf.Timestamp;
> import org.apache.beam.sdk.Pipeline;
> import org.apache.beam.sdk.coders.CannotProvideCoderException;
> import org.apache.beam.sdk.coders.Coder;
> import org.apache.beam.sdk.options.PipelineOptions;
> import org.apache.beam.sdk.options.PipelineOptionsFactory;
> public class CoderTest {
>   public static void main(String[] args) throws CannotProvideCoderException {
> PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create();
> Pipeline pipeline = Pipeline.create(options);
> Coder coder = 
> pipeline.getCoderRegistry().getCoder(Timestamp.class);
> // class org.apache.beam.sdk.coders.SerializableCoder
> System.out.println(coder.getClass());
>   }
> }
> {code}
> Right now we're sorting {{CoderProviderRegistrar}}s by canonical name but 
> {{SerializableCoderProvider}} should be added last as a fallback if there're 
> other {{CoderProvider}}s that support the same type.
> {code}
> Set registrars = 
> Sets.newTreeSet(ObjectsClassComparator.INSTANCE);
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (BEAM-2658) SerializableCoder has higher precedence over ProtoCoder in CoderRegistry#getCoder

2017-07-24 Thread Luke Cwik (JIRA)

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

Luke Cwik commented on BEAM-2658:
-

Arbitrary number based precedence orders are difficult to maintain. I agree 
with you that we should special case `DefaultCoder` and `SerializableCoder` for 
now and once we get the third or fourth special case, that may give us 
constraints on how we should structure coder provider ordering.

> SerializableCoder has higher precedence over ProtoCoder in 
> CoderRegistry#getCoder
> -
>
> Key: BEAM-2658
> URL: https://issues.apache.org/jira/browse/BEAM-2658
> Project: Beam
>  Issue Type: Bug
>  Components: sdk-java-core
>Affects Versions: 2.0.0
>Reporter: Neville Li
>Assignee: Davor Bonaci
>Priority: Minor
>
> {code}
> import com.google.protobuf.Timestamp;
> import org.apache.beam.sdk.Pipeline;
> import org.apache.beam.sdk.coders.CannotProvideCoderException;
> import org.apache.beam.sdk.coders.Coder;
> import org.apache.beam.sdk.options.PipelineOptions;
> import org.apache.beam.sdk.options.PipelineOptionsFactory;
> public class CoderTest {
>   public static void main(String[] args) throws CannotProvideCoderException {
> PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create();
> Pipeline pipeline = Pipeline.create(options);
> Coder coder = 
> pipeline.getCoderRegistry().getCoder(Timestamp.class);
> // class org.apache.beam.sdk.coders.SerializableCoder
> System.out.println(coder.getClass());
>   }
> }
> {code}
> Right now we're sorting {{CoderProviderRegistrar}}s by canonical name but 
> {{SerializableCoderProvider}} should be added last as a fallback if there're 
> other {{CoderProvider}}s that support the same type.
> {code}
> Set registrars = 
> Sets.newTreeSet(ObjectsClassComparator.INSTANCE);
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (BEAM-2658) SerializableCoder has higher precedence over ProtoCoder in CoderRegistry#getCoder

2017-07-23 Thread Neville Li (JIRA)

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

Neville Li commented on BEAM-2658:
--

However I'd still argue that {{DefaultCoder}} and {{SerializableCoder}} are 2 
special cases. {{DefaultCoder}} should have the highest precedence and 
overrides typed based lookup while {{SerializableCoder}} should have the lowest 
as a fallback. Other coders in between don't really overlap.

> SerializableCoder has higher precedence over ProtoCoder in 
> CoderRegistry#getCoder
> -
>
> Key: BEAM-2658
> URL: https://issues.apache.org/jira/browse/BEAM-2658
> Project: Beam
>  Issue Type: Bug
>  Components: sdk-java-core
>Affects Versions: 2.0.0
>Reporter: Neville Li
>Assignee: Davor Bonaci
>Priority: Minor
>
> {code}
> import com.google.protobuf.Timestamp;
> import org.apache.beam.sdk.Pipeline;
> import org.apache.beam.sdk.coders.CannotProvideCoderException;
> import org.apache.beam.sdk.coders.Coder;
> import org.apache.beam.sdk.options.PipelineOptions;
> import org.apache.beam.sdk.options.PipelineOptionsFactory;
> public class CoderTest {
>   public static void main(String[] args) throws CannotProvideCoderException {
> PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create();
> Pipeline pipeline = Pipeline.create(options);
> Coder coder = 
> pipeline.getCoderRegistry().getCoder(Timestamp.class);
> // class org.apache.beam.sdk.coders.SerializableCoder
> System.out.println(coder.getClass());
>   }
> }
> {code}
> Right now we're sorting {{CoderProviderRegistrar}}s by canonical name but 
> {{SerializableCoderProvider}} should be added last as a fallback if there're 
> other {{CoderProvider}}s that support the same type.
> {code}
> Set registrars = 
> Sets.newTreeSet(ObjectsClassComparator.INSTANCE);
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (BEAM-2658) SerializableCoder has higher precedence over ProtoCoder in CoderRegistry#getCoder

2017-07-23 Thread Neville Li (JIRA)

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

Neville Li commented on BEAM-2658:
--

Types covered by each {{CoderProvider}} may overlap and we might want sort them 
based on scope and not name.

2 ideas:
1. Add a `List getSupportedTypes()` and make sure that when 2 
providers overlap, the one with the narrower supported types gets precedence. 
Won't work with `DefaultCoder` though. Also a class can implement multiple 
interfaces and it doesn't specify which one has higher precedence. For example 
`ProtoCoder` supports `Message.class` while `SerializableCoder` supports 
`Serializable.class` but generated Protobuf classes also implements 
`Serializable`.
2. Assign an arbitrary int precedent to each coder, similar to UNIX rc/motd 
files, and store registered coders in a `TreeMap`. Not elegant but works.

> SerializableCoder has higher precedence over ProtoCoder in 
> CoderRegistry#getCoder
> -
>
> Key: BEAM-2658
> URL: https://issues.apache.org/jira/browse/BEAM-2658
> Project: Beam
>  Issue Type: Bug
>  Components: sdk-java-core
>Affects Versions: 2.0.0
>Reporter: Neville Li
>Assignee: Davor Bonaci
>Priority: Minor
>
> {code}
> import com.google.protobuf.Timestamp;
> import org.apache.beam.sdk.Pipeline;
> import org.apache.beam.sdk.coders.CannotProvideCoderException;
> import org.apache.beam.sdk.coders.Coder;
> import org.apache.beam.sdk.options.PipelineOptions;
> import org.apache.beam.sdk.options.PipelineOptionsFactory;
> public class CoderTest {
>   public static void main(String[] args) throws CannotProvideCoderException {
> PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create();
> Pipeline pipeline = Pipeline.create(options);
> Coder coder = 
> pipeline.getCoderRegistry().getCoder(Timestamp.class);
> // class org.apache.beam.sdk.coders.SerializableCoder
> System.out.println(coder.getClass());
>   }
> }
> {code}
> Right now we're sorting {{CoderProviderRegistrar}}s by canonical name but 
> {{SerializableCoderProvider}} should be added last as a fallback if there're 
> other {{CoderProvider}}s that support the same type.
> {code}
> Set registrars = 
> Sets.newTreeSet(ObjectsClassComparator.INSTANCE);
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (BEAM-2658) SerializableCoder has higher precedence over ProtoCoder in CoderRegistry#getCoder

2017-07-23 Thread Luke Cwik (JIRA)

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

Luke Cwik commented on BEAM-2658:
-

I think this issue will come up more often then this single case. Is there a 
more general way to solve this issue which doesn't require CoderRegistry to 
resolve conflicts by having them manually specified?

> SerializableCoder has higher precedence over ProtoCoder in 
> CoderRegistry#getCoder
> -
>
> Key: BEAM-2658
> URL: https://issues.apache.org/jira/browse/BEAM-2658
> Project: Beam
>  Issue Type: Bug
>  Components: sdk-java-core
>Affects Versions: 2.0.0
>Reporter: Neville Li
>Assignee: Davor Bonaci
>Priority: Minor
>
> {code}
> import com.google.protobuf.Timestamp;
> import org.apache.beam.sdk.Pipeline;
> import org.apache.beam.sdk.coders.CannotProvideCoderException;
> import org.apache.beam.sdk.coders.Coder;
> import org.apache.beam.sdk.options.PipelineOptions;
> import org.apache.beam.sdk.options.PipelineOptionsFactory;
> public class CoderTest {
>   public static void main(String[] args) throws CannotProvideCoderException {
> PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create();
> Pipeline pipeline = Pipeline.create(options);
> Coder coder = 
> pipeline.getCoderRegistry().getCoder(Timestamp.class);
> // class org.apache.beam.sdk.coders.SerializableCoder
> System.out.println(coder.getClass());
>   }
> }
> {code}
> Right now we're sorting {{CoderProviderRegistrar}}s by canonical name but 
> {{SerializableCoderProvider}} should be added last as a fallback if there're 
> other {{CoderProvider}}s that support the same type.
> {code}
> Set registrars = 
> Sets.newTreeSet(ObjectsClassComparator.INSTANCE);
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (BEAM-2658) SerializableCoder has higher precedence over ProtoCoder in CoderRegistry#getCoder

2017-07-22 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on BEAM-2658:
--

GitHub user nevillelyh opened a pull request:

https://github.com/apache/beam/pull/3617

[BEAM-2658] register SerializableCoder last



You can merge this pull request into a Git repository by running:

$ git pull https://github.com/nevillelyh/beam neville/BEAM-2658

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/beam/pull/3617.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 #3617


commit 3e92083312b7d56a7b436e927d2389dc5d50ace4
Author: Neville Li 
Date:   2017-07-22T07:14:03Z

[BEAM-2658] register SerializableCoder last




> SerializableCoder has higher precedence over ProtoCoder in 
> CoderRegistry#getCoder
> -
>
> Key: BEAM-2658
> URL: https://issues.apache.org/jira/browse/BEAM-2658
> Project: Beam
>  Issue Type: Bug
>  Components: sdk-java-core
>Affects Versions: 2.0.0
>Reporter: Neville Li
>Assignee: Davor Bonaci
>Priority: Minor
>
> {code}
> import com.google.protobuf.Timestamp;
> import org.apache.beam.sdk.Pipeline;
> import org.apache.beam.sdk.coders.CannotProvideCoderException;
> import org.apache.beam.sdk.coders.Coder;
> import org.apache.beam.sdk.options.PipelineOptions;
> import org.apache.beam.sdk.options.PipelineOptionsFactory;
> public class CoderTest {
>   public static void main(String[] args) throws CannotProvideCoderException {
> PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create();
> Pipeline pipeline = Pipeline.create(options);
> Coder coder = 
> pipeline.getCoderRegistry().getCoder(Timestamp.class);
> // class org.apache.beam.sdk.coders.SerializableCoder
> System.out.println(coder.getClass());
>   }
> }
> {code}
> Right now we're sorting {{CoderProviderRegistrar}}s by canonical name but 
> {{SerializableCoderProvider}} should be added last as a fallback if there're 
> other {{CoderProvider}}s that support the same type.
> {code}
> Set registrars = 
> Sets.newTreeSet(ObjectsClassComparator.INSTANCE);
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)