Print this page
rev 4535 : 6725714: par compact - add a table to speed up bitmap searches
Reviewed-by: jmasa, tschatzl

Split Split Close
Expand all
Collapse all
          --- old/src/share/vm/gc_implementation/parallelScavenge/psParallelCompact.cpp
          +++ new/src/share/vm/gc_implementation/parallelScavenge/psParallelCompact.cpp
↓ open down ↓ 56 lines elided ↑ open up ↑
  57   57  #include "runtime/vmThread.hpp"
  58   58  #include "services/management.hpp"
  59   59  #include "services/memoryService.hpp"
  60   60  #include "services/memTracker.hpp"
  61   61  #include "utilities/events.hpp"
  62   62  #include "utilities/stack.inline.hpp"
  63   63  
  64   64  #include <math.h>
  65   65  
  66   66  // All sizes are in HeapWords.
  67      -const size_t ParallelCompactData::Log2RegionSize  = 9; // 512 words
       67 +const size_t ParallelCompactData::Log2RegionSize  = 16; // 64K words
  68   68  const size_t ParallelCompactData::RegionSize      = (size_t)1 << Log2RegionSize;
  69   69  const size_t ParallelCompactData::RegionSizeBytes =
  70   70    RegionSize << LogHeapWordSize;
  71   71  const size_t ParallelCompactData::RegionSizeOffsetMask = RegionSize - 1;
  72   72  const size_t ParallelCompactData::RegionAddrOffsetMask = RegionSizeBytes - 1;
  73      -const size_t ParallelCompactData::RegionAddrMask  = ~RegionAddrOffsetMask;
       73 +const size_t ParallelCompactData::RegionAddrMask       = ~RegionAddrOffsetMask;
  74   74  
       75 +const size_t ParallelCompactData::Log2BlockSize   = 7; // 128 words
       76 +const size_t ParallelCompactData::BlockSize       = (size_t)1 << Log2BlockSize;
       77 +const size_t ParallelCompactData::BlockSizeBytes  =
       78 +  BlockSize << LogHeapWordSize;
       79 +const size_t ParallelCompactData::BlockSizeOffsetMask = BlockSize - 1;
       80 +const size_t ParallelCompactData::BlockAddrOffsetMask = BlockSizeBytes - 1;
       81 +const size_t ParallelCompactData::BlockAddrMask       = ~BlockAddrOffsetMask;
       82 +
       83 +const size_t ParallelCompactData::BlocksPerRegion = RegionSize / BlockSize;
       84 +const size_t ParallelCompactData::Log2BlocksPerRegion =
       85 +  Log2RegionSize - Log2BlockSize;
       86 +
  75   87  const ParallelCompactData::RegionData::region_sz_t
  76   88  ParallelCompactData::RegionData::dc_shift = 27;
  77   89  
  78   90  const ParallelCompactData::RegionData::region_sz_t
  79   91  ParallelCompactData::RegionData::dc_mask = ~0U << dc_shift;
  80   92  
  81   93  const ParallelCompactData::RegionData::region_sz_t
  82   94  ParallelCompactData::RegionData::dc_one = 0x1U << dc_shift;
  83   95  
  84   96  const ParallelCompactData::RegionData::region_sz_t
