Skip to content

Commit

Permalink
[SPARK-21928][CORE] Set classloader on SerializerManager's private kryo
Browse files Browse the repository at this point in the history
## What changes were proposed in this pull request?

We have to make sure that SerializerManager's private instance of
kryo also uses the right classloader, regardless of the current thread
classloader.  In particular, this fixes serde during remote cache
fetches, as those occur in netty threads.

## How was this patch tested?

Manual tests & existing suite via jenkins.  I haven't been able to reproduce this is in a unit test, because when a remote RDD partition can be fetched, there is a warning message and then the partition is just recomputed locally.  I manually verified the warning message is no longer present.

Author: Imran Rashid <irashid@cloudera.com>

Closes #19280 from squito/SPARK-21928_ser_classloader.
  • Loading branch information
squito authored and Marcelo Vanzin committed Sep 21, 2017
1 parent f10cbf1 commit b75bd17
Show file tree
Hide file tree
Showing 3 changed files with 9 additions and 1 deletion.
3 changes: 3 additions & 0 deletions core/src/main/scala/org/apache/spark/executor/Executor.scala
Original file line number Diff line number Diff line change
Expand Up @@ -131,6 +131,9 @@ private[spark] class Executor(

// Set the classloader for serializer
env.serializer.setDefaultClassLoader(replClassLoader)
// SPARK-21928. SerializerManager's internal instance of Kryo might get used in netty threads
// for fetching remote cached RDD blocks, so need to make sure it uses the right classloader too.
env.serializerManager.setDefaultClassLoader(replClassLoader)

// Max size of direct result. If task result is bigger than this, we use the block manager
// to send the result back.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,10 @@ private[spark] class SerializerManager(

private[this] val kryoSerializer = new KryoSerializer(conf)

def setDefaultClassLoader(classLoader: ClassLoader): Unit = {
kryoSerializer.setDefaultClassLoader(classLoader)
}

private[this] val stringClassTag: ClassTag[String] = implicitly[ClassTag[String]]
private[this] val primitiveAndPrimitiveArrayClassTags: Set[ClassTag[_]] = {
val primitiveClassTags = Set[ClassTag[_]](
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ import org.apache.spark.metrics.MetricsSystem
import org.apache.spark.rdd.RDD
import org.apache.spark.rpc.RpcEnv
import org.apache.spark.scheduler.{FakeTask, ResultTask, TaskDescription}
import org.apache.spark.serializer.JavaSerializer
import org.apache.spark.serializer.{JavaSerializer, SerializerManager}
import org.apache.spark.shuffle.FetchFailedException
import org.apache.spark.storage.BlockManagerId
import org.apache.spark.util.UninterruptibleThread
Expand Down Expand Up @@ -234,6 +234,7 @@ class ExecutorSuite extends SparkFunSuite with LocalSparkContext with MockitoSug
val mockMemoryManager = mock[MemoryManager]
when(mockEnv.conf).thenReturn(conf)
when(mockEnv.serializer).thenReturn(serializer)
when(mockEnv.serializerManager).thenReturn(mock[SerializerManager])
when(mockEnv.rpcEnv).thenReturn(mockRpcEnv)
when(mockEnv.metricsSystem).thenReturn(mockMetricsSystem)
when(mockEnv.memoryManager).thenReturn(mockMemoryManager)
Expand Down

0 comments on commit b75bd17

Please sign in to comment.