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

    https://github.com/apache/storm/pull/1052#discussion_r51025554
  
    --- 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 --
    
    This can lead to higher GC depending on how frequently it is called.
    
https://github.com/apache/avro/blob/trunk/lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumReader.java#L107
    GenericDataumReader internally keeps a map of <ActualSchema, 
ExpectedSchema, ResolvingDecoder>. The resolving decoder objects are expensive 
to create. Map being used is an IdentityHashMap. So even though schema may be 
same, a different object will lead to creation of different ResolvingDecoder. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

Reply via email to