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

    https://github.com/apache/spark/pull/7004#discussion_r35796975
  
    --- Diff: 
core/src/main/scala/org/apache/spark/serializer/GenericAvroSerializer.scala ---
    @@ -0,0 +1,140 @@
    +/*
    + * 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.{ByteArrayInputStream, ByteArrayOutputStream}
    +import java.nio.ByteBuffer
    +
    +import scala.collection.mutable
    +
    +import com.esotericsoftware.kryo.{Kryo, Serializer => KSerializer}
    +import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => 
KryoOutput}
    +import org.apache.avro.{Schema, SchemaNormalization}
    +import org.apache.avro.generic.{GenericData, GenericRecord}
    +import org.apache.avro.io._
    +import org.apache.commons.io.IOUtils
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.io.CompressionCodec
    +
    +/**
    + * 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.
    + */
    +private[serializer] class GenericAvroSerializer(schemas: Map[Long, String])
    +  extends KSerializer[GenericRecord] {
    +
    +  /** Used to reduce the amount of effort to compress the schema */
    +  private val compressCache = new mutable.HashMap[Schema, Array[Byte]]()
    +  private val decompressCache = new mutable.HashMap[ByteBuffer, Schema]()
    +
    +  /** Reuses the same datum reader/writer since the same schema will be 
used many times */
    +  private val writerCache = new mutable.HashMap[Schema, DatumWriter[_]]()
    +  private val readerCache = new mutable.HashMap[Schema, DatumReader[_]]()
    +
    +  /** Fingerprinting is very expensive so this alleviates most of the work 
*/
    +  private val fingerprintCache = new mutable.HashMap[Schema, Long]()
    +  private val schemaCache = new mutable.HashMap[Long, Schema]()
    +
    +  /** This needs to be lazy since SparkEnv is not initialized yet 
sometimes when this is called */
    +  private lazy val codec = CompressionCodec.createCodec(SparkEnv.get.conf)
    --- End diff --
    
    for anybody that is curious -- I had gotten myself pretty confused about 
why this change would make the `SparkConf` serialized.  `KryoSerializer` 
already had a `conf` argument to the constructor, that wasn't changed.  But the 
`conf` there is only accessed in field initialization, never in methods, so it 
wasn't stored.  But through the wonders of scala, when you access that `conf` 
in a method, suddenly `conf` also becomes a member variable, and now you can no 
longer serialize the `KryoSerializer`.
    
    In practice this means the `lazy val codec` here is fine in actual use, but 
it could be very confusing in a unit test where the `SparkEnv` hasn't been set. 
 So I'll add comment explaining this a bit.


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