Details

    • Type: Bug
    • Status: Resolved
    • Priority: Major
    • Resolution: Fixed
    • Affects Version/s: 0.8.0
    • Fix Version/s: 0.8.0
    • Component/s: None
    • Labels:
      None

      Description

      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.

      1. fetcher.patch
        3 kB
        Min Zhou
      2. thread.txt
        1.79 MB
        Min Zhou

        Issue Links

          Activity

          Hide
          coderplay Min Zhou added a comment -

          From the result of I have 3 suspicions.

          1. Take a look at Fetcher.java. Each data fetching create a threadpool and this thread pool live during the whole time of data fetching.
          1. QueryMasterTask.init, each QueryMasterTask create a threadpool by instancing a brand new TajoResourceAllocator. If there are concurrent queries, there should be several thread pools.
          1. ugly implementation of hadoop ipc, each connection start a thread
            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)
            

            But this thread live for a short time.

          Show
          coderplay Min Zhou added a comment - 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 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) But this thread live for a short time.
          Hide
          coderplay Min Zhou added a comment -

          AS TAJO-537, sharing factory among all of the fetchers.

          Show
          coderplay Min Zhou added a comment - AS TAJO-537 , sharing factory among all of the fetchers.
          Hide
          hyunsik Hyunsik Choi added a comment -

          +1 for the patch, and it's a nice investigation!

          As you pointed out, we have abused ThreadFactory. This is the first point that we can easily try to improve. I've tested it on a real cluster. It works fine.

          Show
          hyunsik Hyunsik Choi added a comment - +1 for the patch, and it's a nice investigation! As you pointed out, we have abused ThreadFactory. This is the first point that we can easily try to improve. I've tested it on a real cluster. It works fine.
          Hide
          hyunsik Hyunsik Choi added a comment -

          committed it to master branch. Thank you for nice contribution.

          Show
          hyunsik Hyunsik Choi added a comment - committed it to master branch. Thank you for nice contribution.
          Hide
          hudson Hudson added a comment -

          ABORTED: Integrated in Tajo-master-build #28 (See https://builds.apache.org/job/Tajo-master-build/28/)
          TAJO-544: Thread pool abusing. (Min Zhou via hyunsik) (hyunsik: https://git-wip-us.apache.org/repos/asf?p=incubator-tajo.git&a=commit&h=3125733cc6f2bbce306afd00b8fa8440f07b6e1e)

          Show
          hudson Hudson added a comment - ABORTED: Integrated in Tajo-master-build #28 (See https://builds.apache.org/job/Tajo-master-build/28/ ) TAJO-544 : Thread pool abusing. (Min Zhou via hyunsik) (hyunsik: https://git-wip-us.apache.org/repos/asf?p=incubator-tajo.git&a=commit&h=3125733cc6f2bbce306afd00b8fa8440f07b6e1e ) tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/Fetcher.java TAJO-544 : Thread pool abusing. (add a missed change log) (hyunsik: https://git-wip-us.apache.org/repos/asf?p=incubator-tajo.git&a=commit&h=a74e52188b82b2c753846ddabd9f74bfe0e94e2f ) CHANGES.txt

            People

            • Assignee:
              coderplay Min Zhou
              Reporter:
              coderplay Min Zhou
            • Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development