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

    https://github.com/apache/storm/pull/1052#discussion_r51045767
  
    --- 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 --
    
    The problem here is that the schema is large (JSON) and the data is small 
(no tags which is why you need the schema).  If we send the schema each time we 
send the data we are wasting a lot of resources.  The only real way around this 
is to have a cheap way to get the schema while sending minimal data, ideally 
also caching the parsed schema so we don't have to parse it each time.
    
    The best way I know of would be to have an external schema registry and 
send a reference to the original schema with the data. That is what we do at 
Yahoo, but the registry is unlikely to ever be open sourced (sorry about that 
not my code).
    
    My proposal would be to create a SchemaRegistry API that lets you do 
something like.
    
    ```
    public interface AvroSchemaRegistry {
        public String getKey(Schema schema);
    
        public Schema getSchema(String key);
    }
    ```
    
    We could then have two default implementations, or even a hybrid one.  The 
generic one that would work each time would do exactly what this code does.  
Turns the schema into a string and the deserializes it on the other side.  You 
could do some caching if you wanted to.
    
    A cheaper version would not serialize/deserialize it each time, but go off 
of a checksum like avro RPC does. It could then look to see if a special schema 
file exists in the topology jar that matches the checksum, or possibly full 
name + version and it computes the checksum.  From that it could send the 
checksum, or fullname + version instead of the full schema. The read code would 
read the checksum/version lookup the file read/parse/cache the schema and 
return it.
    
    A hybrid would first try the cached copy, then the checksum/jar and if it 
could not find it fall back to the full/slow schema.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to