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

ASF GitHub Bot commented on STORM-1504:
---------------------------------------

Github user dossett commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1052#discussion_r51063276
  
    --- Diff: 
external/storm-hdfs/src/main/java/org/apache/storm/hdfs/common/AvroGenericSerializer.java
 ---
    @@ -0,0 +1,70 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.storm.hdfs.common;
    +
    +import com.esotericsoftware.kryo.Kryo;
    +import com.esotericsoftware.kryo.Serializer;
    +import com.esotericsoftware.kryo.io.Input;
    +import com.esotericsoftware.kryo.io.Output;
    +import org.apache.avro.Schema;
    +import org.apache.avro.generic.GenericContainer;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericDatumWriter;
    +import org.apache.avro.io.BinaryEncoder;
    +import org.apache.avro.io.Decoder;
    +import org.apache.avro.io.DecoderFactory;
    +import org.apache.avro.io.EncoderFactory;
    +
    +import java.io.IOException;
    +
    +//Generously adapted from:
    
+//https://github.com/kijiproject/kiji-express/blob/master/kiji-express/src/main/scala/org/kiji/express/flow/framework/serialization/AvroSerializer.scala
    +//Which has as an ASL2.0 license
    +public class AvroGenericSerializer extends Serializer<GenericContainer> {
    +    @Override
    +    public void write(Kryo kryo, Output output, GenericContainer record) {
    +        output.writeString(record.getSchema().toString());
    +        GenericDatumWriter<GenericContainer> writer = new 
GenericDatumWriter<>(record.getSchema());
    +
    +        BinaryEncoder encoder = EncoderFactory
    +                .get()
    +                .directBinaryEncoder(output, null);
    +        try {
    +            writer.write(record, encoder);
    +        } catch (IOException e) {
    +            throw new RuntimeException(e);
    +        }
    +    }
    +
    +    @Override
    +    public GenericContainer read(Kryo kryo, Input input, 
Class<GenericContainer> aClass) {
    +        Schema theSchema = new Schema.Parser().parse(input.readString());
    --- End diff --
    
    Thanks for the thoughtful comments @revans2. I think I understand what 
you're describing but want to make sure.
    
    - A generic registry would just treat the schema as the key and vice versa, 
so it's always passed around.
    
    ```
    class GenericAvroSchemaRegistry implements AvroSchemaRegistry {
       public String getKey(Schema schema) {
          return schema.toString();
       }
    
        public Schema getSchema(String key) {
           return new Schema.Parser().parse(key);
       }
    ```
    
    - A jar-based registry approach could be used for schemas that are known in 
advance and worth  persisting across the entire topology, but fall back on the 
generic approach above if an unknown schema is used.
    - You might choose to implement your own own which relies on your 
proprietary registry, which is truly external to Storm
    - At Target we could implement our own since we also use an external schema 
registry, albeit one that is already open sourced.
    
    Do I have the gist of it?


> AvroGenericRecordBolt will not work in a topology with multiple workers
> -----------------------------------------------------------------------
>
>                 Key: STORM-1504
>                 URL: https://issues.apache.org/jira/browse/STORM-1504
>             Project: Apache Storm
>          Issue Type: Bug
>            Reporter: Aaron Dossett
>            Assignee: Aaron Dossett
>            Priority: Critical
>
> It receives Avro GenericRecord objects, which leads to serialization errors 
> in a multi-worker topology.  Specifically, GenericData.Record will throw a 
> serialization error.
> A serializer for this class should be added and registered.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to