You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@tez.apache.org by "Rajesh Balamohan (Jira)" <ji...@apache.org> on 2020/04/13 11:18:00 UTC

[jira] [Commented] (TEZ-4145) Reduce lock contention in TezSpillRecord

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

Rajesh Balamohan commented on TEZ-4145:
---------------------------------------

Note: TezSpillRecord is in the hot path in fetcher, ordered, sorter codebases.

> Reduce lock contention in TezSpillRecord
> ----------------------------------------
>
>                 Key: TEZ-4145
>                 URL: https://issues.apache.org/jira/browse/TEZ-4145
>             Project: Apache Tez
>          Issue Type: Improvement
>            Reporter: Rajesh Balamohan
>            Priority: Major
>
> 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.
> {noformat}
> "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)
> {noformat}
> Note: For the IPC sleep issue, it is related to https://issues.apache.org/jira/browse/HADOOP-16126
>   



--
This message was sent by Atlassian Jira
(v8.3.4#803005)