Skip to content

Commit

Permalink
[SPARK-17350][SQL] Disable default use of KryoSerializer in Thrift Se…
Browse files Browse the repository at this point in the history
…rver

In SPARK-4761 / #3621 (December 2014) we enabled Kryo serialization by default in the Spark Thrift Server. However, I don't think that the original rationale for doing this still holds now that most Spark SQL serialization is now performed via encoders and our UnsafeRow format.

In addition, the use of Kryo as the default serializer can introduce performance problems because the creation of new KryoSerializer instances is expensive and we haven't performed instance-reuse optimizations in several code paths (including DirectTaskResult deserialization).

Given all of this, I propose to revert back to using JavaSerializer as the default serializer in the Thrift Server.

/cc liancheng

Author: Josh Rosen <joshrosen@databricks.com>

Closes #14906 from JoshRosen/disable-kryo-in-thriftserver.
  • Loading branch information
JoshRosen authored and rxin committed Nov 1, 2016
1 parent 01dd008 commit 6e62981
Show file tree
Hide file tree
Showing 2 changed files with 2 additions and 13 deletions.
5 changes: 2 additions & 3 deletions docs/configuration.md
Original file line number Diff line number Diff line change
Expand Up @@ -767,7 +767,7 @@ Apart from these, the following properties are also available, and may be useful
</tr>
<tr>
<td><code>spark.kryo.referenceTracking</code></td>
<td>true (false when using Spark SQL Thrift Server)</td>
<td>true</td>
<td>
Whether to track references to the same object when serializing data with Kryo, which is
necessary if your object graphs have loops and useful for efficiency if they contain multiple
Expand Down Expand Up @@ -838,8 +838,7 @@ Apart from these, the following properties are also available, and may be useful
<tr>
<td><code>spark.serializer</code></td>
<td>
org.apache.spark.serializer.<br />JavaSerializer (org.apache.spark.serializer.<br />
KryoSerializer when using Spark SQL Thrift Server)
org.apache.spark.serializer.<br />JavaSerializer
</td>
<td>
Class to use for serializing objects that will be sent over the network or need to be cached
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,6 @@ package org.apache.spark.sql.hive.thriftserver

import java.io.PrintStream

import scala.collection.JavaConverters._

import org.apache.spark.{SparkConf, SparkContext}
import org.apache.spark.internal.Logging
import org.apache.spark.sql.{SparkSession, SQLContext}
Expand All @@ -37,8 +35,6 @@ private[hive] object SparkSQLEnv extends Logging {
def init() {
if (sqlContext == null) {
val sparkConf = new SparkConf(loadDefaults = true)
val maybeSerializer = sparkConf.getOption("spark.serializer")
val maybeKryoReferenceTracking = sparkConf.getOption("spark.kryo.referenceTracking")
// If user doesn't specify the appName, we want to get [SparkSQL::localHostName] instead of
// the default appName [SparkSQLCLIDriver] in cli or beeline.
val maybeAppName = sparkConf
Expand All @@ -47,12 +43,6 @@ private[hive] object SparkSQLEnv extends Logging {

sparkConf
.setAppName(maybeAppName.getOrElse(s"SparkSQL::${Utils.localHostName()}"))
.set(
"spark.serializer",
maybeSerializer.getOrElse("org.apache.spark.serializer.KryoSerializer"))
.set(
"spark.kryo.referenceTracking",
maybeKryoReferenceTracking.getOrElse("false"))

val sparkSession = SparkSession.builder.config(sparkConf).enableHiveSupport().getOrCreate()
sparkContext = sparkSession.sparkContext
Expand Down

0 comments on commit 6e62981

Please sign in to comment.