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

    https://github.com/apache/spark/pull/7004#discussion_r33267940
  
    --- Diff: 
core/src/main/scala/org/apache/spark/serializer/GenericAvroSerializer.scala ---
    @@ -0,0 +1,185 @@
    +/*
    + * 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.spark.serializer
    +
    +import java.io.ByteArrayOutputStream
    +import java.util.zip.{Inflater, Deflater}
    +
    +import scala.collection.mutable
    +
    +import com.esotericsoftware.kryo.{Kryo, Serializer => KSerializer}
    +import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => 
KryoOutput}
    +import org.apache.avro.generic.{GenericData, GenericRecord}
    +import org.apache.avro.io._
    +import org.apache.avro.{Schema, SchemaNormalization}
    +
    +import org.apache.spark.SparkConf
    +
    +import GenericAvroSerializer._
    +
    +object GenericAvroSerializer {
    +  def avroSchemaKey(implicit fingerprint: Long): String = 
s"avro.schema.$fingerprint"
    +}
    +
    +/**
    + * Custom serializer used for generic Avro records. If the user registers 
the schemas
    + * ahead of time, then the schema's fingerprint will be sent with each 
message instead of the actual
    + * schema, as to reduce network IO.
    + * Actions like parsing or compressing schemas are computationally 
expensive so the serializer
    + * caches all previously seen values as to reduce the amount of work 
needed to do.
    + */
    +class GenericAvroSerializer(conf: SparkConf) extends 
KSerializer[GenericRecord] {
    +
    +  private val serializer = serialize()
    +  private val deserializer = deserialize()
    +
    +  private def confSchema(implicit fingerprint: Long) = 
conf.getOption(avroSchemaKey)
    +
    +  /**
    +   * Used to compress Schemas when they are being sent over the wire.
    +   * The compression results are memoized to reduce the compression time 
since the
    +   * same schema is compressed many times over
    +   */
    +  def compressor(): Schema => Array[Byte] = {
    +    val cache = new mutable.HashMap[Schema, Array[Byte]]()
    +
    --- End diff --
    
    I don't see the value in having methods that return functions --  
`serialize()` is only called once, which in turn will just call `compressor()` 
once.  (Or am I overlooking something?)
    
    How about just move this `cache` to the top level as `compressorCache`, and 
then move all the inner functions to the top level also?  It would also make it 
easier to test each function.  If you like, you could have separate helpers for 
the serialization & deserialization to keep things isolated (though imo just 
separate methods is fine).


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to