[ https://issues.apache.org/jira/browse/HDFS-15493?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17167143#comment-17167143 ]
Stephen O'Donnell commented on HDFS-15493: ------------------------------------------ {quote} I had tested loading the caches and blocks by two single thread executors, same to your test result, there would be a long time to wait the executors terminated, so the time cost was not better than the one executor with four threads. {quote} Did you find the runtime was about the same with a single executor with 4 threads and two executors with a single thread? As my testing showed a small improvement with the two single threaded executors case, and as locking prevents more than one thread to run concurrently, I think it would be better to go with the two executors with a single thread. This think the time required for the executors to shutdown should be about the same in both cases. I also made an earlier comment on this code: {code} if (blocksMapUpdateExecutor != null) { blocksMapUpdateExecutor.shutdown(); Try { while (!blocksMapUpdateExecutor.isTerminated()) { blocksMapUpdateExecutor.awaitTermination(1, TimeUnit.MILLISECONDS); } } catch (InterruptedException e) { LOG.error("Interrupted waiting for blocksMap update threads.", e); throw new IOException(e); } } {code} I mis-understood how it worked, as I thought `awaitTermination(...)` threw an exception after the timeout, which is not the case. However, I think it makes sense to wait 500 or 1000ms rather than 1ms, and log a message indicating the executor is not yet shutdown. Or, we could time how long it takes to shutdown and log a message after the shutdown completes. That means we will get some visibility into how long the executors take to catch up. Also, for info, I ran my tests on trunk and the image also had some snapshots which will have extended the load time. > Update block map and name cache in parallel while loading fsimage. > ------------------------------------------------------------------ > > Key: HDFS-15493 > URL: https://issues.apache.org/jira/browse/HDFS-15493 > Project: Hadoop HDFS > Issue Type: Improvement > Components: namenode > Reporter: Chengwei Wang > Priority: Major > Attachments: HDFS-15493.001.patch, fsimage-loading.log > > > While loading INodeDirectorySection of fsimage, it will update name cache and > block map after added inode file to inode directory. It would reduce time > cost of fsimage loading to enable these steps run in parallel. > In our test case, with patch HDFS-13694 and HDFS-14617, the time cost to load > fsimage (220M files & 240M blocks) is 470s, with this patch , the time cost > reduc to 410s. -- This message was sent by Atlassian Jira (v8.3.4#803005) --------------------------------------------------------------------- To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org