[ 
https://issues.apache.org/jira/browse/SPARK-12366?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15062203#comment-15062203
 ] 

Yin Huai commented on SPARK-12366:
----------------------------------

Based on [~lian cheng]'s investigation, seems it is introduced by 
https://github.com/apache/spark/pull/9923. cc [~vanzin]

> IllegalArgumentException: requirement failed: File not found: 
> ...sql/catalyst/expressions/GeneratedClass.class when df.show
> ---------------------------------------------------------------------------------------------------------------------------
>
>                 Key: SPARK-12366
>                 URL: https://issues.apache.org/jira/browse/SPARK-12366
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 1.6.0
>            Reporter: Jacek Laskowski
>
> When executing {{df.show}} with a dataframe from JSON {{spark-shell}} printed 
> out *loads* of {{java.lang.IllegalArgumentException}}. It was fine the second 
> time I called {{df.show}}. See below.
> {code}
> ➜  spark git:(master) ✗ ./bin/spark-shell
> Setting default log level to "WARN".
> To adjust logging level use sc.setLogLevel(newLevel).
> WARN NativeCodeLoader: Unable to load native-hadoop library for your 
> platform... using builtin-java classes where applicable
> Spark context available as sc (master = local[*], app id = 
> local-1450265501090).
> WARN ObjectStore: Version information not found in metastore. 
> hive.metastore.schema.verification is not enabled so recording the schema 
> version 1.2.0
> WARN ObjectStore: Failed to get database default, returning 
> NoSuchObjectException
> SQL context available as sqlContext.
> Welcome to
>       ____              __
>      / __/__  ___ _____/ /__
>     _\ \/ _ \/ _ `/ __/  '_/
>    /___/ .__/\_,_/_/ /_/\_\   version 1.6.0-SNAPSHOT
>       /_/
> Using Scala version 2.11.7 (Java HotSpot(TM) 64-Bit Server VM, Java 1.8.0_66)
> Type in expressions to have them evaluated.
> Type :help for more information.
> scala> val df = 
> sqlContext.read.json("examples/src/main/resources/people.json")
> df: org.apache.spark.sql.DataFrame = [age: bigint, name: string]
> scala> df.show
> ERROR TransportRequestHandler: Error opening stream 
> /classes/org/apache/spark/sql/catalyst/expressions/GeneratedClass.class for 
> request from /172.20.4.141:55136
> java.lang.IllegalArgumentException: requirement failed: File not found: 
> /classes/org/apache/spark/sql/catalyst/expressions/GeneratedClass.class
>       at scala.Predef$.require(Predef.scala:219)
>       at 
> org.apache.spark.rpc.netty.NettyStreamManager.openStream(NettyStreamManager.scala:60)
>       at 
> org.apache.spark.network.server.TransportRequestHandler.processStreamRequest(TransportRequestHandler.java:136)
>       at 
> org.apache.spark.network.server.TransportRequestHandler.handle(TransportRequestHandler.java:106)
>       at 
> org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:104)
>       at 
> org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:51)
>       at 
> io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:266)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> org.apache.spark.network.util.TransportFrameDecoder.channelRead(TransportFrameDecoder.java:86)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:846)
>       at 
> io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
>       at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
>       at 
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
>       at java.lang.Thread.run(Thread.java:745)
> ERROR NettyRpcEnv: Error downloading stream 
> /classes/org/apache/spark/sql/catalyst/expressions/GeneratedClass.class.
> java.lang.RuntimeException: java.lang.IllegalArgumentException: requirement 
> failed: File not found: 
> /classes/org/apache/spark/sql/catalyst/expressions/GeneratedClass.class
>       at scala.Predef$.require(Predef.scala:219)
>       at 
> org.apache.spark.rpc.netty.NettyStreamManager.openStream(NettyStreamManager.scala:60)
>       at 
> org.apache.spark.network.server.TransportRequestHandler.processStreamRequest(TransportRequestHandler.java:136)
>       at 
> org.apache.spark.network.server.TransportRequestHandler.handle(TransportRequestHandler.java:106)
>       at 
> org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:104)
>       at 
> org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:51)
>       at 
> io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:266)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> org.apache.spark.network.util.TransportFrameDecoder.channelRead(TransportFrameDecoder.java:86)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:846)
>       at 
> io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
>       at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
>       at 
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
>       at java.lang.Thread.run(Thread.java:745)
>       at 
> org.apache.spark.network.client.TransportResponseHandler.handle(TransportResponseHandler.java:219)
>       at 
> org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:106)
>       at 
> org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:51)
>       at 
> io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:266)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> org.apache.spark.network.util.TransportFrameDecoder.channelRead(TransportFrameDecoder.java:86)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:846)
>       at 
> io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
>       at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
>       at 
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
>       at java.lang.Thread.run(Thread.java:745)
> ERROR ExecutorClassLoader: Failed to check existence of class 
> org.apache.spark.sql.catalyst.expressions.GeneratedClass on REPL class server 
> at spark://172.20.4.141:55131/classes
> java.nio.channels.AsynchronousCloseException
>       at 
> java.nio.channels.spi.AbstractInterruptibleChannel.end(AbstractInterruptibleChannel.java:205)
>       at sun.nio.ch.SourceChannelImpl.read(SourceChannelImpl.java:172)
>       at 
> org.apache.spark.rpc.netty.NettyRpcEnv$FileDownloadChannel$$anonfun$1.apply$mcI$sp(NettyRpcEnv.scala:367)
>       at 
> org.apache.spark.rpc.netty.NettyRpcEnv$FileDownloadChannel$$anonfun$1.apply(NettyRpcEnv.scala:367)
>       at 
> org.apache.spark.rpc.netty.NettyRpcEnv$FileDownloadChannel$$anonfun$1.apply(NettyRpcEnv.scala:367)
>       at scala.util.Try$.apply(Try.scala:192)
>       at 
> org.apache.spark.rpc.netty.NettyRpcEnv$FileDownloadChannel.read(NettyRpcEnv.scala:367)
>       at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:65)
>       at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:109)
>       at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:103)
>       at java.io.InputStream.read(InputStream.java:101)
>       at 
> org.apache.spark.repl.ExecutorClassLoader.readAndTransformClass(ExecutorClassLoader.scala:194)
>       at 
> org.apache.spark.repl.ExecutorClassLoader.findClassLocally(ExecutorClassLoader.scala:153)
>       at 
> org.apache.spark.repl.ExecutorClassLoader.findClass(ExecutorClassLoader.scala:80)
>       at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
>       at java.lang.ClassLoader.loadClass(ClassLoader.java:411)
>       at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
>       at 
> org.codehaus.janino.ClassBodyEvaluator.compileToClass(ClassBodyEvaluator.java:319)
>       at 
> org.codehaus.janino.ClassBodyEvaluator.cook(ClassBodyEvaluator.java:233)
>       at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:192)
>       at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:84)
>       at 
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.org$apache$spark$sql$catalyst$expressions$codegen$CodeGenerator$$doCompile(CodeGenerator.scala:550)
>       at 
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:575)
>       at 
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:572)
>       at 
> com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3599)
>       at 
> com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2379)
>       at 
> com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2342)
>       at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2257)
>       at com.google.common.cache.LocalCache.get(LocalCache.java:4000)
>       at com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:4004)
>       at 
> com.google.common.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4874)
>       at 
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.compile(CodeGenerator.scala:515)
>       at 
> org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:358)
>       at 
> org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:317)
>       at 
> org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:32)
>       at 
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.generate(CodeGenerator.scala:588)
>       at 
> org.apache.spark.sql.catalyst.expressions.UnsafeProjection$.create(Projection.scala:125)
>       at 
> org.apache.spark.sql.catalyst.expressions.UnsafeProjection$.create(Projection.scala:114)
>       at 
> org.apache.spark.sql.catalyst.expressions.UnsafeProjection$.create(Projection.scala:108)
>       at 
> org.apache.spark.sql.execution.datasources.json.JSONRelation$$anonfun$buildInternalScan$1.apply(JSONRelation.scala:137)
>       at 
> org.apache.spark.sql.execution.datasources.json.JSONRelation$$anonfun$buildInternalScan$1.apply(JSONRelation.scala:136)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>       at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
>       at org.apache.spark.scheduler.Task.run(Task.scala:88)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:213)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> ERROR TransportRequestHandler: Error opening stream 
> /classes/org/apache/spark/sql/catalyst/expressions/GeneratedClass$SpecificUnsafeProjection.class
>  for request from /172.20.4.141:55136
> java.lang.IllegalArgumentException: requirement failed: File not found: 
> /classes/org/apache/spark/sql/catalyst/expressions/GeneratedClass$SpecificUnsafeProjection.class
>       at scala.Predef$.require(Predef.scala:219)
>       at 
> org.apache.spark.rpc.netty.NettyStreamManager.openStream(NettyStreamManager.scala:60)
>       at 
> org.apache.spark.network.server.TransportRequestHandler.processStreamRequest(TransportRequestHandler.java:136)
>       at 
> org.apache.spark.network.server.TransportRequestHandler.handle(TransportRequestHandler.java:106)
>       at 
> org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:104)
>       at 
> org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:51)
>       at 
> io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:266)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> org.apache.spark.network.util.TransportFrameDecoder.channelRead(TransportFrameDecoder.java:86)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:846)
>       at 
> io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
>       at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
>       at 
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
>       at java.lang.Thread.run(Thread.java:745)
> ERROR NettyRpcEnv: Error downloading stream 
> /classes/org/apache/spark/sql/catalyst/expressions/GeneratedClass$SpecificUnsafeProjection.class.
> java.lang.RuntimeException: java.lang.IllegalArgumentException: requirement 
> failed: File not found: 
> /classes/org/apache/spark/sql/catalyst/expressions/GeneratedClass$SpecificUnsafeProjection.class
>       at scala.Predef$.require(Predef.scala:219)
>       at 
> org.apache.spark.rpc.netty.NettyStreamManager.openStream(NettyStreamManager.scala:60)
>       at 
> org.apache.spark.network.server.TransportRequestHandler.processStreamRequest(TransportRequestHandler.java:136)
>       at 
> org.apache.spark.network.server.TransportRequestHandler.handle(TransportRequestHandler.java:106)
>       at 
> org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:104)
>       at 
> org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:51)
>       at 
> io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:266)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> org.apache.spark.network.util.TransportFrameDecoder.channelRead(TransportFrameDecoder.java:86)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:846)
>       at 
> io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
>       at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
>       at 
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
>       at java.lang.Thread.run(Thread.java:745)
>       at 
> org.apache.spark.network.client.TransportResponseHandler.handle(TransportResponseHandler.java:219)
>       at 
> org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:106)
>       at 
> org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:51)
>       at 
> io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:266)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> org.apache.spark.network.util.TransportFrameDecoder.channelRead(TransportFrameDecoder.java:86)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:846)
>       at 
> io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
>       at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
>       at 
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
>       at java.lang.Thread.run(Thread.java:745)
> ERROR ExecutorClassLoader: Failed to check existence of class 
> org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection
>  on REPL class server at spark://172.20.4.141:55131/classes
> java.nio.channels.AsynchronousCloseException
>       at 
> java.nio.channels.spi.AbstractInterruptibleChannel.end(AbstractInterruptibleChannel.java:205)
>       at sun.nio.ch.SourceChannelImpl.read(SourceChannelImpl.java:172)
>       at 
> org.apache.spark.rpc.netty.NettyRpcEnv$FileDownloadChannel$$anonfun$1.apply$mcI$sp(NettyRpcEnv.scala:367)
>       at 
> org.apache.spark.rpc.netty.NettyRpcEnv$FileDownloadChannel$$anonfun$1.apply(NettyRpcEnv.scala:367)
>       at 
> org.apache.spark.rpc.netty.NettyRpcEnv$FileDownloadChannel$$anonfun$1.apply(NettyRpcEnv.scala:367)
>       at scala.util.Try$.apply(Try.scala:192)
>       at 
> org.apache.spark.rpc.netty.NettyRpcEnv$FileDownloadChannel.read(NettyRpcEnv.scala:367)
>       at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:65)
>       at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:109)
>       at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:103)
>       at java.io.InputStream.read(InputStream.java:101)
>       at 
> org.apache.spark.repl.ExecutorClassLoader.readAndTransformClass(ExecutorClassLoader.scala:194)
>       at 
> org.apache.spark.repl.ExecutorClassLoader.findClassLocally(ExecutorClassLoader.scala:153)
>       at 
> org.apache.spark.repl.ExecutorClassLoader.findClass(ExecutorClassLoader.scala:80)
>       at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
>       at java.lang.ClassLoader.loadClass(ClassLoader.java:411)
>       at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
>       at 
> org.apache.spark.sql.catalyst.expressions.GeneratedClass.generate(Unknown 
> Source)
>       at 
> org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:359)
>       at 
> org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:317)
>       at 
> org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:32)
>       at 
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.generate(CodeGenerator.scala:588)
>       at 
> org.apache.spark.sql.catalyst.expressions.UnsafeProjection$.create(Projection.scala:125)
>       at 
> org.apache.spark.sql.catalyst.expressions.UnsafeProjection$.create(Projection.scala:114)
>       at 
> org.apache.spark.sql.catalyst.expressions.UnsafeProjection$.create(Projection.scala:108)
>       at 
> org.apache.spark.sql.execution.datasources.json.JSONRelation$$anonfun$buildInternalScan$1.apply(JSONRelation.scala:137)
>       at 
> org.apache.spark.sql.execution.datasources.json.JSONRelation$$anonfun$buildInternalScan$1.apply(JSONRelation.scala:136)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>       at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
>       at org.apache.spark.scheduler.Task.run(Task.scala:88)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:213)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> ERROR TransportRequestHandler: Error opening stream 
> /classes/org/apache/spark/sql/catalyst/expressions/GeneratedClass.class for 
> request from /172.20.4.141:55136
> java.lang.IllegalArgumentException: requirement failed: File not found: 
> /classes/org/apache/spark/sql/catalyst/expressions/GeneratedClass.class
>       at scala.Predef$.require(Predef.scala:219)
>       at 
> org.apache.spark.rpc.netty.NettyStreamManager.openStream(NettyStreamManager.scala:60)
>       at 
> org.apache.spark.network.server.TransportRequestHandler.processStreamRequest(TransportRequestHandler.java:136)
>       at 
> org.apache.spark.network.server.TransportRequestHandler.handle(TransportRequestHandler.java:106)
>       at 
> org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:104)
>       at 
> org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:51)
>       at 
> io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:266)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> org.apache.spark.network.util.TransportFrameDecoder.channelRead(TransportFrameDecoder.java:86)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:846)
>       at 
> io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
>       at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
>       at 
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
>       at java.lang.Thread.run(Thread.java:745)
> ERROR NettyRpcEnv: Error downloading stream 
> /classes/org/apache/spark/sql/catalyst/expressions/GeneratedClass.class.
> java.lang.RuntimeException: java.lang.IllegalArgumentException: requirement 
> failed: File not found: 
> /classes/org/apache/spark/sql/catalyst/expressions/GeneratedClass.class
>       at scala.Predef$.require(Predef.scala:219)
>       at 
> org.apache.spark.rpc.netty.NettyStreamManager.openStream(NettyStreamManager.scala:60)
>       at 
> org.apache.spark.network.server.TransportRequestHandler.processStreamRequest(TransportRequestHandler.java:136)
>       at 
> org.apache.spark.network.server.TransportRequestHandler.handle(TransportRequestHandler.java:106)
>       at 
> org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:104)
>       at 
> org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:51)
>       at 
> io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:266)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> org.apache.spark.network.util.TransportFrameDecoder.channelRead(TransportFrameDecoder.java:86)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:846)
>       at 
> io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
>       at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
>       at 
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
>       at java.lang.Thread.run(Thread.java:745)
>       at 
> org.apache.spark.network.client.TransportResponseHandler.handle(TransportResponseHandler.java:219)
>       at 
> org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:106)
>       at 
> org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:51)
>       at 
> io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:266)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> org.apache.spark.network.util.TransportFrameDecoder.channelRead(TransportFrameDecoder.java:86)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:846)
>       at 
> io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
>       at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
>       at 
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
>       at java.lang.Thread.run(Thread.java:745)
> ERROR ExecutorClassLoader: Failed to check existence of class 
> org.apache.spark.sql.catalyst.expressions.GeneratedClass on REPL class server 
> at spark://172.20.4.141:55131/classes
> java.nio.channels.AsynchronousCloseException
>       at 
> java.nio.channels.spi.AbstractInterruptibleChannel.end(AbstractInterruptibleChannel.java:205)
>       at sun.nio.ch.SourceChannelImpl.read(SourceChannelImpl.java:172)
>       at 
> org.apache.spark.rpc.netty.NettyRpcEnv$FileDownloadChannel$$anonfun$1.apply$mcI$sp(NettyRpcEnv.scala:367)
>       at 
> org.apache.spark.rpc.netty.NettyRpcEnv$FileDownloadChannel$$anonfun$1.apply(NettyRpcEnv.scala:367)
>       at 
> org.apache.spark.rpc.netty.NettyRpcEnv$FileDownloadChannel$$anonfun$1.apply(NettyRpcEnv.scala:367)
>       at scala.util.Try$.apply(Try.scala:192)
>       at 
> org.apache.spark.rpc.netty.NettyRpcEnv$FileDownloadChannel.read(NettyRpcEnv.scala:367)
>       at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:65)
>       at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:109)
>       at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:103)
>       at java.io.InputStream.read(InputStream.java:101)
>       at 
> org.apache.spark.repl.ExecutorClassLoader.readAndTransformClass(ExecutorClassLoader.scala:194)
>       at 
> org.apache.spark.repl.ExecutorClassLoader.findClassLocally(ExecutorClassLoader.scala:153)
>       at 
> org.apache.spark.repl.ExecutorClassLoader.findClass(ExecutorClassLoader.scala:80)
>       at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
>       at java.lang.ClassLoader.loadClass(ClassLoader.java:411)
>       at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
>       at 
> org.codehaus.janino.ClassBodyEvaluator.compileToClass(ClassBodyEvaluator.java:319)
>       at 
> org.codehaus.janino.ClassBodyEvaluator.cook(ClassBodyEvaluator.java:233)
>       at org.codehaus.janino.SimpleCompiler.cook(SimpleCompiler.java:192)
>       at org.codehaus.commons.compiler.Cookable.cook(Cookable.java:84)
>       at 
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.org$apache$spark$sql$catalyst$expressions$codegen$CodeGenerator$$doCompile(CodeGenerator.scala:550)
>       at 
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:575)
>       at 
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:572)
>       at 
> com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3599)
>       at 
> com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2379)
>       at 
> com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2342)
>       at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2257)
>       at com.google.common.cache.LocalCache.get(LocalCache.java:4000)
>       at com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:4004)
>       at 
> com.google.common.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4874)
>       at 
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.compile(CodeGenerator.scala:515)
>       at 
> org.apache.spark.sql.catalyst.expressions.codegen.GenerateSafeProjection$.create(GenerateSafeProjection.scala:178)
>       at 
> org.apache.spark.sql.catalyst.expressions.codegen.GenerateSafeProjection$.create(GenerateSafeProjection.scala:30)
>       at 
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.generate(CodeGenerator.scala:588)
>       at 
> org.apache.spark.sql.catalyst.expressions.FromUnsafeProjection$.create(Projection.scala:180)
>       at 
> org.apache.spark.sql.catalyst.expressions.FromUnsafeProjection$.apply(Projection.scala:171)
>       at 
> org.apache.spark.sql.execution.ConvertToSafe$$anonfun$2.apply(rowFormatConverters.scala:57)
>       at 
> org.apache.spark.sql.execution.ConvertToSafe$$anonfun$2.apply(rowFormatConverters.scala:56)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>       at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
>       at org.apache.spark.scheduler.Task.run(Task.scala:88)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:213)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> ERROR TransportRequestHandler: Error opening stream 
> /classes/org/apache/spark/sql/catalyst/expressions/GeneratedClass$SpecificSafeProjection.class
>  for request from /172.20.4.141:55136
> java.lang.IllegalArgumentException: requirement failed: File not found: 
> /classes/org/apache/spark/sql/catalyst/expressions/GeneratedClass$SpecificSafeProjection.class
>       at scala.Predef$.require(Predef.scala:219)
>       at 
> org.apache.spark.rpc.netty.NettyStreamManager.openStream(NettyStreamManager.scala:60)
>       at 
> org.apache.spark.network.server.TransportRequestHandler.processStreamRequest(TransportRequestHandler.java:136)
>       at 
> org.apache.spark.network.server.TransportRequestHandler.handle(TransportRequestHandler.java:106)
>       at 
> org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:104)
>       at 
> org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:51)
>       at 
> io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:266)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> org.apache.spark.network.util.TransportFrameDecoder.channelRead(TransportFrameDecoder.java:86)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:846)
>       at 
> io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
>       at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
>       at 
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
>       at java.lang.Thread.run(Thread.java:745)
> ERROR NettyRpcEnv: Error downloading stream 
> /classes/org/apache/spark/sql/catalyst/expressions/GeneratedClass$SpecificSafeProjection.class.
> java.lang.RuntimeException: java.lang.IllegalArgumentException: requirement 
> failed: File not found: 
> /classes/org/apache/spark/sql/catalyst/expressions/GeneratedClass$SpecificSafeProjection.class
>       at scala.Predef$.require(Predef.scala:219)
>       at 
> org.apache.spark.rpc.netty.NettyStreamManager.openStream(NettyStreamManager.scala:60)
>       at 
> org.apache.spark.network.server.TransportRequestHandler.processStreamRequest(TransportRequestHandler.java:136)
>       at 
> org.apache.spark.network.server.TransportRequestHandler.handle(TransportRequestHandler.java:106)
>       at 
> org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:104)
>       at 
> org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:51)
>       at 
> io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:266)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> org.apache.spark.network.util.TransportFrameDecoder.channelRead(TransportFrameDecoder.java:86)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:846)
>       at 
> io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
>       at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
>       at 
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
>       at java.lang.Thread.run(Thread.java:745)
>       at 
> org.apache.spark.network.client.TransportResponseHandler.handle(TransportResponseHandler.java:219)
>       at 
> org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:106)
>       at 
> org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:51)
>       at 
> io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:266)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> org.apache.spark.network.util.TransportFrameDecoder.channelRead(TransportFrameDecoder.java:86)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>       at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>       at 
> io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:846)
>       at 
> io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
>       at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
>       at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
>       at 
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
>       at java.lang.Thread.run(Thread.java:745)
> ERROR ExecutorClassLoader: Failed to check existence of class 
> org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificSafeProjection
>  on REPL class server at spark://172.20.4.141:55131/classes
> java.nio.channels.AsynchronousCloseException
>       at 
> java.nio.channels.spi.AbstractInterruptibleChannel.end(AbstractInterruptibleChannel.java:205)
>       at sun.nio.ch.SourceChannelImpl.read(SourceChannelImpl.java:172)
>       at 
> org.apache.spark.rpc.netty.NettyRpcEnv$FileDownloadChannel$$anonfun$1.apply$mcI$sp(NettyRpcEnv.scala:367)
>       at 
> org.apache.spark.rpc.netty.NettyRpcEnv$FileDownloadChannel$$anonfun$1.apply(NettyRpcEnv.scala:367)
>       at 
> org.apache.spark.rpc.netty.NettyRpcEnv$FileDownloadChannel$$anonfun$1.apply(NettyRpcEnv.scala:367)
>       at scala.util.Try$.apply(Try.scala:192)
>       at 
> org.apache.spark.rpc.netty.NettyRpcEnv$FileDownloadChannel.read(NettyRpcEnv.scala:367)
>       at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:65)
>       at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:109)
>       at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:103)
>       at java.io.InputStream.read(InputStream.java:101)
>       at 
> org.apache.spark.repl.ExecutorClassLoader.readAndTransformClass(ExecutorClassLoader.scala:194)
>       at 
> org.apache.spark.repl.ExecutorClassLoader.findClassLocally(ExecutorClassLoader.scala:153)
>       at 
> org.apache.spark.repl.ExecutorClassLoader.findClass(ExecutorClassLoader.scala:80)
>       at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
>       at java.lang.ClassLoader.loadClass(ClassLoader.java:411)
>       at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
>       at 
> org.apache.spark.sql.catalyst.expressions.GeneratedClass.generate(Unknown 
> Source)
>       at 
> org.apache.spark.sql.catalyst.expressions.codegen.GenerateSafeProjection$.create(GenerateSafeProjection.scala:179)
>       at 
> org.apache.spark.sql.catalyst.expressions.codegen.GenerateSafeProjection$.create(GenerateSafeProjection.scala:30)
>       at 
> org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.generate(CodeGenerator.scala:588)
>       at 
> org.apache.spark.sql.catalyst.expressions.FromUnsafeProjection$.create(Projection.scala:180)
>       at 
> org.apache.spark.sql.catalyst.expressions.FromUnsafeProjection$.apply(Projection.scala:171)
>       at 
> org.apache.spark.sql.execution.ConvertToSafe$$anonfun$2.apply(rowFormatConverters.scala:57)
>       at 
> org.apache.spark.sql.execution.ConvertToSafe$$anonfun$2.apply(rowFormatConverters.scala:56)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>       at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
>       at org.apache.spark.scheduler.Task.run(Task.scala:88)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:213)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> +----+-------+
> | age|   name|
> +----+-------+
> |null|Michael|
> |  30|   Andy|
> |  19| Justin|
> +----+-------+
> scala> df.show
> +----+-------+
> | age|   name|
> +----+-------+
> |null|Michael|
> |  30|   Andy|
> |  19| Justin|
> +----+-------+
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to