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

Nandakumar commented on HIVE-15807:
-----------------------------------

[~pattipaka], yes it looks like the pool ran out of threads to do the actual 
work, so the actual work is in the queue.

We can split the {{hive.mv.files.thread}} count and create different pool for 
each level (depth)

{code}
private List<ExecutorService> getExecutorServiceList(int 
numberOfExecutorService, int totalThreads) {
  int average = totalThreads / numberOfExecutorService;
  int remainingThreads = totalThreads;
  List<ExecutorService> serviceList = new 
ArrayList<ExecutorService>(numberOfExecutorService);
  while(numberOfExecutorService == 0) {
    --numberOfExecutorService;
    int numberOfThreads = numberOfExecutorService != 0 ? average : 
remainingThreads;
    serviceList.add(Executors.newFixedThreadPool(numberOfThreads, new 
ThreadFactoryBuilder().setDaemon(true).setNameFormat("MSCK-GetPaths-%d").build()));
    remainingThreads = remainingThreads - numberOfThreads;
  }
  return serviceList;
}
{code}

Now instead of passing {{pool}} to  {{private void checkPartitionDirs(final 
ExecutorService pool, final ConcurrentLinkedQueue<Path> basePaths, final 
Set<Path> allDirs, final FileSystem fs, final int depth, final int maxDepth)}}, 
we can pass {{List<ExecutorService> poolList}} and while accessing the pool we 
can use {{poolList\[depth\]#submit}}

> MSCK operation hangs in HiveMetaStoreChecker.checkPartitionDirs
> ---------------------------------------------------------------
>
>                 Key: HIVE-15807
>                 URL: https://issues.apache.org/jira/browse/HIVE-15807
>             Project: Hive
>          Issue Type: Bug
>    Affects Versions: 2.2.0
>            Reporter: Subramanyam Pattipaka
>            Assignee: Pengcheng Xiong
>             Fix For: 2.2.0
>
>         Attachments: msck-jstack.txt
>
>
> The seems to be a regression from HIVE-14511. The operation was hung in 
> checkPartitionDirs. The data has 3 levels of partitions (month, date, id) 
> which has total of 800 partitions.
> An example path would look like month=9/day=30/id=12
> The default value for hive config hive.mv.files.thread was set to 128. I have 
> attached the jstack of hive process used to run msck command
> checkPartitionDirs is implemented as recursive function which uses same pool 
> to submit worker threads. It seems thread pool ran out of thread to do the 
> actual work and all threads seems to be waiting and hung. Please take a look 
> the stack and confirm if this is the case here.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to