Description
Observed the following stack trace when debugging one of the jobs.
Though TezSpillRecord uses LocalFileSystem, it ends up getting blocked due to FileSystem cache.
It would be good to have a way of passing RawLocalFileSystem to TezSpillRecord.
"Task-Executor-11" #273 daemon prio=5 os_prio=0 tid=0x00007fe204664800 nid=0x2343d2 waiting on condition [0x00007fe1fcfda000] java.lang.Thread.State: TIMED_WAITING (sleeping) at java.lang.Thread.sleep(Native Method) at org.apache.hadoop.ipc.Client.stop(Client.java:1329) at org.apache.hadoop.ipc.ClientCache.stopClient(ClientCache.java:113) at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.close(ProtobufRpcEngine.java:302) at org.apache.hadoop.ipc.RPC.stopProxy(RPC.java:677) at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.close(ClientNamenodeProtocolTranslatorPB.java:304) at org.apache.hadoop.ipc.RPC.stopProxy(RPC.java:672) at org.apache.hadoop.io.retry.DefaultFailoverProxyProvider.close(DefaultFailoverProxyProvider.java:57) at org.apache.hadoop.io.retry.RetryInvocationHandler$ProxyDescriptor.close(RetryInvocationHandler.java:234) at org.apache.hadoop.io.retry.RetryInvocationHandler.close(RetryInvocationHandler.java:444) at org.apache.hadoop.ipc.RPC.stopProxy(RPC.java:677) at org.apache.hadoop.hdfs.DFSClient.closeConnectionToNamenode(DFSClient.java:592) at org.apache.hadoop.hdfs.DFSClient.close(DFSClient.java:633) - locked <0x00007fed071063a0> (a org.apache.hadoop.hdfs.DFSClient) at org.apache.hadoop.hdfs.DistributedFileSystem.close(DistributedFileSystem.java:1358) at org.apache.hadoop.fs.FileSystem$Cache.closeAll(FileSystem.java:3463) - locked <0x00007fe63d000000> (a org.apache.hadoop.fs.FileSystem$Cache) at org.apache.hadoop.fs.FileSystem.closeAllForUGI(FileSystem.java:576) at org.apache.hadoop.hive.llap.daemon.impl.TaskRunnerCallable.callInternal(TaskRunnerCallable.java:299) at org.apache.hadoop.hive.llap.daemon.impl.TaskRunnerCallable.callInternal(TaskRunnerCallable.java:93) at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36) at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:108) at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:41) at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:77) 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) "New I/O worker #44" #80 prio=5 os_prio=0 tid=0x00007fede2a03000 nid=0x233f2b waiting for monitor entry [0x00007fe20cd3d000] java.lang.Thread.State: BLOCKED (on object monitor) at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3345) - waiting to lock <0x00007fe63d000000> (a org.apache.hadoop.fs.FileSystem$Cache) at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3320) at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:479) at org.apache.hadoop.fs.FileSystem.getLocal(FileSystem.java:435) at org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord.<init>(TezSpillRecord.java:65) at org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord.<init>(TezSpillRecord.java:58) at org.apache.hadoop.hive.llap.shufflehandler.IndexCache.readIndexFileToCache(IndexCache.java:121) at org.apache.hadoop.hive.llap.shufflehandler.IndexCache.getIndexInformation(IndexCache.java:70) at org.apache.hadoop.hive.llap.shufflehandler.ShuffleHandler$Shuffle.getMapOutputInfo(ShuffleHandler.java:887) at org.apache.hadoop.hive.llap.shufflehandler.ShuffleHandler$Shuffle.populateHeaders(ShuffleHandler.java:908) at org.apache.hadoop.hive.llap.shufflehandler.ShuffleHandler$Shuffle.messageReceived(ShuffleHandler.java:805) at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70) at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564) at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791) at org.jboss.netty.handler.stream.ChunkedWriteHandler.handleUpstream(ChunkedWriteHandler.java:142) at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564) at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791) at org.jboss.netty.handler.codec.http.HttpChunkAggregator.messageReceived(HttpChunkAggregator.java:145) at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70) at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564) at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791) at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:296) at org.jboss.netty.handler.codec.frame.FrameDecoder.unfoldAndFireMessageReceived(FrameDecoder.java:459) at org.jboss.netty.handler.codec.replay.ReplayingDecoder.callDecode(ReplayingDecoder.java:536) at org.jboss.netty.handler.codec.replay.ReplayingDecoder.messageReceived(ReplayingDecoder.java:435) at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70) at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564) at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:559) at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:268) at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:255) at org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:88) at org.jboss.netty.channel.socket.nio.AbstractNioWorker.process(AbstractNioWorker.java:108) at org.jboss.netty.channel.socket.nio.AbstractNioSelector.run(AbstractNioSelector.java:337) at org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:89) at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:178) at org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnable.java:108) at org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42) 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)
Note: For the IPC sleep issue, it is related to https://issues.apache.org/jira/browse/HADOOP-16126
Attachments
Attachments
Issue Links
- blocks
-
HIVE-23190 LLAP: modify IndexCache to pass filesystem object to TezSpillRecord
- Closed