< prev index next >

src/share/vm/gc/g1/concurrentMark.cpp

Print this page




1920     return;
1921   }
1922 
1923   g1h->verify_region_sets_optional();
1924 
1925   if (VerifyDuringGC) {
1926     HandleMark hm;  // handle scope
1927     g1h->prepare_for_verify();
1928     Universe::verify(VerifyOption_G1UsePrevMarking,
1929                      " VerifyDuringGC:(before)");
1930   }
1931   g1h->check_bitmaps("Cleanup Start");
1932 
1933   G1CollectorPolicy* g1p = g1h->g1_policy();
1934   g1p->record_concurrent_mark_cleanup_start();
1935 
1936   double start = os::elapsedTime();
1937 
1938   HeapRegionRemSet::reset_for_cleanup_tasks();
1939 
1940   uint n_workers;
1941 
1942   // Do counting once more with the world stopped for good measure.
1943   G1ParFinalCountTask g1_par_count_task(g1h, &_region_bm, &_card_bm);
1944 
1945   g1h->set_par_threads();
1946   n_workers = g1h->n_par_threads();
1947   assert(g1h->n_par_threads() == n_workers,
1948          "Should not have been reset");
1949   g1h->workers()->run_task(&g1_par_count_task);
1950   // Done with the parallel phase so reset to 0.
1951   g1h->set_par_threads(0);
1952 
1953   if (VerifyDuringGC) {
1954     // Verify that the counting data accumulated during marking matches
1955     // that calculated by walking the marking bitmap.
1956 
1957     // Bitmaps to hold expected values
1958     BitMap expected_region_bm(_region_bm.size(), true);
1959     BitMap expected_card_bm(_card_bm.size(), true);
1960 
1961     G1ParVerifyFinalCountTask g1_par_verify_task(g1h,
1962                                                  &_region_bm,
1963                                                  &_card_bm,
1964                                                  &expected_region_bm,
1965                                                  &expected_card_bm);
1966 
1967     g1h->set_par_threads((int)n_workers);
1968     g1h->workers()->run_task(&g1_par_verify_task);
1969     // Done with the parallel phase so reset to 0.
1970     g1h->set_par_threads(0);
1971 
1972     guarantee(g1_par_verify_task.failures() == 0, "Unexpected accounting failures");
1973   }
1974 
1975   size_t start_used_bytes = g1h->used();
1976   g1h->set_marking_complete();
1977 
1978   double count_end = os::elapsedTime();
1979   double this_final_counting_time = (count_end - start);
1980   _total_counting_time += this_final_counting_time;
1981 
1982   if (G1PrintRegionLivenessInfo) {
1983     G1PrintRegionLivenessInfoClosure cl(gclog_or_tty, "Post-Marking");
1984     _g1h->heap_region_iterate(&cl);
1985   }
1986 
1987   // Install newly created mark bitMap as "prev".
1988   swapMarkBitMaps();
1989 
1990   g1h->reset_gc_time_stamp();
1991 


1992   // Note end of marking in all heap regions.
1993   G1ParNoteEndTask g1_par_note_end_task(g1h, &_cleanup_list, n_workers);
1994   g1h->set_par_threads((int)n_workers);
1995   g1h->workers()->run_task(&g1_par_note_end_task);
1996   g1h->set_par_threads(0);
1997   g1h->check_gc_time_stamps();
1998 
1999   if (!cleanup_list_is_empty()) {
2000     // The cleanup list is not empty, so we'll have to process it
2001     // concurrently. Notify anyone else that might be wanting free
2002     // regions that there will be more free regions coming soon.
2003     g1h->set_free_regions_coming();
2004   }
2005 
2006   // call below, since it affects the metric by which we sort the heap
2007   // regions.
2008   if (G1ScrubRemSets) {
2009     double rs_scrub_start = os::elapsedTime();
2010     G1ParScrubRemSetTask g1_par_scrub_rs_task(g1h, &_region_bm, &_card_bm, n_workers);
2011     g1h->set_par_threads((int)n_workers);
2012     g1h->workers()->run_task(&g1_par_scrub_rs_task);
2013     g1h->set_par_threads(0);
2014 
2015     double rs_scrub_end = os::elapsedTime();
2016     double this_rs_scrub_time = (rs_scrub_end - rs_scrub_start);
2017     _total_rs_scrub_time += this_rs_scrub_time;
2018   }
2019 
2020   // this will also free any regions totally full of garbage objects,
2021   // and sort the regions.
2022   g1h->g1_policy()->record_concurrent_mark_cleanup_end((int)n_workers);
2023 
2024   // Statistics.
2025   double end = os::elapsedTime();
2026   _cleanup_times.add((end - start) * 1000.0);
2027 
2028   if (G1Log::fine()) {
2029     g1h->g1_policy()->print_heap_transition(start_used_bytes);
2030   }
2031 
2032   // Clean up will have freed any regions completely full of garbage.
2033   // Update the soft reference policy with the new heap occupancy.


