You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@tajo.apache.org by "Min Zhou (JIRA)" <ji...@apache.org> on 2014/01/23 21:22:37 UTC

[jira] [Commented] (TAJO-544) Thread pool abusing

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

Min Zhou commented on TAJO-544:
-------------------------------

>From the result of I have 3 suspicions. 

# Take a look at Fetcher.java.  Each data fetching create a threadpool and this thread pool live during the whole time of data fetching.

# QueryMasterTask.init, each QueryMasterTask create a threadpool by instancing a brand new TajoResourceAllocator. If there are concurrent queries, there should be several thread pools.

# ugly implementation of hadoop ipc, each connection start a thread
{noformat}
org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:704)
org.apache.hadoop.ipc.Client$Connection.access$2600(Client.java:314)
org.apache.hadoop.ipc.Client.getConnection(Client.java:1399)
org.apache.hadoop.ipc.Client.call(Client.java:1318)
org.apache.hadoop.ipc.Client.call(Client.java:1300)
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:206)
$Proxy9.getBlockLocations(Unknown Source)
sun.reflect.GeneratedMethodAccessor18.invoke(Unknown Source)
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
java.lang.reflect.Method.invoke(Method.java:597)
org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:186)
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
$Proxy9.getBlockLocations(Unknown Source)
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getBlockLocations(ClientNamenodeProtocolTranslatorPB.java:188)
org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(DFSClient.java:1064)
org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1054)
org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1044)
org.apache.hadoop.hdfs.DFSInputStream.fetchLocatedBlocksAndGetLastBlockLength(DFSInputStream.java:235)
org.apache.hadoop.hdfs.DFSInputStream.openInfo(DFSInputStream.java:202)
org.apache.hadoop.hdfs.DFSInputStream.<init>(DFSInputStream.java:195)
org.apache.hadoop.hdfs.DFSClient.open(DFSClient.java:1212)
org.apache.hadoop.hdfs.DistributedFileSystem$3.doCall(DistributedFileSystem.java:290)
org.apache.hadoop.hdfs.DistributedFileSystem$3.doCall(DistributedFileSystem.java:286)
org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
org.apache.hadoop.hdfs.DistributedFileSystem.open(DistributedFileSystem.java:286)
org.apache.hadoop.fs.FileSystem.open(FileSystem.java:763)
org.apache.tajo.storage.CSVFile$CSVScanner.init(CSVFile.java:292)
org.apache.tajo.engine.planner.physical.SeqScanExec.init(SeqScanExec.java:165)
org.apache.tajo.engine.planner.physical.UnaryPhysicalExec.init(UnaryPhysicalExec.java:47)
org.apache.tajo.engine.planner.physical.UnaryPhysicalExec.init(UnaryPhysicalExec.java:47)
org.apache.tajo.engine.planner.physical.HashShuffleFileWriteExec.init(HashShuffleFileWriteExec.java:81)
org.apache.tajo.worker.Task.run(Task.java:369)
org.apache.tajo.worker.TaskRunner$1.run(TaskRunner.java:392)
java.lang.Thread.run(Thread.java:662)
{noformat}
But this thread live for a short time.








> Thread pool abusing
> -------------------
>
>                 Key: TAJO-544
>                 URL: https://issues.apache.org/jira/browse/TAJO-544
>             Project: Tajo
>          Issue Type: Bug
>    Affects Versions: 0.8-incubating
>            Reporter: Min Zhou
>         Attachments: thread.txt
>
>
> After TAJO-522, TAJO-537, I still came across the "can't create native thread" OOM.  I will submit a stack trace which tracking all the source of creating new threads.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)