Skip to content

Commit

Permalink
Add tests for serializer relocation property.
Browse files Browse the repository at this point in the history
I verified that the Kryo tests will fail if we remove the auto-reset
check in KryoSerializer. I also checked that this test fails if we
mistakenly enable this flag for JavaSerializer. This demonstrates that
the test case is actually capable of detecting the types of bugs that it's
trying to prevent.

Of course, it's possible that certain bugs will only surface when serializing
specific data types, so we'll still have to be cautious when overriding
`supportsRelocationOfSerializedObjects` for new serializers.
  • Loading branch information
JoshRosen committed May 5, 2015
1 parent 450fa21 commit 0ba75e6
Show file tree
Hide file tree
Showing 2 changed files with 106 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -127,7 +127,9 @@ class KryoSerializer(conf: SparkConf)
}

override def supportsRelocationOfSerializedObjects: Boolean = {
// TODO: we should have a citation / explanatory comment here clarifying _why_ this is the case
// If auto-flush is disabled, then Kryo may store references to duplicate occurrences of objects
// in the stream rather than writing those objects' serialized bytes, breaking relocation. See
// https://groups.google.com/d/msg/kryo-users/6ZUSyfjjtdo/FhGG1KHDXPgJ for more details.
newInstance().asInstanceOf[KryoSerializerInstance].getAutoReset()
}
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,103 @@
/*
* 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 scala.util.Random

import org.scalatest.FunSuite

import org.apache.spark.SparkConf
import org.apache.spark.serializer.KryoTest.RegistratorWithoutAutoReset

private case class MyCaseClass(foo: Int, bar: String)

class SerializerPropertiesSuite extends FunSuite {

test("JavaSerializer does not support relocation") {
testSupportsRelocationOfSerializedObjects(new JavaSerializer(new SparkConf()))
}

test("KryoSerializer supports relocation when auto-reset is enabled") {
val ser = new KryoSerializer(new SparkConf)
assert(ser.newInstance().asInstanceOf[KryoSerializerInstance].getAutoReset())
testSupportsRelocationOfSerializedObjects(ser)
}

test("KryoSerializer does not support relocation when auto-reset is disabled") {
val conf = new SparkConf().set("spark.kryo.registrator",
classOf[RegistratorWithoutAutoReset].getName)
val ser = new KryoSerializer(conf)
assert(!ser.newInstance().asInstanceOf[KryoSerializerInstance].getAutoReset())
testSupportsRelocationOfSerializedObjects(ser)
}

def testSupportsRelocationOfSerializedObjects(serializer: Serializer): Unit = {
val NUM_TRIALS = 100
if (!serializer.supportsRelocationOfSerializedObjects) {
return
}
val rand = new Random(42)
val randomFunctions: Seq[() => Any] = Seq(
() => rand.nextInt(),
() => rand.nextString(rand.nextInt(10)),
() => rand.nextDouble(),
() => rand.nextBoolean(),
() => (rand.nextInt(), rand.nextString(rand.nextInt(10))),
() => MyCaseClass(rand.nextInt(), rand.nextString(rand.nextInt(10))),
() => {
val x = MyCaseClass(rand.nextInt(), rand.nextString(rand.nextInt(10)))
(x, x)
}
)
def generateRandomItem(): Any = {
randomFunctions(rand.nextInt(randomFunctions.size)).apply()
}

for (_ <- 1 to NUM_TRIALS) {
val items = {
// Make sure that we have duplicate occurrences of the same object in the stream:
val randomItems = Seq.fill(10)(generateRandomItem())
randomItems ++ randomItems.take(5)
}
val baos = new ByteArrayOutputStream()
val serStream = serializer.newInstance().serializeStream(baos)
def serializeItem(item: Any): Array[Byte] = {
val itemStartOffset = baos.toByteArray.length
serStream.writeObject(item)
serStream.flush()
val itemEndOffset = baos.toByteArray.length
baos.toByteArray.slice(itemStartOffset, itemEndOffset).clone()
}
val itemsAndSerializedItems: Seq[(Any, Array[Byte])] = {
val serItems = items.map {
item => (item, serializeItem(item))
}
serStream.close()
rand.shuffle(serItems)
}
val reorderedSerializedData: Array[Byte] = itemsAndSerializedItems.flatMap(_._2).toArray
val deserializedItemsStream = serializer.newInstance().deserializeStream(
new ByteArrayInputStream(reorderedSerializedData))
assert(deserializedItemsStream.asIterator.toSeq === itemsAndSerializedItems.map(_._1))
deserializedItemsStream.close()
}
}

}

0 comments on commit 0ba75e6

Please sign in to comment.