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

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_r51127135
  
    --- 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 @abhishekagarwal87. I like that implementation when all the schemas 
are known in advance.  The use case I've been writing this for needs to support 
arbitrary schema evolution over the life of the topology, so I have not thought 
about it from that perspective.
    
    As to the default, I think the safest option is the one that doesn't 
require the user to do anything, but I would be interested in hearing other 
opinions on that as well.
    
    Thanks for the feedback everyone, I hope to have some code added to this PR 
soon.


> 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