2294     CMTask* task = _cm->task(worker_id);
2295     G1CMIsAliveClosure g1_is_alive(_g1h);
2296     G1CMKeepAliveAndDrainClosure g1_par_keep_alive(_cm, task, false /* is_serial */);
2297     G1CMDrainMarkingStackClosure g1_par_drain(_cm, task, false /* is_serial */);
2298 
2299     _proc_task.work(worker_id, g1_is_alive, g1_par_keep_alive, g1_par_drain);
2300   }
2301 };
2302 
2303 void G1CMRefProcTaskExecutor::execute(ProcessTask& proc_task) {
2304   assert(_workers != NULL, "Need parallel worker threads.");
2305   assert(_g1h->ref_processor_cm()->processing_is_mt(), "processing is not MT");
2306 
2307   G1CMRefProcTaskProxy proc_task_proxy(proc_task, _g1h, _cm);
2308 
2309   // We need to reset the concurrency level before each
2310   // proxy task execution, so that the termination protocol
2311   // and overflow handling in CMTask::do_marking_step() knows
2312   // how many workers to wait for.
2313   _cm->set_concurrency(_active_workers);
2314   _g1h->set_par_threads(_active_workers);
2315   _workers->run_task(&proc_task_proxy);
2316   _g1h->set_par_threads(0);
2317 }
2318 
2319 class G1CMRefEnqueueTaskProxy: public AbstractGangTask {
2320   typedef AbstractRefProcTaskExecutor::EnqueueTask EnqueueTask;
2321   EnqueueTask& _enq_task;
2322 
2323 public:
2324   G1CMRefEnqueueTaskProxy(EnqueueTask& enq_task) :
2325     AbstractGangTask("Enqueue reference objects in parallel"),
2326     _enq_task(enq_task) { }
2327 
2328   virtual void work(uint worker_id) {
2329     _enq_task.work(worker_id);
2330   }
2331 };
2332 
2333 void G1CMRefProcTaskExecutor::execute(EnqueueTask& enq_task) {
2334   assert(_workers != NULL, "Need parallel worker threads.");
2335   assert(_g1h->ref_processor_cm()->processing_is_mt(), "processing is not MT");
2336 
2337   G1CMRefEnqueueTaskProxy enq_task_proxy(enq_task);
2338 
2339   // Not strictly necessary but...
2340   //
2341   // We need to reset the concurrency level before each
2342   // proxy task execution, so that the termination protocol
2343   // and overflow handling in CMTask::do_marking_step() knows
2344   // how many workers to wait for.
2345   _cm->set_concurrency(_active_workers);
2346   _g1h->set_par_threads(_active_workers);
2347   _workers->run_task(&enq_task_proxy);
2348   _g1h->set_par_threads(0);
2349 }
2350 
2351 void ConcurrentMark::weakRefsWorkParallelPart(BoolObjectClosure* is_alive, bool purged_classes) {
2352   G1CollectedHeap::heap()->parallel_cleaning(is_alive, true, true, purged_classes);
2353 }
2354 
2355 void ConcurrentMark::weakRefsWork(bool clear_all_soft_refs) {
2356   if (has_overflown()) {
2357     // Skip processing the discovered references if we have
2358     // overflown the global marking stack. Reference objects
2359     // only get discovered once so it is OK to not
2360     // de-populate the discovered reference lists. We could have,
2361     // but the only benefit would be that, when marking restarts,
2362     // less reference objects are discovered.
2363     return;
2364   }
2365 
2366   ResourceMark rm;
2367   HandleMark   hm;
2368 


2590       // want to abort remark and do concurrent marking again.
2591       task->record_end_time();
2592     }
2593   }
2594 
2595   CMRemarkTask(ConcurrentMark* cm, uint active_workers) :
2596     AbstractGangTask("Par Remark"), _cm(cm) {
2597     _cm->terminator()->reset_for_reuse(active_workers);
2598   }
2599 };
2600 
2601 void ConcurrentMark::checkpointRootsFinalWork() {
2602   ResourceMark rm;
2603   HandleMark   hm;
2604   G1CollectedHeap* g1h = G1CollectedHeap::heap();
2605 
2606   G1CMTraceTime trace("Finalize Marking", G1Log::finer());
2607 
2608   g1h->ensure_parsability(false);
2609 
2610   StrongRootsScope srs;
2611   // this is remark, so we'll use up all active threads
2612   uint active_workers = g1h->workers()->active_workers();
2613   if (active_workers == 0) {
2614     assert(active_workers > 0, "Should have been set earlier");
2615     active_workers = (uint) ParallelGCThreads;
2616     g1h->workers()->set_active_workers(active_workers);
2617   }
2618   set_concurrency_and_phase(active_workers, false /* concurrent */);
2619   // Leave _parallel_marking_threads at it's
2620   // value originally calculated in the ConcurrentMark
2621   // constructor and pass values of the active workers
2622   // through the gang in the task.
2623 



2624   CMRemarkTask remarkTask(this, active_workers);
2625   // We will start all available threads, even if we decide that the
2626   // active_workers will be fewer. The extra ones will just bail out
2627   // immediately.
2628   g1h->set_par_threads(active_workers);
2629   g1h->workers()->run_task(&remarkTask);
2630   g1h->set_par_threads(0);
2631 
2632   SATBMarkQueueSet& satb_mq_set = JavaThread::satb_mark_queue_set();
2633   guarantee(has_overflown() ||
2634             satb_mq_set.completed_buffers_num() == 0,
2635             err_msg("Invariant: has_overflown = %s, num buffers = %d",
2636                     BOOL_TO_STR(has_overflown()),
2637                     satb_mq_set.completed_buffers_num()));
2638 
2639   print_stats();
2640 }
2641 
2642 void ConcurrentMark::clearRangePrevBitmap(MemRegion mr) {
2643   // Note we are overriding the read-only view of the prev map here, via
2644   // the cast.
2645   ((CMBitMap*)_prevMarkBitMap)->clearRange(mr);
2646 }
2647 
2648 void ConcurrentMark::clearRangeNextBitmap(MemRegion mr) {
2649   _nextMarkBitMap->clearRange(mr);
2650 }


