Details
-
Bug
-
Status: Resolved
-
Minor
-
Resolution: Cannot Reproduce
-
1.4.1
-
None
-
None
-
Spark 1.4.1,Hadoop 2.6.0,Tachyon 0.6.4
-
Patch
Description
RDD persist to OFF_HEAP tachyon got block rdd_x_x not found exception when parallelism is big than data split size
val rdd = sc.parallelize(List(1, 2),2) rdd.persist(org.apache.spark.storage.StorageLevel.OFF_HEAP) rdd.count()
is ok.
val rdd = sc.parallelize(List(1, 2),3) rdd.persist(org.apache.spark.storage.StorageLevel.OFF_HEAP) rdd.count()
got exceptoin:
15/08/27 17:53:07 INFO SparkContext: Starting job: count at <console>:24 15/08/27 17:53:07 INFO DAGScheduler: Got job 0 (count at <console>:24) with 3 output partitions (allowLocal=false) 15/08/27 17:53:07 INFO DAGScheduler: Final stage: ResultStage 0(count at <console>:24) 15/08/27 17:53:07 INFO DAGScheduler: Parents of final stage: List() 15/08/27 17:53:07 INFO DAGScheduler: Missing parents: List() 15/08/27 17:53:07 INFO DAGScheduler: Submitting ResultStage 0 (ParallelCollectionRDD[0] at parallelize at <console>:21), which has no missing parents 15/08/27 17:53:07 INFO MemoryStore: ensureFreeSpace(1096) called with curMem=0, maxMem=741196431 15/08/27 17:53:07 INFO MemoryStore: Block broadcast_0 stored as values in memory (estimated size 1096.0 B, free 706.9 MB) 15/08/27 17:53:07 INFO MemoryStore: ensureFreeSpace(788) called with curMem=1096, maxMem=741196431 15/08/27 17:53:07 INFO MemoryStore: Block broadcast_0_piece0 stored as bytes in memory (estimated size 788.0 B, free 706.9 MB) 15/08/27 17:53:07 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on localhost:43776 (size: 788.0 B, free: 706.9 MB) 15/08/27 17:53:07 INFO SparkContext: Created broadcast 0 from broadcast at DAGScheduler.scala:874 15/08/27 17:53:07 INFO DAGScheduler: Submitting 3 missing tasks from ResultStage 0 (ParallelCollectionRDD[0] at parallelize at <console>:21) 15/08/27 17:53:07 INFO TaskSchedulerImpl: Adding task set 0.0 with 3 tasks 15/08/27 17:53:07 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0, localhost, PROCESS_LOCAL, 1269 bytes) 15/08/27 17:53:07 INFO TaskSetManager: Starting task 1.0 in stage 0.0 (TID 1, localhost, PROCESS_LOCAL, 1270 bytes) 15/08/27 17:53:07 INFO TaskSetManager: Starting task 2.0 in stage 0.0 (TID 2, localhost, PROCESS_LOCAL, 1270 bytes) 15/08/27 17:53:07 INFO Executor: Running task 2.0 in stage 0.0 (TID 2) 15/08/27 17:53:07 INFO Executor: Running task 0.0 in stage 0.0 (TID 0) 15/08/27 17:53:07 INFO Executor: Running task 1.0 in stage 0.0 (TID 1) 15/08/27 17:53:07 INFO CacheManager: Partition rdd_0_2 not found, computing it 15/08/27 17:53:07 INFO CacheManager: Partition rdd_0_1 not found, computing it 15/08/27 17:53:07 INFO CacheManager: Partition rdd_0_0 not found, computing it 15/08/27 17:53:07 INFO ExternalBlockStore: ExternalBlockStore started 15/08/27 17:53:08 WARN : tachyon.home is not set. Using /mnt/tachyon_default_home as the default value. 15/08/27 17:53:08 INFO : Tachyon client (version 0.6.4) is trying to connect master @ localhost/127.0.0.1:19998 15/08/27 17:53:08 INFO : User registered at the master localhost/127.0.0.1:19998 got UserId 109 15/08/27 17:53:08 INFO TachyonBlockManager: Created tachyon directory at /spark/spark-c6ec419f-7c7d-48a6-8448-c2431e761ea5/driver/spark-tachyon-20150827175308-6aa5 15/08/27 17:53:08 INFO : Trying to get local worker host : localhost 15/08/27 17:53:08 INFO : Connecting local worker @ localhost/127.0.0.1:29998 15/08/27 17:53:08 INFO : Folder /mnt/ramdisk/tachyonworker/users/109 was created! 15/08/27 17:53:08 INFO : /mnt/ramdisk/tachyonworker/users/109/4386235351040 was created! 15/08/27 17:53:08 INFO : /mnt/ramdisk/tachyonworker/users/109/4388382834688 was created! 15/08/27 17:53:08 INFO BlockManagerInfo: Added rdd_0_0 on ExternalBlockStore on localhost:43776 (size: 0.0 B) 15/08/27 17:53:08 INFO BlockManagerInfo: Added rdd_0_1 on ExternalBlockStore on localhost:43776 (size: 2.0 B) 15/08/27 17:53:08 INFO BlockManagerInfo: Added rdd_0_2 on ExternalBlockStore on localhost:43776 (size: 2.0 B) 15/08/27 17:53:08 INFO BlockManager: Found block rdd_0_1 locally 15/08/27 17:53:08 INFO BlockManager: Found block rdd_0_2 locally 15/08/27 17:53:08 INFO Executor: Finished task 2.0 in stage 0.0 (TID 2). 1208 bytes result sent to driver 15/08/27 17:53:08 INFO Executor: Finished task 1.0 in stage 0.0 (TID 1). 1208 bytes result sent to driver 15/08/27 17:53:08 INFO TaskSetManager: Finished task 1.0 in stage 0.0 (TID 1) in 278 ms on localhost (1/3) 15/08/27 17:53:08 INFO TaskSetManager: Finished task 2.0 in stage 0.0 (TID 2) in 275 ms on localhost (2/3) 15/08/27 17:53:08 ERROR TransportRequestHandler: Error while invoking RpcHandler#receive() on RPC id 7732829366749232329 org.apache.spark.storage.BlockNotFoundException: Block rdd_0_0 not found at org.apache.spark.storage.BlockManager.getBlockData(BlockManager.scala:308) at org.apache.spark.network.netty.NettyBlockRpcServer$$anonfun$2.apply(NettyBlockRpcServer.scala:57) at org.apache.spark.network.netty.NettyBlockRpcServer$$anonfun$2.apply(NettyBlockRpcServer.scala:57) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33) at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:108) at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) at scala.collection.mutable.ArrayOps$ofRef.map(ArrayOps.scala:108) at org.apache.spark.network.netty.NettyBlockRpcServer.receive(NettyBlockRpcServer.scala:57) at org.apache.spark.network.server.TransportRequestHandler.processRpcRequest(TransportRequestHandler.java:114) at org.apache.spark.network.server.TransportRequestHandler.handle(TransportRequestHandler.java:87) at org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:101) 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:333) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:319) at io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:254) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:319) at io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:319) at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:163) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:319) at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:787) at io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:130) 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:116) at java.lang.Thread.run(Thread.java:745) 15/08/27 17:53:08 ERROR OneForOneBlockFetcher: Failed while starting block fetches java.lang.RuntimeException: org.apache.spark.storage.BlockNotFoundException: Block rdd_0_0 not found at org.apache.spark.storage.BlockManager.getBlockData(BlockManager.scala:308) at org.apache.spark.network.netty.NettyBlockRpcServer$$anonfun$2.apply(NettyBlockRpcServer.scala:57) at org.apache.spark.network.netty.NettyBlockRpcServer$$anonfun$2.apply(NettyBlockRpcServer.scala:57) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33) at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:108) at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) at scala.collection.mutable.ArrayOps$ofRef.map(ArrayOps.scala:108) at org.apache.spark.network.netty.NettyBlockRpcServer.receive(NettyBlockRpcServer.scala:57) at org.apache.spark.network.server.TransportRequestHandler.processRpcRequest(TransportRequestHandler.java:114) at org.apache.spark.network.server.TransportRequestHandler.handle(TransportRequestHandler.java:87) at org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:101) 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:333) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:319) at io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:254) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:319) at io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:319) at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:163) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:319) at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:787) at io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:130) 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:116) at java.lang.Thread.run(Thread.java:745) at org.apache.spark.network.client.TransportResponseHandler.handle(TransportResponseHandler.java:162) at org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:103) 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:333) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:319) at io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:254) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:319) at io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:319) at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:163) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:319) at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:787) at io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:130) 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:116) at java.lang.Thread.run(Thread.java:745)