[ https://issues.apache.org/jira/browse/HIVE-16014?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15881182#comment-15881182 ]
Sahil Takiar commented on HIVE-16014: ------------------------------------- [~vihangk1] - the config METASTORE_FS_HANDLER_THREADS_COUNT control the size of a threadpool in the metastore, that threadpool calls two main methods: {{createLocationForAddedPartition}} and {{initializeAddedPartition}} The method {{createLocationForAddedPartition}} checks if the input directory exists, which seems unnecessary since when running {{msck}} the partition folders should always exist. The method {{initializeAddedPartition}} calls {{MetaStoreUtils.updatePartitionStatsFast}} which may or may not list the filestatus of the partition directory to collect basic statistics. I believe the whole threadpool logic in the {{HiveMetaStore}} was added to address the overhead of the above two methods when running against S3. > HiveMetastoreChecker should use hive.metastore.fshandler.threads instead of > hive.mv.files.thread for pool size > -------------------------------------------------------------------------------------------------------------- > > Key: HIVE-16014 > URL: https://issues.apache.org/jira/browse/HIVE-16014 > Project: Hive > Issue Type: Improvement > Reporter: Vihang Karajgaonkar > Assignee: Vihang Karajgaonkar > Attachments: HIVE-16014.01.patch, HIVE-16014.02.patch > > > HiveMetastoreChecker uses hive.mv.files.thread configuration value for > determining the pool size as below : > {noformat} > private void checkPartitionDirs(Path basePath, Set<Path> allDirs, int > maxDepth) throws IOException, HiveException { > ConcurrentLinkedQueue<Path> basePaths = new ConcurrentLinkedQueue<>(); > basePaths.add(basePath); > Set<Path> dirSet = Collections.newSetFromMap(new ConcurrentHashMap<Path, > Boolean>()); > // Here we just reuse the THREAD_COUNT configuration for > // HIVE_MOVE_FILES_THREAD_COUNT > int poolSize = conf.getInt(ConfVars.HIVE_MOVE_FILES_THREAD_COUNT.varname, > 15); > // Check if too low config is provided for move files. 2x CPU is > reasonable max count. > poolSize = poolSize == 0 ? poolSize : Math.max(poolSize, > Runtime.getRuntime().availableProcessors() * 2); > {noformat} > msck is commonly used to add the missing partitions for the table from the > Filesystem. In such a case different pool sizes for HMSHandler and > HiveMetastoreChecker can affect the performance. Eg. If > {{hive.metastore.fshandler.threads}} is set to a lower value like 15 and > {{hive.mv.files.thread}} is much higher like 100 or vice versa the smaller > pool will become the bottleneck. If would be good to use > {{hive.metastore.fshandler.threads}} to size the pool for > HiveMetastoreChecker since the number missing partitions and number of > partitions to be added will most likely be the same. In such a case the > performance of the query will be optimum when both the pool sizes are same. > Since it is possible to tune both the configs individually it will be very > likely that they may be different. But since there is a strong co-relation > between amount of work done by HiveMetastoreChecker and > HiveMetastore.add_partitions call it might be a good idea to use > {{hive.metastore.fshandler.threads}} for pool size instead of > {{hive.mv.files.thread}} -- This message was sent by Atlassian JIRA (v6.3.15#6346)