< prev index next >

src/hotspot/share/gc/parallel/psParallelCompact.cpp

Print this page




 108   Log2RegionSize - Log2BlockSize;
 109 
 110 const ParallelCompactData::RegionData::region_sz_t
 111 ParallelCompactData::RegionData::dc_shift = 27;
 112 
 113 const ParallelCompactData::RegionData::region_sz_t
 114 ParallelCompactData::RegionData::dc_mask = ~0U << dc_shift;
 115 
 116 const ParallelCompactData::RegionData::region_sz_t
 117 ParallelCompactData::RegionData::dc_one = 0x1U << dc_shift;
 118 
 119 const ParallelCompactData::RegionData::region_sz_t
 120 ParallelCompactData::RegionData::los_mask = ~dc_mask;
 121 
 122 const ParallelCompactData::RegionData::region_sz_t
 123 ParallelCompactData::RegionData::dc_claimed = 0x8U << dc_shift;
 124 
 125 const ParallelCompactData::RegionData::region_sz_t
 126 ParallelCompactData::RegionData::dc_completed = 0xcU << dc_shift;
 127 





 128 SpaceInfo PSParallelCompact::_space_info[PSParallelCompact::last_space_id];
 129 
 130 SpanSubjectToDiscoveryClosure PSParallelCompact::_span_based_discoverer;
 131 ReferenceProcessor* PSParallelCompact::_ref_processor = NULL;
 132 
 133 double PSParallelCompact::_dwl_mean;
 134 double PSParallelCompact::_dwl_std_dev;
 135 double PSParallelCompact::_dwl_first_term;
 136 double PSParallelCompact::_dwl_adjustment;
 137 #ifdef  ASSERT
 138 bool   PSParallelCompact::_dwl_initialized = false;
 139 #endif  // #ifdef ASSERT
 140 
 141 void SplitInfo::record(size_t src_region_idx, size_t partial_obj_size,
 142                        HeapWord* destination)
 143 {
 144   assert(src_region_idx != 0, "invalid src_region_idx");
 145   assert(partial_obj_size != 0, "invalid partial_obj_size argument");
 146   assert(destination != NULL, "invalid destination argument");
 147 


1006   if (VerifyBeforeGC && heap->total_collections() >= VerifyGCStartAt) {
1007     HandleMark hm;  // Discard invalid handles created during verification
1008     Universe::verify("Before GC");
1009   }
1010 
1011   // Verify object start arrays
1012   if (VerifyObjectStartArray &&
1013       VerifyBeforeGC) {
1014     heap->old_gen()->verify_object_start_array();
1015   }
1016 
1017   DEBUG_ONLY(mark_bitmap()->verify_clear();)
1018   DEBUG_ONLY(summary_data().verify_clear();)
1019 
1020   ParCompactionManager::reset_all_bitmap_query_caches();
1021 }
1022 
1023 void PSParallelCompact::post_compact()
1024 {
1025   GCTraceTime(Info, gc, phases) tm("Post Compact", &_gc_timer);

1026 
1027   for (unsigned int id = old_space_id; id < last_space_id; ++id) {
1028     // Clear the marking bitmap, summary data and split info.
1029     clear_data_covering_space(SpaceId(id));
1030     // Update top().  Must be done after clearing the bitmap and summary data.
1031     _space_info[id].publish_new_top();
1032   }
1033 
1034   MutableSpace* const eden_space = _space_info[eden_space_id].space();
1035   MutableSpace* const from_space = _space_info[from_space_id].space();
1036   MutableSpace* const to_space   = _space_info[to_space_id].space();
1037 
1038   ParallelScavengeHeap* heap = ParallelScavengeHeap::heap();
1039   bool eden_empty = eden_space->is_empty();
1040   if (!eden_empty) {
1041     eden_empty = absorb_live_data_from_eden(heap->size_policy(),
1042                                             heap->young_gen(), heap->old_gen());
1043   }
1044 
1045   // Update heap occupancy information which is used as input to the soft ref


2400   // distribute them to the thread stacks.  The iteration is done in reverse
2401   // order (high to low) so the regions will be removed in ascending order.
2402 
2403   const ParallelCompactData& sd = PSParallelCompact::summary_data();
2404 
2405   // id + 1 is used to test termination so unsigned  can
2406   // be used with an old_space_id == 0.
2407   FillableRegionLogger region_logger;
2408   for (unsigned int id = to_space_id; id + 1 > old_space_id; --id) {
2409     SpaceInfo* const space_info = _space_info + id;
2410     MutableSpace* const space = space_info->space();
2411     HeapWord* const new_top = space_info->new_top();
2412 
2413     const size_t beg_region = sd.addr_to_region_idx(space_info->dense_prefix());
2414     const size_t end_region =
2415       sd.addr_to_region_idx(sd.region_align_up(new_top));
2416 
2417     for (size_t cur = end_region - 1; cur + 1 > beg_region; --cur) {
2418       if (sd.region(cur)->claim_unsafe()) {
2419         ParCompactionManager* cm = ParCompactionManager::manager_array(worker_id);

2420         cm->region_stack()->push(cur);
2421         region_logger.handle(cur);

2422         // Assign regions to tasks in round-robin fashion.
2423         if (++worker_id == parallel_gc_threads) {
2424           worker_id = 0;
2425         }
2426       }
2427     }
2428     region_logger.print_line();
2429   }
2430 }
2431 
2432 class TaskQueue : StackObj {
2433   volatile uint _counter;
2434   uint _size;
2435   uint _insert_index;
2436   PSParallelCompact::UpdateDensePrefixTask* _backing_array;
2437 public:
2438   explicit TaskQueue(uint size) : _counter(0), _size(size), _insert_index(0), _backing_array(NULL) {
2439     _backing_array = NEW_C_HEAP_ARRAY(PSParallelCompact::UpdateDensePrefixTask, _size, mtGC);
2440   }
2441   ~TaskQueue() {


2581     }
2582   }
2583 }
2584 #endif // #ifdef ASSERT
2585 
2586 static void compaction_with_stealing_work(ParallelTaskTerminator* terminator, uint worker_id) {
2587   assert(ParallelScavengeHeap::heap()->is_gc_active(), "called outside gc");
2588 
2589   ParCompactionManager* cm =
2590     ParCompactionManager::gc_thread_compaction_manager(worker_id);
2591 
2592   // Drain the stacks that have been preloaded with regions
2593   // that are ready to fill.
2594 
2595   cm->drain_region_stacks();
2596 
2597   guarantee(cm->region_stack()->is_empty(), "Not empty");
2598 
2599   size_t region_index = 0;
2600 

2601   while (true) {
2602     if (ParCompactionManager::steal(worker_id, region_index)) {
2603       PSParallelCompact::fill_and_update_region(cm, region_index);
2604       cm->drain_region_stacks();




2605     } else {
2606       if (terminator->offer_termination()) {
2607         break;
2608       }
2609       // Go around again.
2610     }
2611   }
2612   return;
2613 }
2614 
2615 class UpdateDensePrefixAndCompactionTask: public AbstractGangTask {
2616   typedef AbstractRefProcTaskExecutor::ProcessTask ProcessTask;
2617   TaskQueue& _tq;
2618   TaskTerminator _terminator;
2619   uint _active_workers;
2620 
2621 public:
2622   UpdateDensePrefixAndCompactionTask(TaskQueue& tq, uint active_workers) :
2623       AbstractGangTask("UpdateDensePrefixAndCompactionTask"),
2624       _tq(tq),


2639     // other threads.
2640     compaction_with_stealing_work(_terminator.terminator(), worker_id);
2641   }
2642 };
2643 
2644 void PSParallelCompact::compact() {
2645   GCTraceTime(Info, gc, phases) tm("Compaction Phase", &_gc_timer);
2646 
2647   ParallelScavengeHeap* heap = ParallelScavengeHeap::heap();
2648   PSOldGen* old_gen = heap->old_gen();
2649   old_gen->start_array()->reset();
2650   uint active_gc_threads = ParallelScavengeHeap::heap()->workers().active_workers();
2651 
2652   // for [0..last_space_id)
2653   //     for [0..active_gc_threads * PAR_OLD_DENSE_PREFIX_OVER_PARTITIONING)
2654   //         push
2655   //     push
2656   //
2657   // max push count is thus: last_space_id * (active_gc_threads * PAR_OLD_DENSE_PREFIX_OVER_PARTITIONING + 1)
2658   TaskQueue task_queue(last_space_id * (active_gc_threads * PAR_OLD_DENSE_PREFIX_OVER_PARTITIONING + 1));

2659   prepare_region_draining_tasks(active_gc_threads);
2660   enqueue_dense_prefix_tasks(task_queue, active_gc_threads);
2661 
2662   {
2663     GCTraceTime(Trace, gc, phases) tm("Par Compact", &_gc_timer);
2664 
2665     UpdateDensePrefixAndCompactionTask task(task_queue, active_gc_threads);
2666     ParallelScavengeHeap::heap()->workers().run_task(&task);
2667 
2668 #ifdef  ASSERT
2669     // Verify that all regions have been processed before the deferred updates.
2670     for (unsigned int id = old_space_id; id < last_space_id; ++id) {
2671       verify_complete(SpaceId(id));
2672     }
2673 #endif
2674   }
2675 
2676   {
2677     // Update the deferred objects, if any.  Any compaction manager can be used.
2678     GCTraceTime(Trace, gc, phases) tm("Deferred Updates", &_gc_timer);


2945   MutableSpace* const src_space = _space_info[src_space_id].space();
2946   HeapWord* const beg_addr = sd.region_to_addr(beg_region);
2947   assert(src_space->contains(beg_addr) || beg_addr == src_space->end(),
2948          "src_space_id does not match beg_addr");
2949   assert(src_space->contains(end_addr) || end_addr == src_space->end(),
2950          "src_space_id does not match end_addr");
2951 #endif // #ifdef ASSERT
2952 
2953   RegionData* const beg = sd.region(beg_region);
2954   RegionData* const end = sd.addr_to_region_ptr(sd.region_align_up(end_addr));
2955 
2956   // Regions up to new_top() are enqueued if they become available.
2957   HeapWord* const new_top = _space_info[src_space_id].new_top();
2958   RegionData* const enqueue_end =
2959     sd.addr_to_region_ptr(sd.region_align_up(new_top));
2960 
2961   for (RegionData* cur = beg; cur < end; ++cur) {
2962     assert(cur->data_size() > 0, "region must have live data");
2963     cur->decrement_destination_count();
2964     if (cur < enqueue_end && cur->available() && cur->claim()) {

2965       cm->push_region(sd.region(cur));







2966     }
2967   }
2968 }
2969 
2970 size_t PSParallelCompact::next_src_region(MoveAndUpdateClosure& closure,
2971                                           SpaceId& src_space_id,
2972                                           HeapWord*& src_space_top,
2973                                           HeapWord* end_addr)
2974 {
2975   typedef ParallelCompactData::RegionData RegionData;
2976 
2977   ParallelCompactData& sd = PSParallelCompact::summary_data();
2978   const size_t region_size = ParallelCompactData::RegionSize;
2979 
2980   size_t src_region_idx = 0;
2981 
2982   // Skip empty regions (if any) up to the top of the space.
2983   HeapWord* const src_aligned_up = sd.region_align_up(end_addr);
2984   RegionData* src_region_ptr = sd.addr_to_region_ptr(src_aligned_up);
2985   HeapWord* const top_aligned_up = sd.region_align_up(src_space_top);


3023                  "first live obj in the space must match the destination");
3024           assert(src_cp->partial_obj_size() == 0,
3025                  "a space cannot begin with a partial obj");
3026 
3027           src_space_id = SpaceId(space_id);
3028           src_space_top = space->top();
3029           const size_t src_region_idx = sd.region(src_cp);
3030           closure.set_source(sd.region_to_addr(src_region_idx));
3031           return src_region_idx;
3032         } else {
3033           assert(src_cp->data_size() == 0, "sanity");
3034         }
3035       }
3036     }
3037   } while (++space_id < last_space_id);
3038 
3039   assert(false, "no source region was found");
3040   return 0;
3041 }
3042 
3043 void PSParallelCompact::fill_region(ParCompactionManager* cm, size_t region_idx)
3044 {
3045   typedef ParMarkBitMap::IterationStatus IterationStatus;
3046   const size_t RegionSize = ParallelCompactData::RegionSize;
3047   ParMarkBitMap* const bitmap = mark_bitmap();
3048   ParallelCompactData& sd = summary_data();
3049   RegionData* const region_ptr = sd.region(region_idx);
3050 
3051   // Get the items needed to construct the closure.
3052   HeapWord* dest_addr = sd.region_to_addr(region_idx);
3053   SpaceId dest_space_id = space_id(dest_addr);
3054   ObjectStartArray* start_array = _space_info[dest_space_id].start_array();
3055   HeapWord* new_top = _space_info[dest_space_id].new_top();
3056   assert(dest_addr < new_top, "sanity");
3057   const size_t words = MIN2(pointer_delta(new_top, dest_addr), RegionSize);
3058 
3059   // Get the source region and related info.
3060   size_t src_region_idx = region_ptr->source_region();
3061   SpaceId src_space_id = space_id(sd.region_to_addr(src_region_idx));
3062   HeapWord* src_space_top = _space_info[src_space_id].space()->top();

3063 
3064   MoveAndUpdateClosure closure(bitmap, cm, start_array, dest_addr, words);
3065   closure.set_source(first_src_addr(dest_addr, src_space_id, src_region_idx));
3066 
3067   // Adjust src_region_idx to prepare for decrementing destination counts (the
3068   // destination count is not decremented when a region is copied to itself).
3069   if (src_region_idx == region_idx) {
3070     src_region_idx += 1;
3071   }
3072 
3073   if (bitmap->is_unmarked(closure.source())) {
3074     // The first source word is in the middle of an object; copy the remainder
3075     // of the object or as much as will fit.  The fact that pointer updates were
3076     // deferred will be noted when the object header is processed.
3077     HeapWord* const old_src_addr = closure.source();
3078     closure.copy_partial_obj();
3079     if (closure.is_full()) {
3080       decrement_destination_counts(cm, src_space_id, src_region_idx,
3081                                    closure.source());
3082       region_ptr->set_deferred_obj_addr(NULL);
3083       region_ptr->set_completed();
3084       return;
3085     }
3086 
3087     HeapWord* const end_addr = sd.region_align_down(closure.source());
3088     if (sd.region_align_down(old_src_addr) != end_addr) {
3089       // The partial object was copied from more than one source region.
3090       decrement_destination_counts(cm, src_space_id, src_region_idx, end_addr);
3091 
3092       // Move to the next source region, possibly switching spaces as well.  All
3093       // args except end_addr may be modified.
3094       src_region_idx = next_src_region(closure, src_space_id, src_space_top,
3095                                        end_addr);
3096     }
3097   }
3098 
3099   do {
3100     HeapWord* const cur_addr = closure.source();
3101     HeapWord* const end_addr = MIN2(sd.region_align_up(cur_addr + 1),
3102                                     src_space_top);
3103     IterationStatus status = bitmap->iterate(&closure, cur_addr, end_addr);


3108       assert(closure.source() < end_addr, "sanity");
3109       HeapWord* const obj_beg = closure.source();
3110       HeapWord* const range_end = MIN2(obj_beg + closure.words_remaining(),
3111                                        src_space_top);
3112       HeapWord* const obj_end = bitmap->find_obj_end(obj_beg, range_end);
3113       if (obj_end < range_end) {
3114         // The end was found; the entire object will fit.
3115         status = closure.do_addr(obj_beg, bitmap->obj_size(obj_beg, obj_end));
3116         assert(status != ParMarkBitMap::would_overflow, "sanity");
3117       } else {
3118         // The end was not found; the object will not fit.
3119         assert(range_end < src_space_top, "obj cannot cross space boundary");
3120         status = ParMarkBitMap::would_overflow;
3121       }
3122     }
3123 
3124     if (status == ParMarkBitMap::would_overflow) {
3125       // The last object did not fit.  Note that interior oop updates were
3126       // deferred, then copy enough of the object to fill the region.
3127       region_ptr->set_deferred_obj_addr(closure.destination());

3128       status = closure.copy_until_full(); // copies from closure.source()
3129 
3130       decrement_destination_counts(cm, src_space_id, src_region_idx,
3131                                    closure.source());
3132       region_ptr->set_completed();
3133       return;
3134     }
3135 
3136     if (status == ParMarkBitMap::full) {
3137       decrement_destination_counts(cm, src_space_id, src_region_idx,
3138                                    closure.source());
3139       region_ptr->set_deferred_obj_addr(NULL);
3140       region_ptr->set_completed();
3141       return;
3142     }
3143 
3144     decrement_destination_counts(cm, src_space_id, src_region_idx, end_addr);
3145 
3146     // Move to the next source region, possibly switching spaces as well.  All
3147     // args except end_addr may be modified.
3148     src_region_idx = next_src_region(closure, src_space_id, src_space_top,
3149                                      end_addr);
3150   } while (true);
3151 }
3152 






































