↓ open down ↓ 287 lines elided ↑ open up ↑
 372  384  size_t mark_bitmap_size;
 373  385  #endif  // #ifdef ASSERT
 374  386  
 375  387  ParallelCompactData::ParallelCompactData()
 376  388  {
 377  389    _region_start = 0;
 378  390  
 379  391    _region_vspace = 0;
 380  392    _region_data = 0;
 381  393    _region_count = 0;
      394 +
      395 +  _block_vspace = 0;
      396 +  _block_data = 0;
      397 +  _block_count = 0;
 382  398  }
 383  399  
 384  400  bool ParallelCompactData::initialize(MemRegion covered_region)
 385  401  {
 386  402    _region_start = covered_region.start();
 387  403    const size_t region_size = covered_region.word_size();
 388  404    DEBUG_ONLY(_region_end = _region_start + region_size;)
 389  405  
 390  406    assert(region_align_down(_region_start) == _region_start,
 391  407           "region start not aligned");
 392  408    assert((region_size & RegionSizeOffsetMask) == 0,
 393  409           "region size not a multiple of RegionSize");
 394  410  
 395      -  bool result = initialize_region_data(region_size);
 396      -
      411 +  bool result = initialize_region_data(region_size) && initialize_block_data();
 397  412    return result;
 398  413  }
 399  414  
 400  415  PSVirtualSpace*
 401  416  ParallelCompactData::create_vspace(size_t count, size_t element_size)
 402  417  {
 403  418    const size_t raw_bytes = count * element_size;
 404  419    const size_t page_sz = os::page_size_for_region(raw_bytes, raw_bytes, 10);
 405  420    const size_t granularity = os::vm_allocation_granularity();
 406  421    const size_t bytes = align_size_up(raw_bytes, MAX2(page_sz, granularity));
↓ open down ↓ 24 lines elided ↑ open up ↑
 431  446    const size_t count = (region_size + RegionSizeOffsetMask) >> Log2RegionSize;
 432  447    _region_vspace = create_vspace(count, sizeof(RegionData));
 433  448    if (_region_vspace != 0) {
 434  449      _region_data = (RegionData*)_region_vspace->reserved_low_addr();
 435  450      _region_count = count;
 436  451      return true;
 437  452    }
 438  453    return false;
 439  454  }
 440  455  
      456 +bool ParallelCompactData::initialize_block_data()
      457 +{
      458 +  assert(_region_count != 0, "region data must be initialized first");
      459 +  const size_t count = _region_count << Log2BlocksPerRegion;
      460 +  _block_vspace = create_vspace(count, sizeof(BlockData));
      461 +  if (_block_vspace != 0) {
      462 +    _block_data = (BlockData*)_block_vspace->reserved_low_addr();
      463 +    _block_count = count;
      464 +    return true;
      465 +  }
      466 +  return false;
      467 +}
      468 +
 441  469  void ParallelCompactData::clear()
 442  470  {
 443  471    memset(_region_data, 0, _region_vspace->committed_size());
      472 +  memset(_block_data, 0, _block_vspace->committed_size());
 444  473  }
 445  474  
 446  475  void ParallelCompactData::clear_range(size_t beg_region, size_t end_region) {
 447  476    assert(beg_region <= _region_count, "beg_region out of range");
 448  477    assert(end_region <= _region_count, "end_region out of range");
      478 +  assert(RegionSize % BlockSize == 0, "RegionSize not a multiple of BlockSize");
 449  479  
 450  480    const size_t region_cnt = end_region - beg_region;
 451  481    memset(_region_data + beg_region, 0, region_cnt * sizeof(RegionData));
      482 +
      483 +  const size_t beg_block = beg_region * BlocksPerRegion;
      484 +  const size_t block_cnt = region_cnt * BlocksPerRegion;
      485 +  memset(_block_data + beg_block, 0, block_cnt * sizeof(BlockData));
 452  486  }
 453  487  
 454  488  HeapWord* ParallelCompactData::partial_obj_end(size_t region_idx) const
 455  489  {
 456  490    const RegionData* cur_cp = region(region_idx);
 457  491    const RegionData* const end_cp = region(region_count() - 1);
 458  492  
 459  493    HeapWord* result = region_to_addr(region_idx);
 460  494    if (cur_cp < end_cp) {
 461  495      do {
↓ open down ↓ 258 lines elided ↑ open up ↑
 720  754  
 721  755      ++cur_region;
 722  756    }
 723  757  
 724  758    *target_next = dest_addr;
 725  759    return true;
 726  760  }
 727  761  
 728  762  HeapWord* ParallelCompactData::calc_new_pointer(HeapWord* addr) {
 729  763    assert(addr != NULL, "Should detect NULL oop earlier");
 730      -  assert(PSParallelCompact::gc_heap()->is_in(addr), "addr not in heap");
 731      -#ifdef ASSERT
 732      -  if (PSParallelCompact::mark_bitmap()->is_unmarked(addr)) {
 733      -    gclog_or_tty->print_cr("calc_new_pointer:: addr " PTR_FORMAT, addr);
 734      -  }
 735      -#endif
 736      -  assert(PSParallelCompact::mark_bitmap()->is_marked(addr), "obj not marked");
      764 +  assert(PSParallelCompact::gc_heap()->is_in(addr), "not in heap");
      765 +  assert(PSParallelCompact::mark_bitmap()->is_marked(addr), "not marked");
 737  766  
 738  767    // Region covering the object.
 739      -  size_t region_index = addr_to_region_idx(addr);
 740      -  const RegionData* const region_ptr = region(region_index);
 741      -  HeapWord* const region_addr = region_align_down(addr);
 742      -
 743      -  assert(addr < region_addr + RegionSize, "Region does not cover object");
 744      -  assert(addr_to_region_ptr(region_addr) == region_ptr, "sanity check");
 745      -
      768 +  RegionData* const region_ptr = addr_to_region_ptr(addr);
 746  769    HeapWord* result = region_ptr->destination();
 747  770  
 748      -  // If all the data in the region is live, then the new location of the object
 749      -  // can be calculated from the destination of the region plus the offset of the
 750      -  // object in the region.
      771 +  // If the entire Region is live, the new location is region->destination + the
      772 +  // offset of the object within in the Region.
      773 +
      774 +  // Run some performance tests to determine if this special case pays off.  It
      775 +  // is worth it for pointers into the dense prefix.  If the optimization to
      776 +  // avoid pointer updates in regions that only point to the dense prefix is
      777 +  // ever implemented, this should be revisited.
 751  778    if (region_ptr->data_size() == RegionSize) {
 752      -    result += pointer_delta(addr, region_addr);
 753      -    DEBUG_ONLY(PSParallelCompact::check_new_location(addr, result);)
      779 +    result += region_offset(addr);
 754  780      return result;
 755  781    }
 756  782  
 757      -  // The new location of the object is
 758      -  //    region destination +
 759      -  //    size of the partial object extending onto the region +
 760      -  //    sizes of the live objects in the Region that are to the left of addr
 761      -  const size_t partial_obj_size = region_ptr->partial_obj_size();
 762      -  HeapWord* const search_start = region_addr + partial_obj_size;
      783 +  // Otherwise, the new location is region->destination + block offset + the
      784 +  // number of live words in the Block that are (a) to the left of addr and (b)
      785 +  // due to objects that start in the Block.
 763  786  
 764      -  const ParMarkBitMap* bitmap = PSParallelCompact::mark_bitmap();
 765      -  size_t live_to_left = bitmap->live_words_in_range(search_start, oop(addr));
      787 +  // Fill in the block table if necessary.  This is unsynchronized, so multiple
      788 +  // threads may fill the block table for a region (harmless, since it is
      789 +  // idempotent).
      790 +  if (!region_ptr->blocks_filled()) {
      791 +    PSParallelCompact::fill_blocks(addr_to_region_idx(addr));
      792 +    region_ptr->set_blocks_filled();
      793 +  }
 766  794  
 767      -  result += partial_obj_size + live_to_left;
 768      -  DEBUG_ONLY(PSParallelCompact::check_new_location(addr, result);)
      795 +  HeapWord* const search_start = block_align_down(addr);
      796 +  const size_t block_offset = addr_to_block_ptr(addr)->offset();
      797 +
      798 +  const ParMarkBitMap* bitmap = PSParallelCompact::mark_bitmap();
      799 +  const size_t live = bitmap->live_words_in_range(search_start, oop(addr));
      800 +  result += block_offset + live;
      801 +  DEBUG_ONLY(PSParallelCompact::check_new_location(addr, result));
 769  802    return result;
 770  803  }
 771  804  
 772  805  klassOop ParallelCompactData::calc_new_klass(klassOop old_klass) {
 773  806    klassOop updated_klass;
 774  807    if (PSParallelCompact::should_update_klass(old_klass)) {
 775  808      updated_klass = (klassOop) calc_new_pointer(old_klass);
 776  809    } else {
 777  810      updated_klass = old_klass;
 778  811    }
 779  812  
 780  813    return updated_klass;
 781  814  }
 782  815  
 783      -#ifdef  ASSERT
      816 +#ifdef ASSERT
 784  817  void ParallelCompactData::verify_clear(const PSVirtualSpace* vspace)
 785  818  {
 786  819    const size_t* const beg = (const size_t*)vspace->committed_low_addr();
 787  820    const size_t* const end = (const size_t*)vspace->committed_high_addr();
 788  821    for (const size_t* p = beg; p < end; ++p) {
 789  822      assert(*p == 0, "not zero");
 790  823    }
 791  824  }
 792  825  
 793  826  void ParallelCompactData::verify_clear()
 794  827  {
 795  828    verify_clear(_region_vspace);
      829 +  verify_clear(_block_vspace);
 796  830  }
 797  831  #endif  // #ifdef ASSERT
 798  832  
 799      -#ifdef NOT_PRODUCT
 800      -ParallelCompactData::RegionData* debug_region(size_t region_index) {
 801      -  ParallelCompactData& sd = PSParallelCompact::summary_data();
 802      -  return sd.region(region_index);
 803      -}
 804      -#endif
 805      -
 806  833  STWGCTimer          PSParallelCompact::_gc_timer;
 807  834  ParallelOldTracer   PSParallelCompact::_gc_tracer;
 808  835  elapsedTimer        PSParallelCompact::_accumulated_time;
 809  836  unsigned int        PSParallelCompact::_total_invocations = 0;
 810  837  unsigned int        PSParallelCompact::_maximum_compaction_gc_num = 0;
 811  838  jlong               PSParallelCompact::_time_of_last_gc = 0;
 812  839  CollectorCounters*  PSParallelCompact::_counters = NULL;
 813  840  ParMarkBitMap       PSParallelCompact::_mark_bitmap;
 814  841  ParallelCompactData PSParallelCompact::_summary_data;
 815  842  
↓ open down ↓ 1174 lines elided ↑ open up ↑
1990 2017      PSScavenge::invoke_no_policy();
1991 2018    }
1992 2019  
1993 2020    const bool clear_all_soft_refs =
1994 2021      heap->collector_policy()->should_clear_all_soft_refs();
1995 2022  
1996 2023    PSParallelCompact::invoke_no_policy(clear_all_soft_refs ||
1997 2024                                        maximum_heap_compaction);
1998 2025  }
1999 2026  
2000      -bool ParallelCompactData::region_contains(size_t region_index, HeapWord* addr) {
2001      -  size_t addr_region_index = addr_to_region_idx(addr);
2002      -  return region_index == addr_region_index;
2003      -}
2004      -
2005 2027  // This method contains no policy. You should probably
2006 2028  // be calling invoke() instead.
2007 2029  bool PSParallelCompact::invoke_no_policy(bool maximum_heap_compaction) {
2008 2030    assert(SafepointSynchronize::is_at_safepoint(), "must be at a safepoint");
2009 2031    assert(ref_processor() != NULL, "Sanity");
2010 2032  
2011 2033    if (GC_locker::check_active_before_gc()) {
2012 2034      return false;
2013 2035    }
2014 2036  
↓ open down ↓ 639 lines elided ↑ open up ↑
2654 2676  
2655 2677    // Once a thread has drained it's stack, it should try to steal regions from
2656 2678    // other threads.
2657 2679    if (parallel_gc_threads > 1) {
2658 2680      for (uint j = 0; j < parallel_gc_threads; j++) {
2659 2681        q->enqueue(new StealRegionCompactionTask(terminator_ptr));
2660 2682      }
2661 2683    }
2662 2684  }
2663 2685  
     2686 +#ifdef ASSERT
     2687 +// Write a histogram of the number of times the block table was filled for a
     2688 +// region.
     2689 +void PSParallelCompact::write_block_fill_histogram(outputStream* const out)
     2690 +{
     2691 +  if (!TraceParallelOldGCCompactionPhase) return;
     2692 +
     2693 +  typedef ParallelCompactData::RegionData rd_t;
     2694 +  ParallelCompactData& sd = summary_data();
     2695 +
     2696 +  for (unsigned int id = old_space_id; id < last_space_id; ++id) {
     2697 +    MutableSpace* const spc = _space_info[id].space();
     2698 +    if (spc->bottom() != spc->top()) {
     2699 +      const rd_t* const beg = sd.addr_to_region_ptr(spc->bottom());
     2700 +      HeapWord* const top_aligned_up = sd.region_align_up(spc->top());
     2701 +      const rd_t* const end = sd.addr_to_region_ptr(top_aligned_up);
     2702 +
     2703 +      size_t histo[5] = { 0, 0, 0, 0, 0 };
     2704 +      const size_t histo_len = sizeof(histo) / sizeof(size_t);
     2705 +      const size_t region_cnt = pointer_delta(end, beg, sizeof(rd_t));
     2706 +
     2707 +      for (const rd_t* cur = beg; cur < end; ++cur) {
     2708 +        ++histo[MIN2(cur->blocks_filled_count(), histo_len - 1)];
     2709 +      }
     2710 +      out->print("%u %-4s" SIZE_FORMAT_W(5), id, space_names[id], region_cnt);
     2711 +      for (size_t i = 0; i < histo_len; ++i) {
     2712 +        out->print(" " SIZE_FORMAT_W(5) " %5.1f%%",
     2713 +                   histo[i], 100.0 * histo[i] / region_cnt);
     2714 +      }
     2715 +      out->cr();
     2716 +    }
     2717 +  }
     2718 +}
     2719 +#endif // #ifdef ASSERT
     2720 +
