Skip to content

Commit

Permalink
[SQL] Add a custom serializer for maps since they do not have a no-ar…
Browse files Browse the repository at this point in the history
…g constructor.

Author: Michael Armbrust <[email protected]>

Closes #243 from marmbrus/mapSer and squashes the following commits:

54045f7 [Michael Armbrust] Add a custom serializer for maps since they do not have a no-arg constructor.
  • Loading branch information
marmbrus authored and pwendell committed Mar 27, 2014
1 parent 32cbdfd commit e15e574
Showing 1 changed file with 18 additions and 0 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) {
kryo.setRegistrationRequired(false)
kryo.register(classOf[MutablePair[_, _]])
kryo.register(classOf[Array[Any]])
kryo.register(classOf[scala.collection.immutable.Map$Map1], new MapSerializer)
kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericRow])
kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericMutableRow])
kryo.register(classOf[scala.collection.mutable.ArrayBuffer[_]])
Expand Down Expand Up @@ -70,3 +71,20 @@ class BigDecimalSerializer extends Serializer[BigDecimal] {
BigDecimal(input.readString())
}
}

/**
* Maps do not have a no arg constructor and so cannot be serialized by default. So, we serialize
* them as `Array[(k,v)]`.
*/
class MapSerializer extends Serializer[Map[_,_]] {
def write(kryo: Kryo, output: Output, map: Map[_,_]) {
kryo.writeObject(output, map.flatMap(e => Seq(e._1, e._2)).toArray)
}

def read(kryo: Kryo, input: Input, tpe: Class[Map[_,_]]): Map[_,_] = {
kryo.readObject(input, classOf[Array[Any]])
.sliding(2,2)
.map { case Array(k,v) => (k,v) }
.toMap
}
}

0 comments on commit e15e574

Please sign in to comment.