diff src/share/vm/gc_implementation/g1/concurrentMark.cpp @ 6862:8a5ea0a9ccc4

7127708: G1: change task num types from int to uint in concurrent mark Summary: Change the type of various task num fields, parameters etc to unsigned and rename them to be more consistent with the other collectors. Code changes were also reviewed by Vitaly Davidovich. Reviewed-by: johnc Contributed-by: Kaushik Srenevasan <kaushik@twitter.com>
author johnc
date Sat, 06 Oct 2012 01:17:44 -0700
parents 988bf00cc564
children 4202510ee0fe
line wrap: on
line diff
--- a/src/share/vm/gc_implementation/g1/concurrentMark.cpp	Fri Oct 05 13:37:08 2012 -0700
+++ b/src/share/vm/gc_implementation/g1/concurrentMark.cpp	Sat Oct 06 01:17:44 2012 -0700
@@ -426,11 +426,11 @@
   _markStack(this),
   // _finger set in set_non_marking_state
 
-  _max_task_num(MAX2((uint)ParallelGCThreads, 1U)),
+  _max_worker_id(MAX2((uint)ParallelGCThreads, 1U)),
   // _active_tasks set in set_non_marking_state
   // _tasks set inside the constructor
-  _task_queues(new CMTaskQueueSet((int) _max_task_num)),
-  _terminator(ParallelTaskTerminator((int) _max_task_num, _task_queues)),
+  _task_queues(new CMTaskQueueSet((int) _max_worker_id)),
+  _terminator(ParallelTaskTerminator((int) _max_worker_id, _task_queues)),
 
   _has_overflown(false),
   _concurrent(false),
@@ -481,17 +481,17 @@
 
   _root_regions.init(_g1h, this);
 
-  _tasks = NEW_C_HEAP_ARRAY(CMTask*, _max_task_num, mtGC);
-  _accum_task_vtime = NEW_C_HEAP_ARRAY(double, _max_task_num, mtGC);
-
-  _count_card_bitmaps = NEW_C_HEAP_ARRAY(BitMap,  _max_task_num, mtGC);
-  _count_marked_bytes = NEW_C_HEAP_ARRAY(size_t*, _max_task_num, mtGC);
+  _tasks = NEW_C_HEAP_ARRAY(CMTask*, _max_worker_id, mtGC);
+  _accum_task_vtime = NEW_C_HEAP_ARRAY(double, _max_worker_id, mtGC);
+
+  _count_card_bitmaps = NEW_C_HEAP_ARRAY(BitMap,  _max_worker_id, mtGC);
+  _count_marked_bytes = NEW_C_HEAP_ARRAY(size_t*, _max_worker_id, mtGC);
 
   BitMap::idx_t card_bm_size = _card_bm.size();
 
   // so that the assertion in MarkingTaskQueue::task_queue doesn't fail