3153 void PSParallelCompact::fill_blocks(size_t region_idx)
3154 {
3155   // Fill in the block table elements for the specified region.  Each block
3156   // table element holds the number of live words in the region that are to the
3157   // left of the first object that starts in the block.  Thus only blocks in
3158   // which an object starts need to be filled.
3159   //
3160   // The algorithm scans the section of the bitmap that corresponds to the
3161   // region, keeping a running total of the live words.  When an object start is
3162   // found, if it's the first to start in the block that contains it, the
3163   // current total is written to the block table element.
3164   const size_t Log2BlockSize = ParallelCompactData::Log2BlockSize;
3165   const size_t Log2RegionSize = ParallelCompactData::Log2RegionSize;
3166   const size_t RegionSize = ParallelCompactData::RegionSize;
3167 
3168   ParallelCompactData& sd = summary_data();
3169   const size_t partial_obj_size = sd.region(region_idx)->partial_obj_size();
3170   if (partial_obj_size >= RegionSize) {
3171     return; // No objects start in this region.
3172   }


3205   // We need a monotonically non-decreasing time in ms but
3206   // os::javaTimeMillis() does not guarantee monotonicity.
3207   jlong now = os::javaTimeNanos() / NANOSECS_PER_MILLISEC;
3208   jlong ret_val = now - _time_of_last_gc;
3209   // XXX See note in genCollectedHeap::millis_since_last_gc().
3210   if (ret_val < 0) {
3211     NOT_PRODUCT(log_warning(gc)("time warp: " JLONG_FORMAT, ret_val);)
3212     return 0;
3213   }
3214   return ret_val;
3215 }
3216 
3217 void PSParallelCompact::reset_millis_since_last_gc() {
3218   // We need a monotonically non-decreasing time in ms but
3219   // os::javaTimeMillis() does not guarantee monotonicity.
3220   _time_of_last_gc = os::javaTimeNanos() / NANOSECS_PER_MILLISEC;
3221 }
3222 
3223 ParMarkBitMap::IterationStatus MoveAndUpdateClosure::copy_until_full()
3224 {
3225   if (source() != destination()) {
3226     DEBUG_ONLY(PSParallelCompact::check_new_location(source(), destination());)
3227     Copy::aligned_conjoint_words(source(), destination(), words_remaining());
3228   }
3229   update_state(words_remaining());
3230   assert(is_full(), "sanity");
3231   return ParMarkBitMap::full;
3232 }
3233 
3234 void MoveAndUpdateClosure::copy_partial_obj()
3235 {
3236   size_t words = words_remaining();
3237 
3238   HeapWord* const range_end = MIN2(source() + words, bitmap()->region_end());
3239   HeapWord* const end_addr = bitmap()->find_obj_end(source(), range_end);
3240   if (end_addr < range_end) {
3241     words = bitmap()->obj_size(source(), end_addr);
3242   }
3243 
3244   // This test is necessary; if omitted, the pointer updates to a partial object
3245   // that crosses the dense prefix boundary could be overwritten.
3246   if (source() != destination()) {
3247     DEBUG_ONLY(PSParallelCompact::check_new_location(source(), destination());)
3248     Copy::aligned_conjoint_words(source(), destination(), words);
3249   }
3250   update_state(words);
3251 }
3252 






3253 ParMarkBitMapClosure::IterationStatus
3254 MoveAndUpdateClosure::do_addr(HeapWord* addr, size_t words) {
3255   assert(destination() != NULL, "sanity");
3256   assert(bitmap()->obj_size(addr) == words, "bad size");
3257 
3258   _source = addr;
3259   assert(PSParallelCompact::summary_data().calc_new_pointer(source(), compaction_manager()) ==
3260          destination(), "wrong destination");
3261 
3262   if (words > words_remaining()) {
3263     return ParMarkBitMap::would_overflow;
3264   }
3265 
3266   // The start_array must be updated even if the object is not moving.
3267   if (_start_array != NULL) {
3268     _start_array->allocate_block(destination());
3269   }
3270 
3271   if (destination() != source()) {
3272     DEBUG_ONLY(PSParallelCompact::check_new_location(source(), destination());)
3273     Copy::aligned_conjoint_words(source(), destination(), words);
3274   }
3275 
3276   oop moved_oop = (oop) destination();
3277   compaction_manager()->update_contents(moved_oop);
3278   assert(oopDesc::is_oop_or_null(moved_oop), "Expected an oop or NULL at " PTR_FORMAT, p2i(moved_oop));
3279 
3280   update_state(words);
3281   assert(destination() == (HeapWord*)moved_oop + moved_oop->size(), "sanity");
3282   return is_full() ? ParMarkBitMap::full : ParMarkBitMap::incomplete;
















3283 }
3284 
3285 UpdateOnlyClosure::UpdateOnlyClosure(ParMarkBitMap* mbm,
3286                                      ParCompactionManager* cm,
3287                                      PSParallelCompact::SpaceId space_id) :
3288   ParMarkBitMapClosure(mbm, cm),
3289   _space_id(space_id),
3290   _start_array(PSParallelCompact::start_array(space_id))
3291 {
3292 }
3293 
3294 // Updates the references in the object to their new values.
3295 ParMarkBitMapClosure::IterationStatus
3296 UpdateOnlyClosure::do_addr(HeapWord* addr, size_t words) {
3297   do_addr(addr);
3298   return ParMarkBitMap::incomplete;
3299 }
3300 
3301 FillClosure::FillClosure(ParCompactionManager* cm, PSParallelCompact::SpaceId space_id) :
3302   ParMarkBitMapClosure(PSParallelCompact::mark_bitmap(), cm),


 108   Log2RegionSize - Log2BlockSize;
 109 
 110 const ParallelCompactData::RegionData::region_sz_t
 111 ParallelCompactData::RegionData::dc_shift = 27;
 112 
 113 const ParallelCompactData::RegionData::region_sz_t
 114 ParallelCompactData::RegionData::dc_mask = ~0U << dc_shift;
 115 
 116 const ParallelCompactData::RegionData::region_sz_t
 117 ParallelCompactData::RegionData::dc_one = 0x1U << dc_shift;
 118 
 119 const ParallelCompactData::RegionData::region_sz_t
 120 ParallelCompactData::RegionData::los_mask = ~dc_mask;
 121 
 122 const ParallelCompactData::RegionData::region_sz_t
 123 ParallelCompactData::RegionData::dc_claimed = 0x8U << dc_shift;
 124 
 125 const ParallelCompactData::RegionData::region_sz_t
 126 ParallelCompactData::RegionData::dc_completed = 0xcU << dc_shift;
 127 
 128 const int ParallelCompactData::RegionData::UNUSED = 0;
 129 const int ParallelCompactData::RegionData::SHADOW = 1;
 130 const int ParallelCompactData::RegionData::FILLED = 2;
 131 const int ParallelCompactData::RegionData::FINISH = 3;
 132 
 133 SpaceInfo PSParallelCompact::_space_info[PSParallelCompact::last_space_id];
 134 
 135 SpanSubjectToDiscoveryClosure PSParallelCompact::_span_based_discoverer;
 136 ReferenceProcessor* PSParallelCompact::_ref_processor = NULL;
 137 
 138 double PSParallelCompact::_dwl_mean;
 139 double PSParallelCompact::_dwl_std_dev;
 140 double PSParallelCompact::_dwl_first_term;
 141 double PSParallelCompact::_dwl_adjustment;
 142 #ifdef  ASSERT
 143 bool   PSParallelCompact::_dwl_initialized = false;
 144 #endif  // #ifdef ASSERT
 145 
 146 void SplitInfo::record(size_t src_region_idx, size_t partial_obj_size,
 147                        HeapWord* destination)
 148 {
 149   assert(src_region_idx != 0, "invalid src_region_idx");
 150   assert(partial_obj_size != 0, "invalid partial_obj_size argument");
 151   assert(destination != NULL, "invalid destination argument");
 152 


1011   if (VerifyBeforeGC && heap->total_collections() >= VerifyGCStartAt) {
1012     HandleMark hm;  // Discard invalid handles created during verification
1013     Universe::verify("Before GC");
1014   }
1015 
1016   // Verify object start arrays
1017   if (VerifyObjectStartArray &&
1018       VerifyBeforeGC) {
1019     heap->old_gen()->verify_object_start_array();
1020   }
1021 
1022   DEBUG_ONLY(mark_bitmap()->verify_clear();)
1023   DEBUG_ONLY(summary_data().verify_clear();)
1024 
1025   ParCompactionManager::reset_all_bitmap_query_caches();
1026 }
1027 
1028 void PSParallelCompact::post_compact()
1029 {
1030   GCTraceTime(Info, gc, phases) tm("Post Compact", &_gc_timer);
1031   ParCompactionManager::dequeue_shadow_region();
1032 
1033   for (unsigned int id = old_space_id; id < last_space_id; ++id) {
1034     // Clear the marking bitmap, summary data and split info.
1035     clear_data_covering_space(SpaceId(id));
1036     // Update top().  Must be done after clearing the bitmap and summary data.
1037     _space_info[id].publish_new_top();
1038   }
1039 
1040   MutableSpace* const eden_space = _space_info[eden_space_id].space();
1041   MutableSpace* const from_space = _space_info[from_space_id].space();
1042   MutableSpace* const to_space   = _space_info[to_space_id].space();
1043 
1044   ParallelScavengeHeap* heap = ParallelScavengeHeap::heap();
1045   bool eden_empty = eden_space->is_empty();
1046   if (!eden_empty) {
1047     eden_empty = absorb_live_data_from_eden(heap->size_policy(),
1048                                             heap->young_gen(), heap->old_gen());
1049   }
1050 
1051   // Update heap occupancy information which is used as input to the soft ref


2406   // distribute them to the thread stacks.  The iteration is done in reverse
2407   // order (high to low) so the regions will be removed in ascending order.
2408 
2409   const ParallelCompactData& sd = PSParallelCompact::summary_data();
2410 
2411   // id + 1 is used to test termination so unsigned  can
2412   // be used with an old_space_id == 0.
2413   FillableRegionLogger region_logger;
2414   for (unsigned int id = to_space_id; id + 1 > old_space_id; --id) {
2415     SpaceInfo* const space_info = _space_info + id;
2416     MutableSpace* const space = space_info->space();
2417     HeapWord* const new_top = space_info->new_top();
2418 
2419     const size_t beg_region = sd.addr_to_region_idx(space_info->dense_prefix());
2420     const size_t end_region =
2421       sd.addr_to_region_idx(sd.region_align_up(new_top));
2422 
2423     for (size_t cur = end_region - 1; cur + 1 > beg_region; --cur) {
2424       if (sd.region(cur)->claim_unsafe()) {
2425         ParCompactionManager* cm = ParCompactionManager::manager_array(worker_id);
2426         if (sd.region(cur)->try_push()) {
2427           cm->region_stack()->push(cur);
2428           region_logger.handle(cur);
2429         }
2430         // Assign regions to tasks in round-robin fashion.
2431         if (++worker_id == parallel_gc_threads) {
2432           worker_id = 0;
2433         }
2434       }
2435     }
2436     region_logger.print_line();
2437   }
2438 }
2439 
2440 class TaskQueue : StackObj {
2441   volatile uint _counter;
2442   uint _size;
2443   uint _insert_index;
2444   PSParallelCompact::UpdateDensePrefixTask* _backing_array;
2445 public:
2446   explicit TaskQueue(uint size) : _counter(0), _size(size), _insert_index(0), _backing_array(NULL) {
2447     _backing_array = NEW_C_HEAP_ARRAY(PSParallelCompact::UpdateDensePrefixTask, _size, mtGC);
2448   }
2449   ~TaskQueue() {


2589     }
2590   }
2591 }
2592 #endif // #ifdef ASSERT
2593 
2594 static void compaction_with_stealing_work(ParallelTaskTerminator* terminator, uint worker_id) {
2595   assert(ParallelScavengeHeap::heap()->is_gc_active(), "called outside gc");
2596 
2597   ParCompactionManager* cm =
2598     ParCompactionManager::gc_thread_compaction_manager(worker_id);
2599 
2600   // Drain the stacks that have been preloaded with regions
2601   // that are ready to fill.
2602 
2603   cm->drain_region_stacks();
2604 
2605   guarantee(cm->region_stack()->is_empty(), "Not empty");
2606 
2607   size_t region_index = 0;
2608 
2609   PSParallelCompact::initialize_steal_record(worker_id);
2610   while (true) {
2611     if (ParCompactionManager::steal(worker_id, region_index)) {
2612       PSParallelCompact::fill_and_update_region(cm, region_index);
2613       cm->drain_region_stacks();
2614     } else if (PSParallelCompact::steal_shadow_region(cm, region_index)) {
2615       // Keep working with the help of shadow regions
2616       PSParallelCompact::fill_and_update_shadow_region(cm, region_index);
2617       cm->drain_region_stacks();
2618     } else {
2619       if (terminator->offer_termination()) {
2620         break;
2621       }
2622       // Go around again.
2623     }
2624   }
2625   return;
2626 }
2627 
2628 class UpdateDensePrefixAndCompactionTask: public AbstractGangTask {
2629   typedef AbstractRefProcTaskExecutor::ProcessTask ProcessTask;
2630   TaskQueue& _tq;
2631   TaskTerminator _terminator;
2632   uint _active_workers;
2633 
2634 public:
2635   UpdateDensePrefixAndCompactionTask(TaskQueue& tq, uint active_workers) :
2636       AbstractGangTask("UpdateDensePrefixAndCompactionTask"),
2637       _tq(tq),


2652     // other threads.
2653     compaction_with_stealing_work(_terminator.terminator(), worker_id);
2654   }
2655 };
2656 
2657 void PSParallelCompact::compact() {
2658   GCTraceTime(Info, gc, phases) tm("Compaction Phase", &_gc_timer);
2659 
2660   ParallelScavengeHeap* heap = ParallelScavengeHeap::heap();
2661   PSOldGen* old_gen = heap->old_gen();
2662   old_gen->start_array()->reset();
2663   uint active_gc_threads = ParallelScavengeHeap::heap()->workers().active_workers();
2664 
2665   // for [0..last_space_id)
2666   //     for [0..active_gc_threads * PAR_OLD_DENSE_PREFIX_OVER_PARTITIONING)
2667   //         push
2668   //     push
2669   //
2670   // max push count is thus: last_space_id * (active_gc_threads * PAR_OLD_DENSE_PREFIX_OVER_PARTITIONING + 1)
2671   TaskQueue task_queue(last_space_id * (active_gc_threads * PAR_OLD_DENSE_PREFIX_OVER_PARTITIONING + 1));
2672   enqueue_shadow_region();
2673   prepare_region_draining_tasks(active_gc_threads);
2674   enqueue_dense_prefix_tasks(task_queue, active_gc_threads);
2675 
2676   {
2677     GCTraceTime(Trace, gc, phases) tm("Par Compact", &_gc_timer);
2678 
2679     UpdateDensePrefixAndCompactionTask task(task_queue, active_gc_threads);
2680     ParallelScavengeHeap::heap()->workers().run_task(&task);
2681 
2682 #ifdef  ASSERT
2683     // Verify that all regions have been processed before the deferred updates.
2684     for (unsigned int id = old_space_id; id < last_space_id; ++id) {
2685       verify_complete(SpaceId(id));
2686     }
2687 #endif
2688   }
2689 
2690   {
2691     // Update the deferred objects, if any.  Any compaction manager can be used.
2692     GCTraceTime(Trace, gc, phases) tm("Deferred Updates", &_gc_timer);


2959   MutableSpace* const src_space = _space_info[src_space_id].space();
2960   HeapWord* const beg_addr = sd.region_to_addr(beg_region);
2961   assert(src_space->contains(beg_addr) || beg_addr == src_space->end(),
2962          "src_space_id does not match beg_addr");
2963   assert(src_space->contains(end_addr) || end_addr == src_space->end(),
2964          "src_space_id does not match end_addr");
2965 #endif // #ifdef ASSERT
2966 
2967   RegionData* const beg = sd.region(beg_region);
2968   RegionData* const end = sd.addr_to_region_ptr(sd.region_align_up(end_addr));
2969 
2970   // Regions up to new_top() are enqueued if they become available.
2971   HeapWord* const new_top = _space_info[src_space_id].new_top();
2972   RegionData* const enqueue_end =
2973     sd.addr_to_region_ptr(sd.region_align_up(new_top));
2974 
2975   for (RegionData* cur = beg; cur < end; ++cur) {
2976     assert(cur->data_size() > 0, "region must have live data");
2977     cur->decrement_destination_count();
2978     if (cur < enqueue_end && cur->available() && cur->claim()) {
2979       if (cur->try_push()) {
2980         cm->push_region(sd.region(cur));
2981       } else if (cur->try_copy()) {
2982         // Try to copy the content of the shadow region back to its corresponding
2983         // heap region if the shadow region is filled
2984         copy_back(sd.region_to_addr(cur->shadow_region()), sd.region_to_addr(cur));
2985         cm->release_shadow_region(cur->shadow_region());
2986         cur->set_completed();
2987       }
2988     }
2989   }
2990 }
2991 
2992 size_t PSParallelCompact::next_src_region(MoveAndUpdateClosure& closure,
2993                                           SpaceId& src_space_id,
2994                                           HeapWord*& src_space_top,
2995                                           HeapWord* end_addr)
2996 {
2997   typedef ParallelCompactData::RegionData RegionData;
2998 
2999   ParallelCompactData& sd = PSParallelCompact::summary_data();
3000   const size_t region_size = ParallelCompactData::RegionSize;
3001 
3002   size_t src_region_idx = 0;
3003 
3004   // Skip empty regions (if any) up to the top of the space.
3005   HeapWord* const src_aligned_up = sd.region_align_up(end_addr);
3006   RegionData* src_region_ptr = sd.addr_to_region_ptr(src_aligned_up);
3007   HeapWord* const top_aligned_up = sd.region_align_up(src_space_top);


3045                  "first live obj in the space must match the destination");
3046           assert(src_cp->partial_obj_size() == 0,
3047                  "a space cannot begin with a partial obj");
3048 
3049           src_space_id = SpaceId(space_id);
3050           src_space_top = space->top();
3051           const size_t src_region_idx = sd.region(src_cp);
3052           closure.set_source(sd.region_to_addr(src_region_idx));
3053           return src_region_idx;
3054         } else {
3055           assert(src_cp->data_size() == 0, "sanity");
3056         }
3057       }
3058     }
3059   } while (++space_id < last_space_id);
3060 
3061   assert(false, "no source region was found");
3062   return 0;
3063 }
3064 
3065 void PSParallelCompact::fill_region(ParCompactionManager* cm, MoveAndUpdateClosure& closure, size_t region_idx)
3066 {
3067   typedef ParMarkBitMap::IterationStatus IterationStatus;
3068   const size_t RegionSize = ParallelCompactData::RegionSize;
3069   ParMarkBitMap* const bitmap = mark_bitmap();
3070   ParallelCompactData& sd = summary_data();
3071   RegionData* const region_ptr = sd.region(region_idx);
3072 








3073   // Get the source region and related info.
3074   size_t src_region_idx = region_ptr->source_region();
3075   SpaceId src_space_id = space_id(sd.region_to_addr(src_region_idx));
3076   HeapWord* src_space_top = _space_info[src_space_id].space()->top();
3077   HeapWord* dest_addr = sd.region_to_addr(region_idx);
3078 

3079   closure.set_source(first_src_addr(dest_addr, src_space_id, src_region_idx));
3080 
3081   // Adjust src_region_idx to prepare for decrementing destination counts (the
3082   // destination count is not decremented when a region is copied to itself).
3083   if (src_region_idx == region_idx) {
3084     src_region_idx += 1;
3085   }
3086 
3087   if (bitmap->is_unmarked(closure.source())) {
3088     // The first source word is in the middle of an object; copy the remainder
3089     // of the object or as much as will fit.  The fact that pointer updates were
3090     // deferred will be noted when the object header is processed.
3091     HeapWord* const old_src_addr = closure.source();
3092     closure.copy_partial_obj();
3093     if (closure.is_full()) {
3094       decrement_destination_counts(cm, src_space_id, src_region_idx,
3095                                    closure.source());
3096       region_ptr->set_deferred_obj_addr(NULL);
3097       closure.complete_region(cm, dest_addr, region_ptr);
3098       return;
3099     }
3100 
3101     HeapWord* const end_addr = sd.region_align_down(closure.source());
3102     if (sd.region_align_down(old_src_addr) != end_addr) {
3103       // The partial object was copied from more than one source region.
3104       decrement_destination_counts(cm, src_space_id, src_region_idx, end_addr);
3105 
3106       // Move to the next source region, possibly switching spaces as well.  All
3107       // args except end_addr may be modified.
3108       src_region_idx = next_src_region(closure, src_space_id, src_space_top,
3109                                        end_addr);
3110     }
3111   }
3112 
3113   do {
3114     HeapWord* const cur_addr = closure.source();
3115     HeapWord* const end_addr = MIN2(sd.region_align_up(cur_addr + 1),
3116                                     src_space_top);
3117     IterationStatus status = bitmap->iterate(&closure, cur_addr, end_addr);


3122       assert(closure.source() < end_addr, "sanity");
3123       HeapWord* const obj_beg = closure.source();
3124       HeapWord* const range_end = MIN2(obj_beg + closure.words_remaining(),
3125                                        src_space_top);
3126       HeapWord* const obj_end = bitmap->find_obj_end(obj_beg, range_end);
3127       if (obj_end < range_end) {
3128         // The end was found; the entire object will fit.
3129         status = closure.do_addr(obj_beg, bitmap->obj_size(obj_beg, obj_end));
3130         assert(status != ParMarkBitMap::would_overflow, "sanity");
3131       } else {
3132         // The end was not found; the object will not fit.
3133         assert(range_end < src_space_top, "obj cannot cross space boundary");
3134         status = ParMarkBitMap::would_overflow;
3135       }
3136     }
3137 
3138     if (status == ParMarkBitMap::would_overflow) {
3139       // The last object did not fit.  Note that interior oop updates were
3140       // deferred, then copy enough of the object to fill the region.
3141       region_ptr->set_deferred_obj_addr(closure.destination());
3142 
3143       status = closure.copy_until_full(); // copies from closure.source()
3144 
3145       decrement_destination_counts(cm, src_space_id, src_region_idx,
3146                                    closure.source());
3147       closure.complete_region(cm, dest_addr, region_ptr);
3148       return;
3149     }
3150 
3151     if (status == ParMarkBitMap::full) {
3152       decrement_destination_counts(cm, src_space_id, src_region_idx,
3153                                    closure.source());
3154       region_ptr->set_deferred_obj_addr(NULL);
3155       closure.complete_region(cm, dest_addr, region_ptr);
3156       return;
3157     }
3158 
3159     decrement_destination_counts(cm, src_space_id, src_region_idx, end_addr);
3160 
3161     // Move to the next source region, possibly switching spaces as well.  All
3162     // args except end_addr may be modified.
3163     src_region_idx = next_src_region(closure, src_space_id, src_space_top,
3164                                      end_addr);
3165   } while (true);
3166 }
3167 
3168 void PSParallelCompact::fill_and_update_region(ParCompactionManager* cm, size_t region_idx) {
3169   MoveAndUpdateClosure cl(mark_bitmap(), cm, region_idx);
3170   fill_region(cm, cl, region_idx);
3171 }
3172 
3173 void PSParallelCompact::fill_shadow_region(ParCompactionManager* cm, size_t region_idx)
3174 {
3175   // Acquire a shadow region at first
3176   ParallelCompactData& sd = summary_data();
3177   RegionData* const region_ptr = sd.region(region_idx);
3178   size_t shadow_region = cm->acquire_shadow_region(region_ptr);
3179   // The zero return value indicates the corresponding heap region is available,
3180   // so use MoveAndUpdateClosure to fill the normal region. Otherwise, use
3181   // ShadowClosure to fill the acquired shadow region.
3182   if (shadow_region == 0) {
3183     MoveAndUpdateClosure cl(mark_bitmap(), cm, region_idx);
3184     region_ptr->mark_normal();
3185     return fill_region(cm, cl, region_idx);
3186   } else {
3187     ShadowClosure cl(mark_bitmap(), cm, region_idx, shadow_region);
3188     return fill_region(cm, cl, region_idx);
3189   }
3190 }
3191 
3192 void PSParallelCompact::copy_back(HeapWord *shadow_addr, HeapWord *region_addr) {
3193   Copy::aligned_conjoint_words(shadow_addr, region_addr, _summary_data.RegionSize);
3194 }
3195 
3196 bool PSParallelCompact::steal_shadow_region(ParCompactionManager* cm, size_t &region_idx) {
3197   size_t record = cm->shadow_record();
3198   ParallelCompactData& sd = _summary_data;
3199   size_t old_new_top = sd.addr_to_region_idx(_space_info[old_space_id].new_top());
3200   uint active_gc_threads = ParallelScavengeHeap::heap()->workers().active_workers();
3201 
3202   while (record < old_new_top) {
3203     if (sd.region(record)->try_steal()) {
3204       region_idx = record;
3205       return true;
3206     }
3207     record = cm->next_shadow_record(active_gc_threads);
3208   }
3209 
3210   return false;
3211 }
3212 
3213 void PSParallelCompact::enqueue_shadow_region() {
3214   const ParallelCompactData& sd = PSParallelCompact::summary_data();
3215 
3216   for (unsigned int id = old_space_id; id < last_space_id; ++id) {
3217     SpaceInfo* const space_info = _space_info + id;
3218     MutableSpace* const space = space_info->space();
3219 
3220     const size_t beg_region =
3221       sd.addr_to_region_idx(sd.region_align_up(MAX2(space_info->new_top(), space->top())));
3222     const size_t end_region =
3223       sd.addr_to_region_idx(sd.region_align_down(space->end()));
3224 
3225     for (size_t cur = beg_region + 1; cur < end_region; ++cur) {
3226       ParCompactionManager::enqueue_shadow_region(cur);
3227     }
3228   }
3229 }
3230 
3231 void PSParallelCompact::initialize_steal_record(uint which) {
3232   ParCompactionManager* cm = ParCompactionManager::gc_thread_compaction_manager(which);
3233 
3234   size_t record = _summary_data.addr_to_region_idx(_space_info[old_space_id].dense_prefix());
3235   cm->set_shadow_record(record + which);
3236 }
3237 
3238 void PSParallelCompact::fill_blocks(size_t region_idx)
3239 {
3240   // Fill in the block table elements for the specified region.  Each block
3241   // table element holds the number of live words in the region that are to the
3242   // left of the first object that starts in the block.  Thus only blocks in
3243   // which an object starts need to be filled.
3244   //
3245   // The algorithm scans the section of the bitmap that corresponds to the
3246   // region, keeping a running total of the live words.  When an object start is
3247   // found, if it's the first to start in the block that contains it, the
3248   // current total is written to the block table element.
3249   const size_t Log2BlockSize = ParallelCompactData::Log2BlockSize;
3250   const size_t Log2RegionSize = ParallelCompactData::Log2RegionSize;
3251   const size_t RegionSize = ParallelCompactData::RegionSize;
3252 
3253   ParallelCompactData& sd = summary_data();
3254   const size_t partial_obj_size = sd.region(region_idx)->partial_obj_size();
3255   if (partial_obj_size >= RegionSize) {
3256     return; // No objects start in this region.
3257   }


3290   // We need a monotonically non-decreasing time in ms but
3291   // os::javaTimeMillis() does not guarantee monotonicity.
3292   jlong now = os::javaTimeNanos() / NANOSECS_PER_MILLISEC;
3293   jlong ret_val = now - _time_of_last_gc;
3294   // XXX See note in genCollectedHeap::millis_since_last_gc().
3295   if (ret_val < 0) {
3296     NOT_PRODUCT(log_warning(gc)("time warp: " JLONG_FORMAT, ret_val);)
3297     return 0;
3298   }
3299   return ret_val;
3300 }
3301 
3302 void PSParallelCompact::reset_millis_since_last_gc() {
3303   // We need a monotonically non-decreasing time in ms but
3304   // os::javaTimeMillis() does not guarantee monotonicity.
3305   _time_of_last_gc = os::javaTimeNanos() / NANOSECS_PER_MILLISEC;
3306 }
3307 
3308 ParMarkBitMap::IterationStatus MoveAndUpdateClosure::copy_until_full()
3309 {
3310   if (source() != copy_destination()) {
3311     DEBUG_ONLY(PSParallelCompact::check_new_location(source(), destination());)
3312     Copy::aligned_conjoint_words(source(), copy_destination(), words_remaining());
3313   }
3314   update_state(words_remaining());
3315   assert(is_full(), "sanity");
3316   return ParMarkBitMap::full;
3317 }
3318 
3319 void MoveAndUpdateClosure::copy_partial_obj()
3320 {
3321   size_t words = words_remaining();
3322 
3323   HeapWord* const range_end = MIN2(source() + words, bitmap()->region_end());
3324   HeapWord* const end_addr = bitmap()->find_obj_end(source(), range_end);
3325   if (end_addr < range_end) {
3326     words = bitmap()->obj_size(source(), end_addr);
3327   }
3328 
3329   // This test is necessary; if omitted, the pointer updates to a partial object
3330   // that crosses the dense prefix boundary could be overwritten.
3331   if (source() != copy_destination()) {
3332     DEBUG_ONLY(PSParallelCompact::check_new_location(source(), destination());)
3333     Copy::aligned_conjoint_words(source(), copy_destination(), words);
3334   }
3335   update_state(words);
3336 }
3337 
3338 void MoveAndUpdateClosure::complete_region(ParCompactionManager *cm, HeapWord *dest_addr,
3339                                            PSParallelCompact::RegionData *region_ptr) {
3340   assert(region_ptr->shadow_state() == ParallelCompactData::RegionData::FINISH, "Region should be finished");
3341   region_ptr->set_completed();
3342 }
3343 
3344 ParMarkBitMapClosure::IterationStatus
3345 MoveAndUpdateClosure::do_addr(HeapWord* addr, size_t words) {
3346   assert(destination() != NULL, "sanity");
3347   assert(bitmap()->obj_size(addr) == words, "bad size");
3348 
3349   _source = addr;
3350   assert(PSParallelCompact::summary_data().calc_new_pointer(source(), compaction_manager()) ==
3351          destination(), "wrong destination");
3352 
3353   if (words > words_remaining()) {
3354     return ParMarkBitMap::would_overflow;
3355   }
3356 
3357   // The start_array must be updated even if the object is not moving.
3358   if (_start_array != NULL) {
3359     _start_array->allocate_block(destination());
3360   }
3361 
3362   if (copy_destination() != source()) {
3363     DEBUG_ONLY(PSParallelCompact::check_new_location(source(), destination());)
3364     Copy::aligned_conjoint_words(source(), copy_destination(), words);
3365   }
3366 
3367   oop moved_oop = (oop) copy_destination();
3368   compaction_manager()->update_contents(moved_oop);
3369   assert(oopDesc::is_oop_or_null(moved_oop), "Expected an oop or NULL at " PTR_FORMAT, p2i(moved_oop));
3370 
3371   update_state(words);
3372   assert(copy_destination() == (HeapWord*)moved_oop + moved_oop->size(), "sanity");
3373   return is_full() ? ParMarkBitMap::full : ParMarkBitMap::incomplete;
3374 }
3375 
3376 void ShadowClosure::complete_region(ParCompactionManager *cm, HeapWord *dest_addr,
3377                                     PSParallelCompact::RegionData *region_ptr) {
3378   assert(region_ptr->shadow_state() == ParallelCompactData::RegionData::SHADOW, "Region should be shadow");
3379   // Record the shadow region index
3380   region_ptr->set_shadow_region(_shadow);
3381   // Mark the shadow region filled
3382   region_ptr->mark_filled();
3383   // Try to copy the content of the shadow region back to its corresponding
3384   // heap region if available
3385   if (((region_ptr->available() && region_ptr->claim()) || region_ptr->claimed()) && region_ptr->try_copy()) {
3386     region_ptr->set_completed();
3387     PSParallelCompact::copy_back(PSParallelCompact::summary_data().region_to_addr(_shadow), dest_addr);
3388     cm->release_shadow_region(_shadow);
3389   }
3390 }
3391 
3392 UpdateOnlyClosure::UpdateOnlyClosure(ParMarkBitMap* mbm,
3393                                      ParCompactionManager* cm,
3394                                      PSParallelCompact::SpaceId space_id) :
3395   ParMarkBitMapClosure(mbm, cm),
3396   _space_id(space_id),
3397   _start_array(PSParallelCompact::start_array(space_id))
3398 {
3399 }
3400 
3401 // Updates the references in the object to their new values.
3402 ParMarkBitMapClosure::IterationStatus
3403 UpdateOnlyClosure::do_addr(HeapWord* addr, size_t words) {
3404   do_addr(addr);
3405   return ParMarkBitMap::incomplete;
3406 }
3407 
3408 FillClosure::FillClosure(ParCompactionManager* cm, PSParallelCompact::SpaceId space_id) :
3409   ParMarkBitMapClosure(PSParallelCompact::mark_bitmap(), cm),
< prev index next >