forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[SPARK-7311] Introduce internal Serializer API for determining if ser…
…ializers support object relocation This patch extends the `Serializer` interface with a new `Private` API which allows serializers to indicate whether they support relocation of serialized objects in serializer stream output. This relocatibilty property is described in more detail in `Serializer.scala`, but in a nutshell a serializer supports relocation if reordering the bytes of serialized objects in serialization stream output is equivalent to having re-ordered those elements prior to serializing them. The optimized shuffle path introduced in apache#4450 and apache#5868 both rely on serializers having this property; this patch just centralizes the logic for determining whether a serializer has this property. I also added tests and comments clarifying when this works for KryoSerializer. This change allows the optimizations in apache#4450 to be applied for shuffles that use `SqlSerializer2`. Author: Josh Rosen <[email protected]> Closes apache#5924 from JoshRosen/SPARK-7311 and squashes the following commits: 50a68ca [Josh Rosen] Address minor nits 0a7ebd7 [Josh Rosen] Clarify reason why SqlSerializer2 supports this serializer 123b992 [Josh Rosen] Cleanup for submitting as standalone patch. 4aa61b2 [Josh Rosen] Add missing newline 2c1233a [Josh Rosen] Small refactoring of SerializerPropertiesSuite to enable test re-use: 0ba75e6 [Josh Rosen] Add tests for serializer relocation property. 450fa21 [Josh Rosen] Back out accidental log4j.properties change 86d4dcd [Josh Rosen] Flag that SparkSqlSerializer2 supports relocation b9624ee [Josh Rosen] Expand serializer API and use new function to help control when new UnsafeShuffle path is used.
- Loading branch information
Showing
5 changed files
with
166 additions
and
3 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
119 changes: 119 additions & 0 deletions
119
core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,119 @@ | ||
/* | ||
* 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.{Assertions, FunSuite} | ||
|
||
import org.apache.spark.SparkConf | ||
import org.apache.spark.serializer.KryoTest.RegistratorWithoutAutoReset | ||
|
||
/** | ||
* Tests to ensure that [[Serializer]] implementations obey the API contracts for methods that | ||
* describe properties of the serialized stream, such as | ||
* [[Serializer.supportsRelocationOfSerializedObjects]]. | ||
*/ | ||
class SerializerPropertiesSuite extends FunSuite { | ||
|
||
import SerializerPropertiesSuite._ | ||
|
||
test("JavaSerializer does not support relocation") { | ||
// Per a comment on the SPARK-4550 JIRA ticket, Java serialization appears to write out the | ||
// full class name the first time an object is written to an output stream, but subsequent | ||
// references to the class write a more compact identifier; this prevents relocation. | ||
val ser = new JavaSerializer(new SparkConf()) | ||
testSupportsRelocationOfSerializedObjects(ser, generateRandomItem) | ||
} | ||
|
||
test("KryoSerializer supports relocation when auto-reset is enabled") { | ||
val ser = new KryoSerializer(new SparkConf) | ||
assert(ser.newInstance().asInstanceOf[KryoSerializerInstance].getAutoReset()) | ||
testSupportsRelocationOfSerializedObjects(ser, generateRandomItem) | ||
} | ||
|
||
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, generateRandomItem) | ||
} | ||
|
||
} | ||
|
||
object SerializerPropertiesSuite extends Assertions { | ||
|
||
def generateRandomItem(rand: Random): Any = { | ||
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) | ||
} | ||
) | ||
randomFunctions(rand.nextInt(randomFunctions.size)).apply() | ||
} | ||
|
||
def testSupportsRelocationOfSerializedObjects( | ||
serializer: Serializer, | ||
generateRandomItem: Random => Any): Unit = { | ||
if (!serializer.supportsRelocationOfSerializedObjects) { | ||
return | ||
} | ||
val NUM_TRIALS = 5 | ||
val rand = new Random(42) | ||
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(rand)) | ||
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() | ||
} | ||
} | ||
} | ||
|
||
private case class MyCaseClass(foo: Int, bar: String) |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters