Hadoop Map/Reduce
  1. Hadoop Map/Reduce
  2. MAPREDUCE-5368

Save memory by set capacity, load factor and concurrency level for ConcurrentHashMap in TaskInProgress

    Details

    • Type: Improvement Improvement
    • Status: Closed
    • Priority: Major Major
    • Resolution: Fixed
    • Affects Version/s: 1.2.0
    • Fix Version/s: 1.2.1
    • Component/s: mrv1
    • Labels:
      None

      Description

      Below is histo from our JobTracker:

      num #instances #bytes class name
      ----------------------------------------------
      1: 136048824 11347237456 [C
      2: 124156992 5959535616 java.util.concurrent.locks.ReentrantLock$NonfairSync
      3: 124156973 5959534704 java.util.concurrent.ConcurrentHashMap$Segment
      4: 135887753 5435510120 java.lang.String
      5: 124213692 3975044400 [Ljava.util.concurrent.ConcurrentHashMap$HashEntry;
      6: 63777311 3061310928 java.util.HashMap$Entry
      7: 35038252 2803060160 java.util.TreeMap
      8: 16921110 2712480072 [Ljava.util.HashMap$Entry;
      9: 4803617 2420449192 [Ljava.lang.Object;
      10: 50392816 2015712640 org.apache.hadoop.mapred.Counters$Counter
      11: 7775438 1181866576 [Ljava.util.concurrent.ConcurrentHashMap$Segment;
      12: 3882847 1118259936 org.apache.hadoop.mapred.TaskInProgress

      ConcurrentHashMap takes more than 14G(5959535616 + 5959534704 + 3975044400).
      The trouble maker are below codes in TaskInProgress.java:
      Map<TaskAttemptID, Locality> taskLocality =
      new ConcurrentHashMap<TaskAttemptID, Locality>();
      Map<TaskAttemptID, Avataar> taskAvataar =
      new ConcurrentHashMap<TaskAttemptID, Avataar>();

        Activity

        Hide
        Arun C Murthy added a comment -

        +1. I just committed this. Thanks zhaoyunjiong!

        Show
        Arun C Murthy added a comment - +1. I just committed this. Thanks zhaoyunjiong !
        Hide
        zhaoyunjiong added a comment -

        Normally taskLocality & taskAvataar won't exceeds 4 items(mapred.map/reduce.max.attemps is 4), most of them should be 1.
        If use default initial capacity & concurrency level, the number of instance of NonfairSync, Segment and HashEntry will have 32 times of TaskInProgress, which will consume a lot of memory as shown in above description.
        And there will be very few concurrent access to taskLocality & taskAvataar, actually I was thinking to reduce it to 1 or even replace ConcurrentHashMap.

        Show
        zhaoyunjiong added a comment - Normally taskLocality & taskAvataar won't exceeds 4 items(mapred.map/reduce.max.attemps is 4), most of them should be 1. If use default initial capacity & concurrency level, the number of instance of NonfairSync, Segment and HashEntry will have 32 times of TaskInProgress, which will consume a lot of memory as shown in above description. And there will be very few concurrent access to taskLocality & taskAvataar, actually I was thinking to reduce it to 1 or even replace ConcurrentHashMap.
        Hide
        Mayank Bansal added a comment -

        The Default values of the Load factor is .75 anyways
        The Default concurrency level is 16 which I think is reasonable for the jobs.
        The Default initial capacity is also 16 which is also reasonable.

        I am not sure how we are saving memory here. Can you please explain a bit?

        Moreover I really dont think to change the concurrency level so low as it will increase the contention in the threads a lot.

        Thoughts?

        Thanks,
        Mayank

        Show
        Mayank Bansal added a comment - The Default values of the Load factor is .75 anyways The Default concurrency level is 16 which I think is reasonable for the jobs. The Default initial capacity is also 16 which is also reasonable. I am not sure how we are saving memory here. Can you please explain a bit? Moreover I really dont think to change the concurrency level so low as it will increase the contention in the threads a lot. Thoughts? Thanks, Mayank
        Hide
        Karthik Kambatla added a comment -

        The patch is a definite improvement over the current situation. As in branch-2, can we get rid of the ConcurrentHashMaps altogether, and move Locality and Avataar to TaskAttemptID itself?

        Show
        Karthik Kambatla added a comment - The patch is a definite improvement over the current situation. As in branch-2, can we get rid of the ConcurrentHashMaps altogether, and move Locality and Avataar to TaskAttemptID itself?
        Hide
        zhaoyunjiong added a comment -

        This simple patch can save more than 10GB when there are 4m TaskInProgress instances.

        Show
        zhaoyunjiong added a comment - This simple patch can save more than 10GB when there are 4m TaskInProgress instances.

          People

          • Assignee:
            zhaoyunjiong
            Reporter:
            zhaoyunjiong
          • Votes:
            0 Vote for this issue
            Watchers:
            6 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development