2983       _g1h(g1h), _cm(cm), _cm_card_bm(cm_card_bm),
2984       _max_worker_id(max_worker_id),
2985       _active_workers(n_workers),
2986       _hrclaimer(_active_workers) {
2987   }
2988 
2989   void work(uint worker_id) {
2990     AggregateCountDataHRClosure cl(_g1h, _cm_card_bm, _max_worker_id);
2991 
2992     _g1h->heap_region_par_iterate(&cl, worker_id, &_hrclaimer);
2993   }
2994 };
2995 
2996 
2997 void ConcurrentMark::aggregate_count_data() {
2998   uint n_workers = _g1h->workers()->active_workers();
2999 
3000   G1AggregateCountDataTask g1_par_agg_task(_g1h, this, &_card_bm,
3001                                            _max_worker_id, n_workers);
3002 
3003   _g1h->set_par_threads(n_workers);
3004   _g1h->workers()->run_task(&g1_par_agg_task);
3005   _g1h->set_par_threads(0);
3006 }
3007 
3008 // Clear the per-worker arrays used to store the per-region counting data
3009 void ConcurrentMark::clear_all_count_data() {
3010   // Clear the global card bitmap - it will be filled during
3011   // liveness count aggregation (during remark) and the
3012   // final counting task.
3013   _card_bm.clear();
3014 
3015   // Clear the global region bitmap - it will be filled as part
3016   // of the final counting task.
3017   _region_bm.clear();
3018 
3019   uint max_regions = _g1h->max_regions();
3020   assert(_max_worker_id > 0, "uninitialized");
3021 
3022   for (uint i = 0; i < _max_worker_id; i += 1) {
3023     BitMap* task_card_bm = count_card_bitmap_for(i);
3024     size_t* marked_bytes_array = count_marked_bytes_array_for(i);
3025 




1920     return;
1921   }
1922 
1923   g1h->verify_region_sets_optional();
1924 
1925   if (VerifyDuringGC) {
1926     HandleMark hm;  // handle scope
1927     g1h->prepare_for_verify();
1928     Universe::verify(VerifyOption_G1UsePrevMarking,
1929                      " VerifyDuringGC:(before)");
1930   }
1931   g1h->check_bitmaps("Cleanup Start");
1932 
1933   G1CollectorPolicy* g1p = g1h->g1_policy();
1934   g1p->record_concurrent_mark_cleanup_start();
1935 
1936   double start = os::elapsedTime();
1937 
1938   HeapRegionRemSet::reset_for_cleanup_tasks();
1939 


1940   // Do counting once more with the world stopped for good measure.
1941   G1ParFinalCountTask g1_par_count_task(g1h, &_region_bm, &_card_bm);
1942 




1943   g1h->workers()->run_task(&g1_par_count_task);


1944 
1945   if (VerifyDuringGC) {
1946     // Verify that the counting data accumulated during marking matches
1947     // that calculated by walking the marking bitmap.
1948 
1949     // Bitmaps to hold expected values
1950     BitMap expected_region_bm(_region_bm.size(), true);
1951     BitMap expected_card_bm(_card_bm.size(), true);
1952 
1953     G1ParVerifyFinalCountTask g1_par_verify_task(g1h,
1954                                                  &_region_bm,
1955                                                  &_card_bm,
1956                                                  &expected_region_bm,
1957                                                  &expected_card_bm);
1958 

1959     g1h->workers()->run_task(&g1_par_verify_task);


1960 
1961     guarantee(g1_par_verify_task.failures() == 0, "Unexpected accounting failures");
1962   }
1963 
1964   size_t start_used_bytes = g1h->used();
1965   g1h->set_marking_complete();
1966 
1967   double count_end = os::elapsedTime();
1968   double this_final_counting_time = (count_end - start);
1969   _total_counting_time += this_final_counting_time;
1970 
1971   if (G1PrintRegionLivenessInfo) {
1972     G1PrintRegionLivenessInfoClosure cl(gclog_or_tty, "Post-Marking");
1973     _g1h->heap_region_iterate(&cl);
1974   }
1975 
1976   // Install newly created mark bitMap as "prev".
1977   swapMarkBitMaps();
1978 
1979   g1h->reset_gc_time_stamp();
1980 
1981   uint n_workers = _g1h->workers()->active_workers();
1982 
1983   // Note end of marking in all heap regions.
1984   G1ParNoteEndTask g1_par_note_end_task(g1h, &_cleanup_list, n_workers);

1985   g1h->workers()->run_task(&g1_par_note_end_task);

1986   g1h->check_gc_time_stamps();
1987 
1988   if (!cleanup_list_is_empty()) {
1989     // The cleanup list is not empty, so we'll have to process it
1990     // concurrently. Notify anyone else that might be wanting free
1991     // regions that there will be more free regions coming soon.
1992     g1h->set_free_regions_coming();
1993   }
1994 
1995   // call below, since it affects the metric by which we sort the heap
1996   // regions.
1997   if (G1ScrubRemSets) {
1998     double rs_scrub_start = os::elapsedTime();
1999     G1ParScrubRemSetTask g1_par_scrub_rs_task(g1h, &_region_bm, &_card_bm, n_workers);

2000     g1h->workers()->run_task(&g1_par_scrub_rs_task);

2001 
2002     double rs_scrub_end = os::elapsedTime();
2003     double this_rs_scrub_time = (rs_scrub_end - rs_scrub_start);
2004     _total_rs_scrub_time += this_rs_scrub_time;
2005   }
2006 
2007   // this will also free any regions totally full of garbage objects,
2008   // and sort the regions.
2009   g1h->g1_policy()->record_concurrent_mark_cleanup_end((int)n_workers);
2010 
2011   // Statistics.
2012   double end = os::elapsedTime();
2013   _cleanup_times.add((end - start) * 1000.0);
2014 
2015   if (G1Log::fine()) {
2016     g1h->g1_policy()->print_heap_transition(start_used_bytes);
2017   }
2018 
2019   // Clean up will have freed any regions completely full of garbage.
2020   // Update the soft reference policy with the new heap occupancy.


2281     CMTask* task = _cm->task(worker_id);
2282     G1CMIsAliveClosure g1_is_alive(_g1h);
2283     G1CMKeepAliveAndDrainClosure g1_par_keep_alive(_cm, task, false /* is_serial */);
2284     G1CMDrainMarkingStackClosure g1_par_drain(_cm, task, false /* is_serial */);
2285 
2286     _proc_task.work(worker_id, g1_is_alive, g1_par_keep_alive, g1_par_drain);
2287   }
2288 };
2289 
2290 void G1CMRefProcTaskExecutor::execute(ProcessTask& proc_task) {
2291   assert(_workers != NULL, "Need parallel worker threads.");
2292   assert(_g1h->ref_processor_cm()->processing_is_mt(), "processing is not MT");
2293 
2294   G1CMRefProcTaskProxy proc_task_proxy(proc_task, _g1h, _cm);
2295 
2296   // We need to reset the concurrency level before each
2297   // proxy task execution, so that the termination protocol
2298   // and overflow handling in CMTask::do_marking_step() knows
2299   // how many workers to wait for.
2300   _cm->set_concurrency(_active_workers);

2301   _workers->run_task(&proc_task_proxy);

2302 }
2303 
2304 class G1CMRefEnqueueTaskProxy: public AbstractGangTask {
2305   typedef AbstractRefProcTaskExecutor::EnqueueTask EnqueueTask;
2306   EnqueueTask& _enq_task;
2307 
2308 public:
2309   G1CMRefEnqueueTaskProxy(EnqueueTask& enq_task) :
2310     AbstractGangTask("Enqueue reference objects in parallel"),
2311     _enq_task(enq_task) { }
2312 
2313   virtual void work(uint worker_id) {
2314     _enq_task.work(worker_id);
2315   }
2316 };
2317 
2318 void G1CMRefProcTaskExecutor::execute(EnqueueTask& enq_task) {
2319   assert(_workers != NULL, "Need parallel worker threads.");
2320   assert(_g1h->ref_processor_cm()->processing_is_mt(), "processing is not MT");
2321 
2322   G1CMRefEnqueueTaskProxy enq_task_proxy(enq_task);
2323 
2324   // Not strictly necessary but...
2325   //
2326   // We need to reset the concurrency level before each
2327   // proxy task execution, so that the termination protocol
2328   // and overflow handling in CMTask::do_marking_step() knows
2329   // how many workers to wait for.
2330   _cm->set_concurrency(_active_workers);

2331   _workers->run_task(&enq_task_proxy);

2332 }
2333 
2334 void ConcurrentMark::weakRefsWorkParallelPart(BoolObjectClosure* is_alive, bool purged_classes) {
2335   G1CollectedHeap::heap()->parallel_cleaning(is_alive, true, true, purged_classes);
2336 }
2337 
2338 void ConcurrentMark::weakRefsWork(bool clear_all_soft_refs) {
2339   if (has_overflown()) {
2340     // Skip processing the discovered references if we have
2341     // overflown the global marking stack. Reference objects
2342     // only get discovered once so it is OK to not
2343     // de-populate the discovered reference lists. We could have,
2344     // but the only benefit would be that, when marking restarts,
2345     // less reference objects are discovered.
2346     return;
2347   }
2348 
2349   ResourceMark rm;
2350   HandleMark   hm;
2351 


2573       // want to abort remark and do concurrent marking again.
2574       task->record_end_time();
2575     }
2576   }
2577 
2578   CMRemarkTask(ConcurrentMark* cm, uint active_workers) :
2579     AbstractGangTask("Par Remark"), _cm(cm) {
2580     _cm->terminator()->reset_for_reuse(active_workers);
2581   }
2582 };
2583 
2584 void ConcurrentMark::checkpointRootsFinalWork() {
2585   ResourceMark rm;
2586   HandleMark   hm;
2587   G1CollectedHeap* g1h = G1CollectedHeap::heap();
2588 
2589   G1CMTraceTime trace("Finalize Marking", G1Log::finer());
2590 
2591   g1h->ensure_parsability(false);
2592 

2593   // this is remark, so we'll use up all active threads
2594   uint active_workers = g1h->workers()->active_workers();
2595   if (active_workers == 0) {
2596     assert(active_workers > 0, "Should have been set earlier");
2597     active_workers = (uint) ParallelGCThreads;
2598     g1h->workers()->set_active_workers(active_workers);
2599   }
2600   set_concurrency_and_phase(active_workers, false /* concurrent */);
2601   // Leave _parallel_marking_threads at it's
2602   // value originally calculated in the ConcurrentMark
2603   // constructor and pass values of the active workers
2604   // through the gang in the task.
2605 
2606   {
2607     StrongRootsScope srs(active_workers);
2608 
2609     CMRemarkTask remarkTask(this, active_workers);
2610     // We will start all available threads, even if we decide that the
2611     // active_workers will be fewer. The extra ones will just bail out
2612     // immediately.

2613     g1h->workers()->run_task(&remarkTask);
2614   }
2615 
2616   SATBMarkQueueSet& satb_mq_set = JavaThread::satb_mark_queue_set();
2617   guarantee(has_overflown() ||
2618             satb_mq_set.completed_buffers_num() == 0,
2619             err_msg("Invariant: has_overflown = %s, num buffers = %d",
2620                     BOOL_TO_STR(has_overflown()),
2621                     satb_mq_set.completed_buffers_num()));
2622 
2623   print_stats();
2624 }
2625 
2626 void ConcurrentMark::clearRangePrevBitmap(MemRegion mr) {
2627   // Note we are overriding the read-only view of the prev map here, via
2628   // the cast.
2629   ((CMBitMap*)_prevMarkBitMap)->clearRange(mr);
2630 }
2631 
2632 void ConcurrentMark::clearRangeNextBitmap(MemRegion mr) {
2633   _nextMarkBitMap->clearRange(mr);
2634 }


