beam-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Neville Li (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (BEAM-2658) SerializableCoder has higher precedence over ProtoCoder in CoderRegistry#getCoder
Date Sun, 23 Jul 2017 15:50:00 GMT

    [ https://issues.apache.org/jira/browse/BEAM-2658?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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<TypeDescriptor> 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<Integer, List<CoderProvider>>`. 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<Timestamp> 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<CoderProviderRegistrar> registrars = Sets.newTreeSet(ObjectsClassComparator.INSTANCE);
> {code}



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

Mime
View raw message