Uploaded image for project: 'Hadoop HDFS'
  1. Hadoop HDFS
  2. HDFS-4879

Add "blocked ArrayList" collection to avoid CMS full GCs

    Details

    • Type: Improvement
    • Status: Closed
    • Priority: Major
    • Resolution: Fixed
    • Affects Version/s: 2.0.4-alpha, 3.0.0-alpha1
    • Fix Version/s: 2.3.0
    • Component/s: namenode
    • Labels:
      None
    • Target Version/s:

      Description

      We recently saw an issue where a large deletion was issued which caused 25M blocks to be collected during deleteInternal. Currently, the list of collected blocks is an ArrayList, meaning that we had to allocate a contiguous 25M-entry array (~400MB). After a NN has been running for a long amount of time, the old generation may become fragmented such that it's hard to find a 400MB contiguous chunk of heap.

      In general, we should try to design the NN such that the only large objects are long-lived and created at startup time. We can improve this particular case (and perhaps some others) by introducing a new List implementation which is made of a linked list of arrays, each of which is size-limited (eg to 1MB).

      1. hdfs-4879.txt
        16 kB
        Todd Lipcon
      2. hdfs-4879.txt
        16 kB
        Todd Lipcon
      3. hdfs-4879.txt
        49 kB
        Todd Lipcon
      4. hdfs-4879.txt
        48 kB
        Todd Lipcon

        Activity

        Hide
        tlipcon Todd Lipcon added a comment -

        Attached patch implements the simple data structure and integrates it in the delete and deleteSnapshot cases. There may be a few other cases where this would be helpful, but nothing sprang to mind immediately.

        The patch looks large mostly because I had to change a bunch of "List<INode>" to "ChunkedArrayList<INode>". I decided this was better than making the new data structure implement the whole List interface, given that we only need very few of the list operations (add, iterate, and clear).

        The patch has a unit test for the data structure which includes a simple performance test - performance is comparable to ArrayList as expected for our use cases.

        The changes to the NN code itself are covered by existing tests. I added some log info to the new data structure code and ran TestLargeDirectoryDelete and verified that it was creating multiple "chunks" in the data structure.

        Show
        tlipcon Todd Lipcon added a comment - Attached patch implements the simple data structure and integrates it in the delete and deleteSnapshot cases. There may be a few other cases where this would be helpful, but nothing sprang to mind immediately. The patch looks large mostly because I had to change a bunch of "List<INode>" to "ChunkedArrayList<INode>". I decided this was better than making the new data structure implement the whole List interface, given that we only need very few of the list operations (add, iterate, and clear). The patch has a unit test for the data structure which includes a simple performance test - performance is comparable to ArrayList as expected for our use cases. The changes to the NN code itself are covered by existing tests. I added some log info to the new data structure code and ran TestLargeDirectoryDelete and verified that it was creating multiple "chunks" in the data structure.
        Hide
        tlipcon Todd Lipcon added a comment -

        New rev with slight improvements I noticed during further testing:

        • made default chunk size 8K elements instead of 4K - should be slightly faster, and still small enough chunks to fit in a fragmented heap
        • previously was allowing even the first chunk to realloc itself up to maxChunkSize, but now just allocates a new chunk whenever any chunk fills its allocated capacity.
        • cache the last chunk, which sped up add() a bit
        • change the perf test to alternate between the two implementations to be a little more fair in terms of jit, gc, etc

        This should be ready for review now.

        Show
        tlipcon Todd Lipcon added a comment - New rev with slight improvements I noticed during further testing: made default chunk size 8K elements instead of 4K - should be slightly faster, and still small enough chunks to fit in a fragmented heap previously was allowing even the first chunk to realloc itself up to maxChunkSize, but now just allocates a new chunk whenever any chunk fills its allocated capacity. cache the last chunk, which sped up add() a bit change the perf test to alternate between the two implementations to be a little more fair in terms of jit, gc, etc This should be ready for review now.
        Hide
        cmccabe Colin P. McCabe added a comment -

        This is a good improvement; it should be really helpful in keeping those GCs down.

        Since you're not actually implementing java.util.List, why not name it something like BlockedArrayContainer?

        As long as we're merely implementing Iterable, do we have to implement get at all? it seems to be fairly inefficient, so it might encourage the wrong kind of programmer behavior.

        Show
        cmccabe Colin P. McCabe added a comment - This is a good improvement; it should be really helpful in keeping those GCs down. Since you're not actually implementing java.util.List , why not name it something like BlockedArrayContainer ? As long as we're merely implementing Iterable , do we have to implement get at all? it seems to be fairly inefficient, so it might encourage the wrong kind of programmer behavior.
        Hide
        sureshms Suresh Srinivas added a comment -

        Given that the list is short lived, why not just use linked list. I don't think the cost of an object and next pointer should be a big deal for a short lived object.

        Show
        sureshms Suresh Srinivas added a comment - Given that the list is short lived, why not just use linked list. I don't think the cost of an object and next pointer should be a big deal for a short lived object.
        Hide
        jbellis Jonathan Ellis added a comment -

        Why not implement an actual j.u.List, e.g. via j.u.AbstractList?

        Show
        jbellis Jonathan Ellis added a comment - Why not implement an actual j.u.List, e.g. via j.u.AbstractList?
        Hide
        daryn Daryn Sharp added a comment -

        I think this is a great change, but agree that ChunkedArrayList should ideally be a full-fledged list. We may find this list implementation to be useful in other places, which is a benefit over using an actual linked list.

        Comments/suggestions:

        • Default ctor should invoke the ctor with capacity/size to avoid code duplication.
        • Consider avoiding need to compute size by tracking it via add/remove? This would simplify isEmpty() to size == 0.
        • Consider removing multiple calls to addChunk() to seed the main list by folding the logic into add? It could add a new chunk if the list is either empty, or the existing full chunk logic.
        • Why does each additional chunk's capacity quadruple? If necessary, it would be more understandable to multiple by 4.
        Show
        daryn Daryn Sharp added a comment - I think this is a great change, but agree that ChunkedArrayList should ideally be a full-fledged list. We may find this list implementation to be useful in other places, which is a benefit over using an actual linked list. Comments/suggestions: Default ctor should invoke the ctor with capacity/size to avoid code duplication. Consider avoiding need to compute size by tracking it via add/remove? This would simplify isEmpty() to size == 0. Consider removing multiple calls to addChunk() to seed the main list by folding the logic into add? It could add a new chunk if the list is either empty, or the existing full chunk logic. Why does each additional chunk's capacity quadruple? If necessary, it would be more understandable to multiple by 4.
        Hide
        tlipcon Todd Lipcon added a comment -

        Given that the list is short lived, why not just use linked list. I don't think the cost of an object and next pointer should be a big deal for a short lived object.

        In the case of a delete of 25M blocks, like we saw, the linked list is going to be significantly bigger than the ArrayList. I think each Node object takes up 64 bytes, right? So the short lived linked list would be 1.6GB instead of 400MB, which is likely to push it out of the young generation. It also has worse locality of access, causing many more CPU cache misses to traverse over.

        Jonathan: Why not implement an actual j.u.List, e.g. via j.u.AbstractList?

        Daryn: I think this is a great change, but agree that ChunkedArrayList should ideally be a full-fledged list. We may find this list implementation to be useful in other places, which is a benefit over using an actual linked list.

        I started out down this path, but given that the target use cases today only require accumulating entries and then enumerating them, I didn't want to add a bunch of unused code for future use cases we haven't found yet. I'd prefer to commit this simple fix for now, since it addresses a real problem, and then if someone finds a further use case for it later which requires all of the other List features, we can always add them. It's a private internal API, so I don't see any problem expanding it later.

        Consider removing multiple calls to addChunk() to seed the main list by folding the logic into add? It could add a new chunk if the list is either empty, or the existing full chunk logic.

        I'm not following what you mean here. Which code path are you talking about?

        Why does each additional chunk's capacity quadruple? If necessary, it would be more understandable to multiple by 4.

        It is actually tripling, which I thought was mirroring what ArrayList does. But in fact, it looks like ArrayList grows by 1.5x each time:

            private void grow(int minCapacity) {
                // overflow-conscious code
                int oldCapacity = elementData.length;
                int newCapacity = oldCapacity + (oldCapacity >> 1);
        

        I'll switch to that method here.

        Will also make the other changes suggested above in the next revision.

        Show
        tlipcon Todd Lipcon added a comment - Given that the list is short lived, why not just use linked list. I don't think the cost of an object and next pointer should be a big deal for a short lived object. In the case of a delete of 25M blocks, like we saw, the linked list is going to be significantly bigger than the ArrayList. I think each Node object takes up 64 bytes, right? So the short lived linked list would be 1.6GB instead of 400MB, which is likely to push it out of the young generation. It also has worse locality of access, causing many more CPU cache misses to traverse over. Jonathan: Why not implement an actual j.u.List, e.g. via j.u.AbstractList? Daryn: I think this is a great change, but agree that ChunkedArrayList should ideally be a full-fledged list. We may find this list implementation to be useful in other places, which is a benefit over using an actual linked list. I started out down this path, but given that the target use cases today only require accumulating entries and then enumerating them, I didn't want to add a bunch of unused code for future use cases we haven't found yet. I'd prefer to commit this simple fix for now, since it addresses a real problem, and then if someone finds a further use case for it later which requires all of the other List features, we can always add them. It's a private internal API, so I don't see any problem expanding it later. Consider removing multiple calls to addChunk() to seed the main list by folding the logic into add? It could add a new chunk if the list is either empty, or the existing full chunk logic. I'm not following what you mean here. Which code path are you talking about? Why does each additional chunk's capacity quadruple? If necessary, it would be more understandable to multiple by 4. It is actually tripling, which I thought was mirroring what ArrayList does. But in fact, it looks like ArrayList grows by 1.5x each time: private void grow( int minCapacity) { // overflow-conscious code int oldCapacity = elementData.length; int newCapacity = oldCapacity + (oldCapacity >> 1); I'll switch to that method here. Will also make the other changes suggested above in the next revision.
        Hide
        daryn Daryn Sharp added a comment -

        I started out down this path, but given that the target use cases today only require accumulating entries and then enumerating them, I didn't want to add a bunch of unused code for future use cases we haven't found yet.

        How about a middle of the road approach: implement as a list but throw UnsupportedOperationException for the unimplemented methods? Then it becomes a drop-in replacement that doesn't require changing all the data types in the code.

        Consider removing multiple calls to addChunk() to seed the main list by folding the logic into add? It could add a new chunk if the list is either empty, or the existing full chunk logic.

        I'm not following what you mean here. Which code path are you talking about?

        Ie. By adding an isEmpty check, I think the ctor no longer needs to add a chunk. It's a minor suggestion.

        public boolean add(T e) {
          if (chunks.isEmpty() || lastChunk.size() >= lastChunkCapacity) {
            int newCapacity = lastChunkCapacity + (lastChunkCapacity << 1);
            addChunk(Math.min(newCapacity, maxChunkSize));
          }
          return lastChunk.add(e);
        }
        

        Regarding capacity increase, I understand ArrayList does it to avoid excessive reallocs. In this impl, wouldn't a uniform chunk size would be more desirable? Otherwise the last few chunks of an extremely large list will be huge. I don't have a strong opinion either way.

        Show
        daryn Daryn Sharp added a comment - I started out down this path, but given that the target use cases today only require accumulating entries and then enumerating them, I didn't want to add a bunch of unused code for future use cases we haven't found yet. How about a middle of the road approach: implement as a list but throw UnsupportedOperationException for the unimplemented methods? Then it becomes a drop-in replacement that doesn't require changing all the data types in the code. Consider removing multiple calls to addChunk() to seed the main list by folding the logic into add? It could add a new chunk if the list is either empty, or the existing full chunk logic. I'm not following what you mean here. Which code path are you talking about? Ie. By adding an isEmpty check, I think the ctor no longer needs to add a chunk. It's a minor suggestion. public boolean add(T e) { if (chunks.isEmpty() || lastChunk.size() >= lastChunkCapacity) { int newCapacity = lastChunkCapacity + (lastChunkCapacity << 1); addChunk( Math .min(newCapacity, maxChunkSize)); } return lastChunk.add(e); } Regarding capacity increase, I understand ArrayList does it to avoid excessive reallocs. In this impl, wouldn't a uniform chunk size would be more desirable? Otherwise the last few chunks of an extremely large list will be huge. I don't have a strong opinion either way.
        Hide
        sureshms Suresh Srinivas added a comment - - edited

        In the case of a delete of 25M blocks, like we saw, the linked list is going to be significantly bigger than the ArrayList. I think each Node object takes up 64 bytes, right? So the short lived linked list would be 1.6GB instead of 400MB, which is likely to push it out of the young generation. It also has worse locality of access, causing many more CPU cache misses to traverse over.

        Each node object takes 40 bytes. But namenodes are run with enough head room and this much memory should not be an issue. As regards to CPU cache misses, given delete had to touch so many objects and namenode in general has so much active memory, garbage collection etc. I do not think it is such a big deal. Even if namenode were to do more work, this kind of big deletes are such a rarity, trying to make things performant for it seems unnecessary to me.

        Show
        sureshms Suresh Srinivas added a comment - - edited In the case of a delete of 25M blocks, like we saw, the linked list is going to be significantly bigger than the ArrayList. I think each Node object takes up 64 bytes, right? So the short lived linked list would be 1.6GB instead of 400MB, which is likely to push it out of the young generation. It also has worse locality of access, causing many more CPU cache misses to traverse over. Each node object takes 40 bytes. But namenodes are run with enough head room and this much memory should not be an issue. As regards to CPU cache misses, given delete had to touch so many objects and namenode in general has so much active memory, garbage collection etc. I do not think it is such a big deal. Even if namenode were to do more work, this kind of big deletes are such a rarity, trying to make things performant for it seems unnecessary to me.
        Hide
        tlipcon Todd Lipcon added a comment -

        New patch changes ChunkedArrayList to extend AbstractList, which just throws exceptions on unsupported operations. I also addressed the above feedback.

        Given that the list now implements the List interface, the patch is much smaller and I think much improved. Thanks for the review, folks.

        I ran TestLargeDirectoryDelete and it passed.

        Show
        tlipcon Todd Lipcon added a comment - New patch changes ChunkedArrayList to extend AbstractList, which just throws exceptions on unsupported operations. I also addressed the above feedback. Given that the list now implements the List interface, the patch is much smaller and I think much improved. Thanks for the review, folks. I ran TestLargeDirectoryDelete and it passed.
        Hide
        hadoopqa Hadoop QA added a comment -

        +1 overall. Here are the results of testing the latest attachment
        http://issues.apache.org/jira/secure/attachment/12591245/hdfs-4879.txt
        against trunk revision .

        +1 @author. The patch does not contain any @author tags.

        +1 tests included. The patch appears to include 1 new or modified test files.

        +1 javac. The applied patch does not increase the total number of javac compiler warnings.

        +1 javadoc. The javadoc tool did not generate any warning messages.

        +1 eclipse:eclipse. The patch built with eclipse:eclipse.

        +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

        +1 release audit. The applied patch does not increase the total number of release audit warnings.

        +1 core tests. The patch passed unit tests in hadoop-hdfs-project/hadoop-hdfs.

        +1 contrib tests. The patch passed contrib unit tests.

        Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/4603//testReport/
        Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/4603//console

        This message is automatically generated.

        Show
        hadoopqa Hadoop QA added a comment - +1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12591245/hdfs-4879.txt against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 1 new or modified test files. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +1 javadoc . The javadoc tool did not generate any warning messages. +1 eclipse:eclipse . The patch built with eclipse:eclipse. +1 findbugs . The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. +1 core tests . The patch passed unit tests in hadoop-hdfs-project/hadoop-hdfs. +1 contrib tests . The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/4603//testReport/ Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/4603//console This message is automatically generated.
        Hide
        tlipcon Todd Lipcon added a comment -

        Any further comments, folks?

        Show
        tlipcon Todd Lipcon added a comment - Any further comments, folks?
        Hide
        cmccabe Colin P. McCabe added a comment -
        +      assertEquals(fromList, i);
        

        Since 'fromList' is what we are testing, it should appear as the second argument here, not first.

        +1 once this is addressed

        Show
        cmccabe Colin P. McCabe added a comment - + assertEquals(fromList, i); Since 'fromList' is what we are testing, it should appear as the second argument here, not first. +1 once this is addressed
        Hide
        atm Aaron T. Myers added a comment -

        I agree with Colin's comment, but would also add that I believe the following code comment is no longer correct as of the latest rev, due to the lazy initialization:

        +   * The chunks which make up the full list. This array always
        +   * has at least one element, even in an empty list, to make
        +   * some of the code simpler.
        

        +1 from me once both of the above are addressed.

        Show
        atm Aaron T. Myers added a comment - I agree with Colin's comment, but would also add that I believe the following code comment is no longer correct as of the latest rev, due to the lazy initialization: + * The chunks which make up the full list. This array always + * has at least one element, even in an empty list, to make + * some of the code simpler. +1 from me once both of the above are addressed.
        Hide
        hadoopqa Hadoop QA added a comment -

        -1 overall. Here are the results of testing the latest attachment
        http://issues.apache.org/jira/secure/attachment/12600285/hdfs-4879.txt
        against trunk revision .

        +1 @author. The patch does not contain any @author tags.

        +1 tests included. The patch appears to include 1 new or modified test files.

        +1 javac. The applied patch does not increase the total number of javac compiler warnings.

        +1 javadoc. The javadoc tool did not generate any warning messages.

        +1 eclipse:eclipse. The patch built with eclipse:eclipse.

        +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

        +1 release audit. The applied patch does not increase the total number of release audit warnings.

        -1 core tests. The patch failed these unit tests in hadoop-hdfs-project/hadoop-hdfs:

        org.apache.hadoop.hdfs.server.namenode.TestListCorruptFileBlocks

        +1 contrib tests. The patch passed contrib unit tests.

        Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/4901//testReport/
        Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/4901//console

        This message is automatically generated.

        Show
        hadoopqa Hadoop QA added a comment - -1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12600285/hdfs-4879.txt against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 1 new or modified test files. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +1 javadoc . The javadoc tool did not generate any warning messages. +1 eclipse:eclipse . The patch built with eclipse:eclipse. +1 findbugs . The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. -1 core tests . The patch failed these unit tests in hadoop-hdfs-project/hadoop-hdfs: org.apache.hadoop.hdfs.server.namenode.TestListCorruptFileBlocks +1 contrib tests . The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/4901//testReport/ Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/4901//console This message is automatically generated.
        Hide
        tlipcon Todd Lipcon added a comment -

        Test failure appears to be HDFS-5073

        Show
        tlipcon Todd Lipcon added a comment - Test failure appears to be HDFS-5073
        Hide
        cmccabe Colin P. McCabe added a comment -

        It looks like both ATM's comment and mine have been addressed in the latest patch. +1.

        Will commit in a day or two if there's no more comments.

        Show
        cmccabe Colin P. McCabe added a comment - It looks like both ATM's comment and mine have been addressed in the latest patch. +1. Will commit in a day or two if there's no more comments.
        Hide
        cmccabe Colin P. McCabe added a comment -

        committed to branch 2.3

        Show
        cmccabe Colin P. McCabe added a comment - committed to branch 2.3
        Hide
        hudson Hudson added a comment -

        SUCCESS: Integrated in Hadoop-trunk-Commit #4380 (See https://builds.apache.org/job/Hadoop-trunk-Commit/4380/)
        HDFS-4879. Add BlockedArrayList collection to avoid CMS full GCs (Contributed by Todd Lipcon) (cmccabe: http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1520667)

        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/ChunkedArrayList.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestChunkedArrayList.java
        Show
        hudson Hudson added a comment - SUCCESS: Integrated in Hadoop-trunk-Commit #4380 (See https://builds.apache.org/job/Hadoop-trunk-Commit/4380/ ) HDFS-4879 . Add BlockedArrayList collection to avoid CMS full GCs (Contributed by Todd Lipcon) (cmccabe: http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1520667 ) /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/ChunkedArrayList.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestChunkedArrayList.java
        Hide
        hudson Hudson added a comment -

        SUCCESS: Integrated in Hadoop-Yarn-trunk #325 (See https://builds.apache.org/job/Hadoop-Yarn-trunk/325/)
        HDFS-4879. Add BlockedArrayList collection to avoid CMS full GCs (Contributed by Todd Lipcon) (cmccabe: http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1520667)

        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/ChunkedArrayList.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestChunkedArrayList.java
        Show
        hudson Hudson added a comment - SUCCESS: Integrated in Hadoop-Yarn-trunk #325 (See https://builds.apache.org/job/Hadoop-Yarn-trunk/325/ ) HDFS-4879 . Add BlockedArrayList collection to avoid CMS full GCs (Contributed by Todd Lipcon) (cmccabe: http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1520667 ) /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/ChunkedArrayList.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestChunkedArrayList.java
        Hide
        hudson Hudson added a comment -

        SUCCESS: Integrated in Hadoop-Hdfs-trunk #1515 (See https://builds.apache.org/job/Hadoop-Hdfs-trunk/1515/)
        HDFS-4879. Add BlockedArrayList collection to avoid CMS full GCs (Contributed by Todd Lipcon) (cmccabe: http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1520667)

        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/ChunkedArrayList.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestChunkedArrayList.java
        Show
        hudson Hudson added a comment - SUCCESS: Integrated in Hadoop-Hdfs-trunk #1515 (See https://builds.apache.org/job/Hadoop-Hdfs-trunk/1515/ ) HDFS-4879 . Add BlockedArrayList collection to avoid CMS full GCs (Contributed by Todd Lipcon) (cmccabe: http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1520667 ) /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/ChunkedArrayList.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestChunkedArrayList.java
        Hide
        hudson Hudson added a comment -

        FAILURE: Integrated in Hadoop-Mapreduce-trunk #1541 (See https://builds.apache.org/job/Hadoop-Mapreduce-trunk/1541/)
        HDFS-4879. Add BlockedArrayList collection to avoid CMS full GCs (Contributed by Todd Lipcon) (cmccabe: http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1520667)

        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/ChunkedArrayList.java
        • /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestChunkedArrayList.java
        Show
        hudson Hudson added a comment - FAILURE: Integrated in Hadoop-Mapreduce-trunk #1541 (See https://builds.apache.org/job/Hadoop-Mapreduce-trunk/1541/ ) HDFS-4879 . Add BlockedArrayList collection to avoid CMS full GCs (Contributed by Todd Lipcon) (cmccabe: http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1520667 ) /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/ChunkedArrayList.java /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestChunkedArrayList.java

          People

          • Assignee:
            tlipcon Todd Lipcon
            Reporter:
            tlipcon Todd Lipcon
          • Votes:
            0 Vote for this issue
            Watchers:
            20 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development