2967       _g1h(g1h), _cm(cm), _cm_card_bm(cm_card_bm),
2968       _max_worker_id(max_worker_id),
2969       _active_workers(n_workers),
2970       _hrclaimer(_active_workers) {
2971   }
2972 
2973   void work(uint worker_id) {
2974     AggregateCountDataHRClosure cl(_g1h, _cm_card_bm, _max_worker_id);
2975 
2976     _g1h->heap_region_par_iterate(&cl, worker_id, &_hrclaimer);
2977   }
2978 };
2979 
2980 
2981 void ConcurrentMark::aggregate_count_data() {
2982   uint n_workers = _g1h->workers()->active_workers();
2983 
2984   G1AggregateCountDataTask g1_par_agg_task(_g1h, this, &_card_bm,
2985                                            _max_worker_id, n_workers);
2986 

2987   _g1h->workers()->run_task(&g1_par_agg_task);

2988 }
2989 
2990 // Clear the per-worker arrays used to store the per-region counting data
2991 void ConcurrentMark::clear_all_count_data() {
2992   // Clear the global card bitmap - it will be filled during
2993   // liveness count aggregation (during remark) and the
2994   // final counting task.
2995   _card_bm.clear();
2996 
2997   // Clear the global region bitmap - it will be filled as part
2998   // of the final counting task.
2999   _region_bm.clear();
3000 
3001   uint max_regions = _g1h->max_regions();
3002   assert(_max_worker_id > 0, "uninitialized");
3003 
3004   for (uint i = 0; i < _max_worker_id; i += 1) {
3005     BitMap* task_card_bm = count_card_bitmap_for(i);
3006     size_t* marked_bytes_array = count_marked_bytes_array_for(i);
3007 


< prev index next >