jojochuang commented on a change in pull request #1028: HDFS-14617 - Improve 
fsimage load time by writing sub-sections to the fsimage index
URL: https://github.com/apache/hadoop/pull/1028#discussion_r311835690
 
 

 ##########
 File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
 ##########
 @@ -197,16 +203,66 @@ public static void updateBlocksMap(INodeFile file, 
BlockManager bm) {
     private final FSDirectory dir;
     private final FSNamesystem fsn;
     private final FSImageFormatProtobuf.Loader parent;
+    private ReentrantLock cacheNameMapLock;
+    private ReentrantLock blockMapLock;
 
     Loader(FSNamesystem fsn, final FSImageFormatProtobuf.Loader parent) {
       this.fsn = fsn;
       this.dir = fsn.dir;
       this.parent = parent;
+      cacheNameMapLock = new ReentrantLock(true);
+      blockMapLock = new ReentrantLock(true);
+    }
+
+    void loadINodeDirectorySectionInParallel(ExecutorService service,
+        ArrayList<FileSummary.Section> sections, String compressionCodec)
+        throws IOException {
+      LOG.info("Loading the INodeDirectory section in parallel with {} sub-" +
+              "sections", sections.size());
+      CountDownLatch latch = new CountDownLatch(sections.size());
+      final CopyOnWriteArrayList<IOException> exceptions =
+          new CopyOnWriteArrayList<>();
+      for (FileSummary.Section s : sections) {
+        service.submit(new Runnable() {
+          public void run() {
+            InputStream ins = null;
+            try {
+              ins = parent.getInputStreamForSection(s,
+                  compressionCodec);
+              loadINodeDirectorySection(ins);
+            } catch (Exception e) {
+              LOG.error("An exception occurred loading INodeDirectories in " +
+                  "parallel", e);
+              exceptions.add(new IOException(e));
+            } finally {
+              latch.countDown();
+              try {
+                ins.close();
+              } catch (IOException ioe) {
+                LOG.warn("Failed to close the input stream, ignoring", ioe);
+              }
+            }
+          }
+        });
+      }
+      try {
+        latch.await();
+      } catch (InterruptedException e) {
+        LOG.error("Interrupted waiting for countdown latch", e);
+        throw new IOException(e);
+      }
+      if (exceptions.size() != 0) {
+        LOG.error("{} exceptions occurred loading INodeDirectories",
+            exceptions.size());
+        throw exceptions.get(0);
+      }
+      LOG.info("Completed loading all INodeDirectory sub-sections");
     }
 
     void loadINodeDirectorySection(InputStream in) throws IOException {
       final List<INodeReference> refList = parent.getLoaderContext()
           .getRefList();
+      ArrayList<INode> inodeList = new ArrayList<>();
 
 Review comment:
   i think the performance benefit of batch update outweighs cache locality.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to