2664 2721  void PSParallelCompact::compact() {
2665 2722    // trace("5");
2666 2723    GCTraceTime tm("compaction phase", print_phases(), true, &_gc_timer);
2667 2724  
2668 2725    ParallelScavengeHeap* heap = (ParallelScavengeHeap*)Universe::heap();
2669 2726    assert(heap->kind() == CollectedHeap::ParallelScavengeHeap, "Sanity");
2670 2727    PSOldGen* old_gen = heap->old_gen();
2671 2728    old_gen->start_array()->reset();
2672 2729    uint parallel_gc_threads = heap->gc_task_manager()->workers();
2673 2730    uint active_gc_threads = heap->gc_task_manager()->active_workers();
↓ open down ↓ 21 lines elided ↑ open up ↑
2695 2752    }
2696 2753  
2697 2754    {
2698 2755      // Update the deferred objects, if any.  Any compaction manager can be used.
2699 2756      GCTraceTime tm_du("deferred updates", print_phases(), true, &_gc_timer);
2700 2757      ParCompactionManager* cm = ParCompactionManager::manager_array(0);
2701 2758      for (unsigned int id = old_space_id; id < last_space_id; ++id) {
2702 2759        update_deferred_objects(cm, SpaceId(id));
2703 2760      }
2704 2761    }
     2762 +
     2763 +  DEBUG_ONLY(write_block_fill_histogram(gclog_or_tty));