-  _active_tasks = _max_task_num;
-  for (int i = 0; i < (int) _max_task_num; ++i) {
+  _active_tasks = _max_worker_id;
+  for (uint i = 0; i < _max_worker_id; ++i) {
     CMTaskQueue* task_queue = new CMTaskQueue();
     task_queue->initialize();
     _task_queues->register_queue(i, task_queue);
@@ -638,7 +638,7 @@
   // We do reset all of them, since different phases will use
   // different number of active threads. So, it's easiest to have all
   // of them ready.
-  for (int i = 0; i < (int) _max_task_num; ++i) {
+  for (uint i = 0; i < _max_worker_id; ++i) {
     _tasks[i]->reset(_nextMarkBitMap);
   }
 
@@ -648,7 +648,7 @@
 }
 
 void ConcurrentMark::set_phase(uint active_tasks, bool concurrent) {
-  assert(active_tasks <= _max_task_num, "we should not have more");
+  assert(active_tasks <= _max_worker_id, "we should not have more");
 
   _active_tasks = active_tasks;
   // Need to update the three data structures below according to the
@@ -659,7 +659,7 @@
 
   _concurrent = concurrent;
   // We propagate this to all tasks, not just the active ones.
-  for (int i = 0; i < (int) _max_task_num; ++i)
+  for (uint i = 0; i < _max_worker_id; ++i)
     _tasks[i]->set_concurrent(concurrent);
 
   if (concurrent) {
@@ -818,9 +818,9 @@
  * doesn't manipulate any data structures afterwards.
  */
 
-void ConcurrentMark::enter_first_sync_barrier(int task_num) {
+void ConcurrentMark::enter_first_sync_barrier(uint worker_id) {
   if (verbose_low()) {
-    gclog_or_tty->print_cr("[%d] entering first barrier", task_num);
+    gclog_or_tty->print_cr("[%u] entering first barrier", worker_id);
   }
 
   if (concurrent()) {
@@ -834,11 +834,11 @@
   // more work
 
   if (verbose_low()) {
-    gclog_or_tty->print_cr("[%d] leaving first barrier", task_num);
+    gclog_or_tty->print_cr("[%u] leaving first barrier", worker_id);
   }
 
-  // let task 0 do this
-  if (task_num == 0) {
+  // let the task associated with with worker 0 do this
+  if (worker_id == 0) {
     // task 0 is responsible for clearing the global data structures
     // We should be here because of an overflow. During STW we should
     // not clear the overflow flag since we rely on it being true when
@@ -858,9 +858,9 @@
   // then go into the second barrier
 }
 
-void ConcurrentMark::enter_second_sync_barrier(int task_num) {
+void ConcurrentMark::enter_second_sync_barrier(uint worker_id) {
   if (verbose_low()) {
-    gclog_or_tty->print_cr("[%d] entering second barrier", task_num);
+    gclog_or_tty->print_cr("[%u] entering second barrier", worker_id);
   }
 
   if (concurrent()) {
@@ -873,7 +873,7 @@
   // at this point everything should be re-initialised and ready to go
 
   if (verbose_low()) {
-    gclog_or_tty->print_cr("[%d] leaving second barrier", task_num);
+    gclog_or_tty->print_cr("[%u] leaving second barrier", worker_id);
   }
 }
 
@@ -2113,9 +2113,9 @@
     if (!_cm->has_overflown()) {
       oop obj = oopDesc::load_decode_heap_oop(p);
       if (_cm->verbose_high()) {
-        gclog_or_tty->print_cr("\t[%d] we're looking at location "
+        gclog_or_tty->print_cr("\t[%u] we're looking at location "
                                "*"PTR_FORMAT" = "PTR_FORMAT,
-                               _task->task_id(), p, (void*) obj);
+                               _task->worker_id(), p, (void*) obj);
       }
 
       _task->deal_with_reference(obj);
@@ -2144,7 +2144,7 @@
       }
     } else {
       if (_cm->verbose_high()) {
-         gclog_or_tty->print_cr("\t[%d] CM Overflow", _task->task_id());
+         gclog_or_tty->print_cr("\t[%u] CM Overflow", _task->worker_id());
       }
     }
   }
@@ -2160,8 +2160,8 @@
   void do_void() {
     do {
       if (_cm->verbose_high()) {
-        gclog_or_tty->print_cr("\t[%d] Drain: Calling do marking_step",
-                               _task->task_id());
+        gclog_or_tty->print_cr("\t[%u] Drain: Calling do marking_step",
+                               _task->worker_id());
       }
 
       // We call CMTask::do_marking_step() to completely drain the local and
@@ -2300,7 +2300,7 @@
     // We use the work gang from the G1CollectedHeap and we utilize all
     // the worker threads.
     uint active_workers = g1h->workers() ? g1h->workers()->active_workers() : 1U;
-    active_workers = MAX2(MIN2(active_workers, _max_task_num), 1U);
+    active_workers = MAX2(MIN2(active_workers, _max_worker_id), 1U);
 
     G1CMRefProcTaskExecutor par_task_executor(g1h, this,
                                               g1h->workers(), active_workers);
@@ -2619,7 +2619,7 @@
 }
 
 HeapRegion*
-ConcurrentMark::claim_region(int task_num) {
+ConcurrentMark::claim_region(uint worker_id) {
   // "checkpoint" the finger
   HeapWord* finger = _finger;
 
@@ -2657,10 +2657,10 @@
     HeapWord*   limit         = curr_region->next_top_at_mark_start();
 
     if (verbose_low()) {
-      gclog_or_tty->print_cr("[%d] curr_region = "PTR_FORMAT" "
+      gclog_or_tty->print_cr("[%u] curr_region = "PTR_FORMAT" "
                              "["PTR_FORMAT", "PTR_FORMAT"), "
                              "limit = "PTR_FORMAT,
-                             task_num, curr_region, bottom, end, limit);
+                             worker_id, curr_region, bottom, end, limit);
     }
 
     // Is the gap between reading the finger and doing the CAS too long?
@@ -2673,22 +2673,22 @@
       assert(_finger >= end, "the finger should have moved forward");
 
       if (verbose_low()) {
-        gclog_or_tty->print_cr("[%d] we were successful with region = "
-                               PTR_FORMAT, task_num, curr_region);
+        gclog_or_tty->print_cr("[%u] we were successful with region = "
+                               PTR_FORMAT, worker_id, curr_region);
       }
 
       if (limit > bottom) {
         if (verbose_low()) {
-          gclog_or_tty->print_cr("[%d] region "PTR_FORMAT" is not empty, "
-                                 "returning it ", task_num, curr_region);
+          gclog_or_tty->print_cr("[%u] region "PTR_FORMAT" is not empty, "
+                                 "returning it ", worker_id, curr_region);
         }
         return curr_region;
       } else {
         assert(limit == bottom,
                "the region limit should be at bottom");
         if (verbose_low()) {
-          gclog_or_tty->print_cr("[%d] region "PTR_FORMAT" is empty, "
-                                 "returning NULL", task_num, curr_region);
+          gclog_or_tty->print_cr("[%u] region "PTR_FORMAT" is empty, "
+                                 "returning NULL", worker_id, curr_region);
         }
         // we return NULL and the caller should try calling
         // claim_region() again.
@@ -2697,10 +2697,10 @@
     } else {
       assert(_finger > finger, "the finger should have moved forward");
       if (verbose_low()) {
-        gclog_or_tty->print_cr("[%d] somebody else moved the finger, "
+        gclog_or_tty->print_cr("[%u] somebody else moved the finger, "
                                "global finger = "PTR_FORMAT", "
                                "our finger = "PTR_FORMAT,
-                               task_num, _finger, finger);
+                               worker_id, _finger, finger);
       }
 
       // read it again
@@ -2783,7 +2783,7 @@
     _markStack.oops_do(&cl);
 
     // Verify entries on the task queues
-    for (int i = 0; i < (int) _max_task_num; i += 1) {
+    for (uint i = 0; i < _max_worker_id; i += 1) {
       cl.set_phase(VerifyNoCSetOopsQueues, i);
       OopTaskQueue* queue = _task_queues->queue(i);
       queue->oops_do(&cl);
@@ -2822,7 +2822,7 @@
     }
 
     // Verify the task fingers
-    assert(parallel_marking_threads() <= _max_task_num, "sanity");
+    assert(parallel_marking_threads() <= _max_worker_id, "sanity");
     for (int i = 0; i < (int) parallel_marking_threads(); i += 1) {
       CMTask* task = _tasks[i];
       HeapWord* task_finger = task->finger();
@@ -2849,7 +2849,7 @@
   }
   _finger = _heap_start;
 
-  for (int i = 0; i < (int)_max_task_num; ++i) {
+  for (uint i = 0; i < _max_worker_id; ++i) {
     OopTaskQueue* queue = _task_queues->queue(i);
     queue->set_empty();
   }
@@ -2862,15 +2862,15 @@
   ConcurrentMark* _cm;
   CardTableModRefBS* _ct_bs;
   BitMap* _cm_card_bm;
-  size_t _max_task_num;
+  uint _max_worker_id;
 
  public:
   AggregateCountDataHRClosure(G1CollectedHeap* g1h,
                               BitMap* cm_card_bm,
-                              size_t max_task_num) :
+                              uint max_worker_id) :
     _g1h(g1h), _cm(g1h->concurrent_mark()),
     _ct_bs((CardTableModRefBS*) (g1h->barrier_set())),
-    _cm_card_bm(cm_card_bm), _max_task_num(max_task_num) { }
+    _cm_card_bm(cm_card_bm), _max_worker_id(max_worker_id) { }
 
   bool doHeapRegion(HeapRegion* hr) {
     if (hr->continuesHumongous()) {
@@ -2927,7 +2927,7 @@
     uint hrs_index = hr->hrs_index();
     size_t marked_bytes = 0;
 
-    for (int i = 0; (size_t)i < _max_task_num; i += 1) {
+    for (uint i = 0; i < _max_worker_id; i += 1) {
       size_t* marked_bytes_array = _cm->count_marked_bytes_array_for(i);
       BitMap* task_card_bm = _cm->count_card_bitmap_for(i);
 
@@ -2935,7 +2935,7 @@
       // add it to the running total for this region.
       marked_bytes += marked_bytes_array[hrs_index];
 
-      // Now union the bitmaps[0,max_task_num)[start_idx..limit_idx)
+      // Now union the bitmaps[0,max_worker_id)[start_idx..limit_idx)
       // into the global card bitmap.
       BitMap::idx_t scan_idx = task_card_bm->get_next_one_offset(start_idx, limit_idx);
 
@@ -2967,22 +2967,22 @@
   G1CollectedHeap* _g1h;
   ConcurrentMark* _cm;
   BitMap* _cm_card_bm;
-  size_t _max_task_num;
+  uint _max_worker_id;
   int _active_workers;
 
 public:
   G1AggregateCountDataTask(G1CollectedHeap* g1h,
                            ConcurrentMark* cm,
                            BitMap* cm_card_bm,
-                           size_t max_task_num,
+                           uint max_worker_id,
                            int n_workers) :
     AbstractGangTask("Count Aggregation"),
     _g1h(g1h), _cm(cm), _cm_card_bm(cm_card_bm),
-    _max_task_num(max_task_num),
+    _max_worker_id(max_worker_id),
     _active_workers(n_workers) { }
 
   void work(uint worker_id) {
-    AggregateCountDataHRClosure cl(_g1h, _cm_card_bm, _max_task_num);
+    AggregateCountDataHRClosure cl(_g1h, _cm_card_bm, _max_worker_id);
 
     if (G1CollectedHeap::use_parallel_gc_threads()) {
       _g1h->heap_region_par_iterate_chunked(&cl, worker_id,
@@ -3001,7 +3001,7 @@
                         1);
 
   G1AggregateCountDataTask g1_par_agg_task(_g1h, this, &_card_bm,
-                                           _max_task_num, n_workers);
+                                           _max_worker_id, n_workers);
 
   if (G1CollectedHeap::use_parallel_gc_threads()) {
     assert(_g1h->check_heap_region_claim_values(HeapRegion::InitialClaimValue),
@@ -3030,9 +3030,9 @@
   _region_bm.clear();
 
   uint max_regions = _g1h->max_regions();
-  assert(_max_task_num != 0, "unitialized");
-
-  for (int i = 0; (size_t) i < _max_task_num; i += 1) {
+  assert(_max_worker_id > 0, "uninitialized");
+
+  for (uint i = 0; i < _max_worker_id; i += 1) {
     BitMap* task_card_bm = count_card_bitmap_for(i);
     size_t* marked_bytes_array = count_marked_bytes_array_for(i);
 
@@ -3062,7 +3062,7 @@
   clear_all_count_data();
   // Empty mark stack
   clear_marking_state();
-  for (int i = 0; i < (int)_max_task_num; ++i) {
+  for (uint i = 0; i < _max_worker_id; ++i) {
     _tasks[i]->clear_region_fields();
   }
   _has_aborted = true;
@@ -3154,8 +3154,8 @@
 void ConcurrentMark::print_finger() {
   gclog_or_tty->print_cr("heap ["PTR_FORMAT", "PTR_FORMAT"), global finger = "PTR_FORMAT,
                          _heap_start, _heap_end, _finger);
-  for (int i = 0; i < (int) _max_task_num; ++i) {
-    gclog_or_tty->print("   %d: "PTR_FORMAT, i, _tasks[i]->finger());
+  for (uint i = 0; i < _max_worker_id; ++i) {
+    gclog_or_tty->print("   %u: "PTR_FORMAT, i, _tasks[i]->finger());
   }
   gclog_or_tty->print_cr("");
 }
@@ -3165,8 +3165,8 @@
   assert(_nextMarkBitMap->isMarked((HeapWord*) obj), "invariant");
 
   if (_cm->verbose_high()) {
-    gclog_or_tty->print_cr("[%d] we're scanning object "PTR_FORMAT,
-                           _task_id, (void*) obj);
+    gclog_or_tty->print_cr("[%u] we're scanning object "PTR_FORMAT,
+                           _worker_id, (void*) obj);
   }
 
   size_t obj_size = obj->size();
@@ -3245,8 +3245,8 @@
         "claim_region() should have filtered out continues humongous regions");
 
   if (_cm->verbose_low()) {
-    gclog_or_tty->print_cr("[%d] setting up for region "PTR_FORMAT,
-                           _task_id, hr);
+    gclog_or_tty->print_cr("[%u] setting up for region "PTR_FORMAT,
+                           _worker_id, hr);
   }
 
   _curr_region  = hr;
@@ -3261,9 +3261,9 @@
 
   if (limit == bottom) {
     if (_cm->verbose_low()) {
-      gclog_or_tty->print_cr("[%d] found an empty region "
+      gclog_or_tty->print_cr("[%u] found an empty region "
                              "["PTR_FORMAT", "PTR_FORMAT")",
-                             _task_id, bottom, limit);
+                             _worker_id, bottom, limit);
     }
     // The region was collected underneath our feet.
     // We set the finger to bottom to ensure that the bitmap
@@ -3294,8 +3294,8 @@
 void CMTask::giveup_current_region() {
   assert(_curr_region != NULL, "invariant");
   if (_cm->verbose_low()) {
-    gclog_or_tty->print_cr("[%d] giving up region "PTR_FORMAT,
-                           _task_id, _curr_region);
+    gclog_or_tty->print_cr("[%u] giving up region "PTR_FORMAT,
+                           _worker_id, _curr_region);
   }
   clear_region_fields();
 }
@@ -3321,7 +3321,7 @@
   guarantee(nextMarkBitMap != NULL, "invariant");
 
   if (_cm->verbose_low()) {
-    gclog_or_tty->print_cr("[%d] resetting", _task_id);
+    gclog_or_tty->print_cr("[%u] resetting", _worker_id);
   }
 
   _nextMarkBitMap                = nextMarkBitMap;
@@ -3415,9 +3415,9 @@
   _all_clock_intervals_ms.add(last_interval_ms);
 
   if (_cm->verbose_medium()) {
-      gclog_or_tty->print_cr("[%d] regular clock, interval = %1.2lfms, "
+      gclog_or_tty->print_cr("[%u] regular clock, interval = %1.2lfms, "
                         "scanned = %d%s, refs reached = %d%s",
-                        _task_id, last_interval_ms,
+                        _worker_id, last_interval_ms,
                         _words_scanned,
                         (_words_scanned >= _words_scanned_limit) ? " (*)" : "",
                         _refs_reached,
@@ -3449,8 +3449,8 @@
   SATBMarkQueueSet& satb_mq_set = JavaThread::satb_mark_queue_set();
   if (!_draining_satb_buffers && satb_mq_set.process_completed_buffers()) {
     if (_cm->verbose_low()) {
-      gclog_or_tty->print_cr("[%d] aborting to deal with pending SATB buffers",
-                             _task_id);
+      gclog_or_tty->print_cr("[%u] aborting to deal with pending SATB buffers",
+                             _worker_id);
     }
     // we do need to process SATB buffers, we'll abort and restart
     // the marking task to do so
@@ -3475,7 +3475,7 @@
   // scanning limit so that the clock is called earlier.
 
   if (_cm->verbose_medium()) {
-    gclog_or_tty->print_cr("[%d] decreasing limits", _task_id);
+    gclog_or_tty->print_cr("[%u] decreasing limits", _worker_id);
   }
 
   _words_scanned_limit = _real_words_scanned_limit -
@@ -3503,16 +3503,16 @@
 
     if (!_cm->mark_stack_push(buffer, n)) {
       if (_cm->verbose_low()) {
-        gclog_or_tty->print_cr("[%d] aborting due to global stack overflow",
-                               _task_id);
+        gclog_or_tty->print_cr("[%u] aborting due to global stack overflow",
+                               _worker_id);
       }
       set_has_aborted();
     } else {
       // the transfer was successful
 
       if (_cm->verbose_medium()) {
-        gclog_or_tty->print_cr("[%d] pushed %d entries to the global stack",
-                               _task_id, n);
+        gclog_or_tty->print_cr("[%u] pushed %d entries to the global stack",
+                               _worker_id, n);
       }
       statsOnly( int tmp_size = _cm->mark_stack_size();
                  if (tmp_size > _global_max_size) {
@@ -3539,8 +3539,8 @@
 
     statsOnly( ++_global_transfers_from; _global_pops += n );
     if (_cm->verbose_medium()) {
-      gclog_or_tty->print_cr("[%d] popped %d entries from the global stack",
-                             _task_id, n);
+      gclog_or_tty->print_cr("[%u] popped %d entries from the global stack",
+                             _worker_id, n);
     }
     for (int i = 0; i < n; ++i) {
       bool success = _task_queue->push(buffer[i]);
@@ -3575,8 +3575,8 @@
 
   if (_task_queue->size() > target_size) {
     if (_cm->verbose_high()) {
-      gclog_or_tty->print_cr("[%d] draining local queue, target size = %d",
-                             _task_id, target_size);
+      gclog_or_tty->print_cr("[%u] draining local queue, target size = %d",
+                             _worker_id, target_size);
     }
 
     oop obj;
@@ -3585,7 +3585,7 @@
       statsOnly( ++_local_pops );
 
       if (_cm->verbose_high()) {
-        gclog_or_tty->print_cr("[%d] popped "PTR_FORMAT, _task_id,
+        gclog_or_tty->print_cr("[%u] popped "PTR_FORMAT, _worker_id,
                                (void*) obj);
       }
 
@@ -3603,8 +3603,8 @@
     }
 
     if (_cm->verbose_high()) {
-      gclog_or_tty->print_cr("[%d] drained local queue, size = %d",
-                             _task_id, _task_queue->size());
+      gclog_or_tty->print_cr("[%u] drained local queue, size = %d",
+                             _worker_id, _task_queue->size());
     }
   }
 }
@@ -3631,8 +3631,8 @@
 
   if (_cm->mark_stack_size() > target_size) {
     if (_cm->verbose_low()) {
-      gclog_or_tty->print_cr("[%d] draining global_stack, target size %d",
-                             _task_id, target_size);
+      gclog_or_tty->print_cr("[%u] draining global_stack, target size %d",
+                             _worker_id, target_size);
     }
 
     while (!has_aborted() && _cm->mark_stack_size() > target_size) {
@@ -3641,8 +3641,8 @@
     }
 
     if (_cm->verbose_low()) {
-      gclog_or_tty->print_cr("[%d] drained global stack, size = %d",
-                             _task_id, _cm->mark_stack_size());
+      gclog_or_tty->print_cr("[%u] drained global stack, size = %d",
+                             _worker_id, _cm->mark_stack_size());
     }
   }
 }
@@ -3663,7 +3663,7 @@
   CMObjectClosure oc(this);
   SATBMarkQueueSet& satb_mq_set = JavaThread::satb_mark_queue_set();
   if (G1CollectedHeap::use_parallel_gc_threads()) {
-    satb_mq_set.set_par_closure(_task_id, &oc);
+    satb_mq_set.set_par_closure(_worker_id, &oc);
   } else {
     satb_mq_set.set_closure(&oc);
   }
@@ -3672,9 +3672,9 @@
   // until we run out of buffers or we need to abort.
   if (G1CollectedHeap::use_parallel_gc_threads()) {
     while (!has_aborted() &&
-           satb_mq_set.par_apply_closure_to_completed_buffer(_task_id)) {
+           satb_mq_set.par_apply_closure_to_completed_buffer(_worker_id)) {
       if (_cm->verbose_medium()) {
-        gclog_or_tty->print_cr("[%d] processed an SATB buffer", _task_id);
+        gclog_or_tty->print_cr("[%u] processed an SATB buffer", _worker_id);
       }
       statsOnly( ++_satb_buffers_processed );
       regular_clock_call();
@@ -3683,7 +3683,7 @@
     while (!has_aborted() &&
            satb_mq_set.apply_closure_to_completed_buffer()) {
       if (_cm->verbose_medium()) {
-        gclog_or_tty->print_cr("[%d] processed an SATB buffer", _task_id);
+        gclog_or_tty->print_cr("[%u] processed an SATB buffer", _worker_id);
       }
       statsOnly( ++_satb_buffers_processed );
       regular_clock_call();
@@ -3693,7 +3693,7 @@
   if (!concurrent() && !has_aborted()) {
     // We should only do this during remark.
     if (G1CollectedHeap::use_parallel_gc_threads()) {
-      satb_mq_set.par_iterate_closure_all_threads(_task_id);
+      satb_mq_set.par_iterate_closure_all_threads(_worker_id);
     } else {
       satb_mq_set.iterate_closure_all_threads();
     }
@@ -3706,7 +3706,7 @@
          satb_mq_set.completed_buffers_num() == 0, "invariant");
 
   if (G1CollectedHeap::use_parallel_gc_threads()) {
-    satb_mq_set.set_par_closure(_task_id, NULL);
+    satb_mq_set.set_par_closure(_worker_id, NULL);
   } else {
     satb_mq_set.set_closure(NULL);
   }
@@ -3717,8 +3717,8 @@
 }
 
 void CMTask::print_stats() {
-  gclog_or_tty->print_cr("Marking Stats, task = %d, calls = %d",
-                         _task_id, _calls);
+  gclog_or_tty->print_cr("Marking Stats, task = %u, calls = %d",
+                         _worker_id, _calls);
   gclog_or_tty->print_cr("  Elapsed time = %1.2lfms, Termination time = %1.2lfms",
                          _elapsed_time_ms, _termination_time_ms);
   gclog_or_tty->print_cr("  Step Times (cum): num = %d, avg = %1.2lfms, sd = %1.2lfms",
@@ -3866,7 +3866,7 @@
   G1CollectorPolicy* g1_policy = _g1h->g1_policy();
   assert(_task_queues != NULL, "invariant");
   assert(_task_queue != NULL, "invariant");
-  assert(_task_queues->queue(_task_id) == _task_queue, "invariant");
+  assert(_task_queues->queue(_worker_id) == _task_queue, "invariant");
 
   assert(!_claimed,
          "only one thread should claim this task at any one time");
@@ -3898,9 +3898,9 @@
   ++_calls;
 
   if (_cm->verbose_low()) {
-    gclog_or_tty->print_cr("[%d] >>>>>>>>>> START, call = %d, "
+    gclog_or_tty->print_cr("[%u] >>>>>>>>>> START, call = %d, "
                            "target = %1.2lfms >>>>>>>>>>",
-                           _task_id, _calls, _time_target_ms);
+                           _worker_id, _calls, _time_target_ms);
   }
 
   // Set up the bitmap and oop closures. Anything that uses them is
@@ -3948,10 +3948,10 @@
       MemRegion mr = MemRegion(_finger, _region_limit);
 
       if (_cm->verbose_low()) {
-        gclog_or_tty->print_cr("[%d] we're scanning part "
+        gclog_or_tty->print_cr("[%u] we're scanning part "
                                "["PTR_FORMAT", "PTR_FORMAT") "
                                "of region "PTR_FORMAT,
-                               _task_id, _finger, _region_limit, _curr_region);
+                               _worker_id, _finger, _region_limit, _curr_region);
       }
 
       // Let's iterate over the bitmap of the part of the
@@ -4007,17 +4007,17 @@
       assert(_finger       == NULL, "invariant");
       assert(_region_limit == NULL, "invariant");
       if (_cm->verbose_low()) {
-        gclog_or_tty->print_cr("[%d] trying to claim a new region", _task_id);
+        gclog_or_tty->print_cr("[%u] trying to claim a new region", _worker_id);
       }
-      HeapRegion* claimed_region = _cm->claim_region(_task_id);
+      HeapRegion* claimed_region = _cm->claim_region(_worker_id);
       if (claimed_region != NULL) {
         // Yes, we managed to claim one
         statsOnly( ++_regions_claimed );
 
         if (_cm->verbose_low()) {
-          gclog_or_tty->print_cr("[%d] we successfully claimed "
+          gclog_or_tty->print_cr("[%u] we successfully claimed "
                                  "region "PTR_FORMAT,
-                                 _task_id, claimed_region);
+                                 _worker_id, claimed_region);
         }
 
         setup_for_region(claimed_region);
@@ -4044,7 +4044,7 @@
            "at this point we should be out of regions");
 
     if (_cm->verbose_low()) {
-      gclog_or_tty->print_cr("[%d] all regions claimed", _task_id);
+      gclog_or_tty->print_cr("[%u] all regions claimed", _worker_id);
     }
 
     // Try to reduce the number of available SATB buffers so that
@@ -4068,17 +4068,17 @@
            "only way to reach here");
 
     if (_cm->verbose_low()) {
-      gclog_or_tty->print_cr("[%d] starting to steal", _task_id);
+      gclog_or_tty->print_cr("[%u] starting to steal", _worker_id);
     }
 
     while (!has_aborted()) {
       oop obj;
       statsOnly( ++_steal_attempts );
 
-      if (_cm->try_stealing(_task_id, &_hash_seed, obj)) {
+      if (_cm->try_stealing(_worker_id, &_hash_seed, obj)) {
         if (_cm->verbose_medium()) {
-          gclog_or_tty->print_cr("[%d] stolen "PTR_FORMAT" successfully",
-                                 _task_id, (void*) obj);
+          gclog_or_tty->print_cr("[%u] stolen "PTR_FORMAT" successfully",
+                                 _worker_id, (void*) obj);
         }
 
         statsOnly( ++_steals );
@@ -4116,7 +4116,7 @@
     assert(_task_queue->size() == 0, "only way to reach here");
 
     if (_cm->verbose_low()) {
-      gclog_or_tty->print_cr("[%d] starting termination protocol", _task_id);
+      gclog_or_tty->print_cr("[%u] starting termination protocol", _worker_id);
     }
 
     _termination_start_time_ms = os::elapsedVTime() * 1000.0;
@@ -4131,7 +4131,7 @@
     if (finished) {
       // We're all done.
 
-      if (_task_id == 0) {
+      if (_worker_id == 0) {
         // let's allow task 0 to do this
         if (concurrent()) {
           assert(_cm->concurrent_marking_in_progress(), "invariant");
@@ -4153,15 +4153,15 @@
       guarantee(!_cm->mark_stack_overflow(), "only way to reach here");
 
       if (_cm->verbose_low()) {
-        gclog_or_tty->print_cr("[%d] all tasks terminated", _task_id);
+        gclog_or_tty->print_cr("[%u] all tasks terminated", _worker_id);
       }
     } else {
       // Apparently there's more work to do. Let's abort this task. It
       // will restart it and we can hopefully find more things to do.
 
       if (_cm->verbose_low()) {
-        gclog_or_tty->print_cr("[%d] apparently there is more work to do",
-                               _task_id);
+        gclog_or_tty->print_cr("[%u] apparently there is more work to do",
+                               _worker_id);
       }
 
       set_has_aborted();
@@ -4200,10 +4200,10 @@
       // will achieve this with the use of two barrier sync points.
 
       if (_cm->verbose_low()) {
-        gclog_or_tty->print_cr("[%d] detected overflow", _task_id);
+        gclog_or_tty->print_cr("[%u] detected overflow", _worker_id);
       }
 
-      _cm->enter_first_sync_barrier(_task_id);
+      _cm->enter_first_sync_barrier(_worker_id);
       // When we exit this sync barrier we know that all tasks have
       // stopped doing marking work. So, it's now safe to
       // re-initialise our data structures. At the end of this method,
@@ -4215,39 +4215,39 @@
       clear_region_fields();
 
       // ...and enter the second barrier.
-      _cm->enter_second_sync_barrier(_task_id);
+      _cm->enter_second_sync_barrier(_worker_id);
       // At this point everything has bee re-initialised and we're
       // ready to restart.
     }
 
     if (_cm->verbose_low()) {
-      gclog_or_tty->print_cr("[%d] <<<<<<<<<< ABORTING, target = %1.2lfms, "
+      gclog_or_tty->print_cr("[%u] <<<<<<<<<< ABORTING, target = %1.2lfms, "
                              "elapsed = %1.2lfms <<<<<<<<<<",
-                             _task_id, _time_target_ms, elapsed_time_ms);
+                             _worker_id, _time_target_ms, elapsed_time_ms);
       if (_cm->has_aborted()) {
-        gclog_or_tty->print_cr("[%d] ========== MARKING ABORTED ==========",
-                               _task_id);
+        gclog_or_tty->print_cr("[%u] ========== MARKING ABORTED ==========",
+                               _worker_id);
       }
     }
   } else {
     if (_cm->verbose_low()) {
-      gclog_or_tty->print_cr("[%d] <<<<<<<<<< FINISHED, target = %1.2lfms, "
+      gclog_or_tty->print_cr("[%u] <<<<<<<<<< FINISHED, target = %1.2lfms, "
                              "elapsed = %1.2lfms <<<<<<<<<<",
-                             _task_id, _time_target_ms, elapsed_time_ms);
+                             _worker_id, _time_target_ms, elapsed_time_ms);
     }
   }
 
   _claimed = false;
 }
 
-CMTask::CMTask(int task_id,
+CMTask::CMTask(uint worker_id,
                ConcurrentMark* cm,
                size_t* marked_bytes,
                BitMap* card_bm,
                CMTaskQueue* task_queue,
                CMTaskQueueSet* task_queues)
   : _g1h(G1CollectedHeap::heap()),
-    _task_id(task_id), _cm(cm),
+    _worker_id(worker_id), _cm(cm),
     _claimed(false),
     _nextMarkBitMap(NULL), _hash_seed(17),
     _task_queue(task_queue),