2705 2764  }
2706 2765  
2707 2766  #ifdef  ASSERT
2708 2767  void PSParallelCompact::verify_complete(SpaceId space_id) {
2709 2768    // All Regions between space bottom() to new_top() should be marked as filled
2710 2769    // and all Regions between new_top() and top() should be available (i.e.,
2711 2770    // should have been emptied).
2712 2771    ParallelCompactData& sd = summary_data();
2713 2772    SpaceInfo si = _space_info[space_id];
2714 2773    HeapWord* new_top_addr = sd.region_align_up(si.new_top());
↓ open down ↓ 649 lines elided ↑ open up ↑
3364 3423  
3365 3424      decrement_destination_counts(cm, src_space_id, src_region_idx, end_addr);
3366 3425  
3367 3426      // Move to the next source region, possibly switching spaces as well.  All
3368 3427      // args except end_addr may be modified.
3369 3428      src_region_idx = next_src_region(closure, src_space_id, src_space_top,
3370 3429                                       end_addr);
3371 3430    } while (true);
3372 3431  }
3373 3432  
     3433 +void PSParallelCompact::fill_blocks(size_t region_idx)
     3434 +{
     3435 +  // Fill in the block table elements for the specified region.  Each block
     3436 +  // table element holds the number of live words in the region that are to the
     3437 +  // left of the first object that starts in the block.  Thus only blocks in
     3438 +  // which an object starts need to be filled.
     3439 +  //
     3440 +  // The algorithm scans the section of the bitmap that corresponds to the
     3441 +  // region, keeping a running total of the live words.  When an object start is
     3442 +  // found, if it's the first to start in the block that contains it, the
     3443 +  // current total is written to the block table element.
     3444 +  const size_t Log2BlockSize = ParallelCompactData::Log2BlockSize;
     3445 +  const size_t Log2RegionSize = ParallelCompactData::Log2RegionSize;
     3446 +  const size_t RegionSize = ParallelCompactData::RegionSize;
     3447 +
     3448 +  ParallelCompactData& sd = summary_data();
     3449 +  const size_t partial_obj_size = sd.region(region_idx)->partial_obj_size();
     3450 +  if (partial_obj_size >= RegionSize) {
     3451 +    return; // No objects start in this region.
     3452 +  }
     3453 +
     3454 +  // Ensure the first loop iteration decides that the block has changed.
     3455 +  size_t cur_block = sd.block_count();
     3456 +
     3457 +  const ParMarkBitMap* const bitmap = mark_bitmap();
     3458 +
     3459 +  const size_t Log2BitsPerBlock = Log2BlockSize - LogMinObjAlignment;
     3460 +  assert((size_t)1 << Log2BitsPerBlock ==
     3461 +         bitmap->words_to_bits(ParallelCompactData::BlockSize), "sanity");
     3462 +
     3463 +  size_t beg_bit = bitmap->words_to_bits(region_idx << Log2RegionSize);
     3464 +  const size_t range_end = beg_bit + bitmap->words_to_bits(RegionSize);
     3465 +  size_t live_bits = bitmap->words_to_bits(partial_obj_size);
     3466 +  beg_bit = bitmap->find_obj_beg(beg_bit + live_bits, range_end);
     3467 +  while (beg_bit < range_end) {
     3468 +    const size_t new_block = beg_bit >> Log2BitsPerBlock;
     3469 +    if (new_block != cur_block) {
     3470 +      cur_block = new_block;
     3471 +      sd.block(cur_block)->set_offset(bitmap->bits_to_words(live_bits));
     3472 +    }
     3473 +
     3474 +    const size_t end_bit = bitmap->find_obj_end(beg_bit, range_end);
     3475 +    if (end_bit < range_end - 1) {
     3476 +      live_bits += end_bit - beg_bit + 1;
     3477 +      beg_bit = bitmap->find_obj_beg(end_bit + 1, range_end);
     3478 +    } else {
     3479 +      return;
     3480 +    }
     3481 +  }
     3482 +}
     3483 +
3374 3484  void
3375 3485  PSParallelCompact::move_and_update(ParCompactionManager* cm, SpaceId space_id) {
3376 3486    const MutableSpace* sp = space(space_id);
3377 3487    if (sp->is_empty()) {
3378 3488      return;
3379 3489    }
3380 3490  
3381 3491    ParallelCompactData& sd = PSParallelCompact::summary_data();
3382 3492    ParMarkBitMap* const bitmap = mark_bitmap();
3383 3493    HeapWord* const dp_addr = dense_prefix(space_id);
↓ open down ↓ 135 lines elided ↑ open up ↑
XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX