8205921: Optimizing best-of-2 work stealing queue selection
authorzgu
Fri, 03 Aug 2018 11:06:10 +0200
changeset 51292 0538a5cdb474
parent 51291 57aa80913140
child 51293 53c3b460503c
8205921: Optimizing best-of-2 work stealing queue selection Summary: Bias towards stealing from queues that we recently successfully stole from to decrease the number of unsuccessful steal attempts. Reviewed-by: eosterlund, kbarrett Contributed-by: Zhengyu Gu <zgu@redhat.com>, Thomas Schatzl <thomas.schatzl@oracle.com>
src/hotspot/share/gc/cms/concurrentMarkSweepGeneration.cpp
src/hotspot/share/gc/cms/concurrentMarkSweepGeneration.hpp
src/hotspot/share/gc/cms/parNewGeneration.cpp
src/hotspot/share/gc/cms/parNewGeneration.hpp
src/hotspot/share/gc/g1/g1ConcurrentMark.cpp
src/hotspot/share/gc/g1/g1ConcurrentMark.hpp
src/hotspot/share/gc/g1/g1FullGCMarker.cpp
src/hotspot/share/gc/g1/g1ParScanThreadState.cpp
src/hotspot/share/gc/g1/g1ParScanThreadState.hpp
src/hotspot/share/gc/g1/g1ParScanThreadState.inline.hpp
src/hotspot/share/gc/parallel/pcTasks.cpp
src/hotspot/share/gc/parallel/psCompactionManager.hpp
src/hotspot/share/gc/parallel/psCompactionManager.inline.hpp
src/hotspot/share/gc/parallel/psPromotionManager.hpp
src/hotspot/share/gc/parallel/psPromotionManager.inline.hpp
src/hotspot/share/gc/parallel/psTasks.cpp
src/hotspot/share/gc/shared/taskqueue.cpp
src/hotspot/share/gc/shared/taskqueue.hpp
src/hotspot/share/gc/shared/taskqueue.inline.hpp
--- a/src/hotspot/share/gc/cms/concurrentMarkSweepGeneration.cpp	Fri Aug 03 09:57:10 2018 +0100
+++ b/src/hotspot/share/gc/cms/concurrentMarkSweepGeneration.cpp	Fri Aug 03 11:06:10 2018 +0200
@@ -573,7 +573,6 @@
         log_warning(gc)("task_queues allocation failure.");
         return;
       }
-      _hash_seed = NEW_C_HEAP_ARRAY(int, num_queues, mtGC);
       typedef Padded<OopTaskQueue> PaddedOopTaskQueue;
       for (i = 0; i < num_queues; i++) {
         PaddedOopTaskQueue *q = new PaddedOopTaskQueue();
@@ -585,7 +584,6 @@
       }
       for (i = 0; i < num_queues; i++) {
         _task_queues->queue(i)->initialize();
-        _hash_seed[i] = 17;  // copied from ParNew
       }
     }
   }
@@ -3416,7 +3414,6 @@
   oop obj_to_scan;
   CMSBitMap* bm = &(_collector->_markBitMap);
   CMSMarkStack* ovflw = &(_collector->_markStack);
-  int* seed = _collector->hash_seed(i);
   ParConcMarkingClosure cl(_collector, this, work_q, bm, ovflw);
   while (true) {
     cl.trim_queue(0);
@@ -3426,7 +3423,7 @@
       // overflow stack may already have been stolen from us.
       // assert(work_q->size() > 0, "Work from overflow stack");
       continue;
-    } else if (task_queues()->steal(i, seed, /* reference */ obj_to_scan)) {
+    } else if (task_queues()->steal(i, /* reference */ obj_to_scan)) {
       assert(oopDesc::is_oop(obj_to_scan), "Should be an oop");
       assert(bm->isMarked((HeapWord*)obj_to_scan), "Grey object");
       obj_to_scan->oop_iterate(&cl);
@@ -4325,7 +4322,7 @@
                                   ParMarkRefsIntoAndScanClosure* cl);
 
   // ... work stealing for the above
-  void do_work_steal(int i, ParMarkRefsIntoAndScanClosure* cl, int* seed);
+  void do_work_steal(int i, ParMarkRefsIntoAndScanClosure* cl);
 };
 
 class RemarkCLDClosure : public CLDClosure {
@@ -4470,7 +4467,7 @@
   // ---------- ... and drain overflow list.
   _timer.reset();
   _timer.start();
-  do_work_steal(worker_id, &par_mrias_cl, _collector->hash_seed(worker_id));
+  do_work_steal(worker_id, &par_mrias_cl);
   _timer.stop();
   log_trace(gc, task)("Finished work stealing in %dth thread: %3.3f sec", worker_id, _timer.seconds());
 }
@@ -4619,8 +4616,7 @@
 
 // . see if we can share work_queues with ParNew? XXX
 void
-CMSParRemarkTask::do_work_steal(int i, ParMarkRefsIntoAndScanClosure* cl,
-                                int* seed) {
+CMSParRemarkTask::do_work_steal(int i, ParMarkRefsIntoAndScanClosure* cl) {
   OopTaskQueue* work_q = work_queue(i);
   NOT_PRODUCT(int num_steals = 0;)
   oop obj_to_scan;
@@ -4651,7 +4647,7 @@
     // Verify that we have no work before we resort to stealing
     assert(work_q->size() == 0, "Have work, shouldn't steal");
     // Try to steal from other queues that have work
-    if (task_queues()->steal(i, seed, /* reference */ obj_to_scan)) {
+    if (task_queues()->steal(i, /* reference */ obj_to_scan)) {
       NOT_PRODUCT(num_steals++;)
       assert(oopDesc::is_oop(obj_to_scan), "Oops, not an oop!");
       assert(bm->isMarked((HeapWord*)obj_to_scan), "Stole an unmarked oop?");
@@ -5041,8 +5037,7 @@
 
   void do_work_steal(int i,
                      CMSParDrainMarkingStackClosure* drain,
-                     CMSParKeepAliveClosure* keep_alive,
-                     int* seed);
+                     CMSParKeepAliveClosure* keep_alive);
 
   virtual void work(uint worker_id);
 };
@@ -5060,8 +5055,7 @@
   CMSIsAliveClosure is_alive_closure(_span, _mark_bit_map);
   _task.work(worker_id, is_alive_closure, par_keep_alive, par_drain_stack);
   if (_task.marks_oops_alive()) {
-    do_work_steal(worker_id, &par_drain_stack, &par_keep_alive,
-                  _collector->hash_seed(worker_id));
+    do_work_steal(worker_id, &par_drain_stack, &par_keep_alive);
   }
   assert(work_queue(worker_id)->size() == 0, "work_queue should be empty");
   assert(_collector->_overflow_list == NULL, "non-empty _overflow_list");
@@ -5080,8 +5074,7 @@
 // . see if we can share work_queues with ParNew? XXX
 void CMSRefProcTaskProxy::do_work_steal(int i,
   CMSParDrainMarkingStackClosure* drain,
-  CMSParKeepAliveClosure* keep_alive,
-  int* seed) {
+  CMSParKeepAliveClosure* keep_alive) {
   OopTaskQueue* work_q = work_queue(i);
   NOT_PRODUCT(int num_steals = 0;)
   oop obj_to_scan;
@@ -5110,7 +5103,7 @@
     // Verify that we have no work before we resort to stealing
     assert(work_q->size() == 0, "Have work, shouldn't steal");
     // Try to steal from other queues that have work
-    if (task_queues()->steal(i, seed, /* reference */ obj_to_scan)) {
+    if (task_queues()->steal(i, /* reference */ obj_to_scan)) {
       NOT_PRODUCT(num_steals++;)
       assert(oopDesc::is_oop(obj_to_scan), "Oops, not an oop!");
       assert(_mark_bit_map->isMarked((HeapWord*)obj_to_scan), "Stole an unmarked oop?");
--- a/src/hotspot/share/gc/cms/concurrentMarkSweepGeneration.hpp	Fri Aug 03 09:57:10 2018 +0100
+++ b/src/hotspot/share/gc/cms/concurrentMarkSweepGeneration.hpp	Fri Aug 03 11:06:10 2018 +0200
@@ -544,8 +544,6 @@
   Stack<oop, mtGC>     _preserved_oop_stack;
   Stack<markOop, mtGC> _preserved_mark_stack;
 
-  int*             _hash_seed;
-
   // In support of multi-threaded concurrent phases
   YieldingFlexibleWorkGang* _conc_workers;
 
@@ -713,7 +711,6 @@
   bool stop_world_and_do(CMS_op_type op);
 
   OopTaskQueueSet* task_queues() { return _task_queues; }
-  int*             hash_seed(int i) { return &_hash_seed[i]; }
   YieldingFlexibleWorkGang* conc_workers() { return _conc_workers; }
 
   // Support for parallelizing Eden rescan in CMS remark phase
--- a/src/hotspot/share/gc/cms/parNewGeneration.cpp	Fri Aug 03 09:57:10 2018 +0100
+++ b/src/hotspot/share/gc/cms/parNewGeneration.cpp	Fri Aug 03 11:06:10 2018 +0200
@@ -106,7 +106,6 @@
   #endif // TASKQUEUE_STATS
 
   _survivor_chunk_array = (ChunkArray*) old_gen()->get_data_recorder(thread_num());
-  _hash_seed = 17;  // Might want to take time-based random value.
   _start = os::elapsedTime();
   _old_gen_closure.set_generation(old_gen_);
   _old_gen_root_closure.set_generation(old_gen_);
@@ -550,7 +549,6 @@
 
     // Attempt to steal work from promoted.
     if (task_queues()->steal(par_scan_state()->thread_num(),
-                             par_scan_state()->hash_seed(),
                              obj_to_scan)) {
       bool res = work_q->push(obj_to_scan);
       assert(res, "Empty queue should have room for a push.");
--- a/src/hotspot/share/gc/cms/parNewGeneration.hpp	Fri Aug 03 09:57:10 2018 +0100
+++ b/src/hotspot/share/gc/cms/parNewGeneration.hpp	Fri Aug 03 11:06:10 2018 +0200
@@ -96,7 +96,6 @@
 
   HeapWord *_young_old_boundary;
 
-  int _hash_seed;
   int _thread_num;
   AgeTable _ageTable;
 
@@ -165,7 +164,6 @@
   // Is new_obj a candidate for scan_partial_array_and_push_remainder method.
   inline bool should_be_partially_scanned(oop new_obj, oop old_obj) const;
 
-  int* hash_seed()  { return &_hash_seed; }
   int  thread_num() { return _thread_num; }
 
   // Allocate a to-space block of size "sz", or else return NULL.
--- a/src/hotspot/share/gc/g1/g1ConcurrentMark.cpp	Fri Aug 03 09:57:10 2018 +0100
+++ b/src/hotspot/share/gc/g1/g1ConcurrentMark.cpp	Fri Aug 03 11:06:10 2018 +0200
@@ -2466,8 +2466,8 @@
                        hits, misses, percent_of(hits, hits + misses));
 }
 
-bool G1ConcurrentMark::try_stealing(uint worker_id, int* hash_seed, G1TaskQueueEntry& task_entry) {
-  return _task_queues->steal(worker_id, hash_seed, task_entry);
+bool G1ConcurrentMark::try_stealing(uint worker_id, G1TaskQueueEntry& task_entry) {
+  return _task_queues->steal(worker_id, task_entry);
 }
 
 /*****************************************************************************
@@ -2773,7 +2773,7 @@
            "only way to reach here");
     while (!has_aborted()) {
       G1TaskQueueEntry entry;
-      if (_cm->try_stealing(_worker_id, &_hash_seed, entry)) {
+      if (_cm->try_stealing(_worker_id, entry)) {
         scan_task_entry(entry);
 
         // And since we're towards the end, let's totally drain the
@@ -2915,7 +2915,6 @@
   _refs_reached(0),
   _refs_reached_limit(0),
   _real_refs_reached_limit(0),
-  _hash_seed(17),
   _has_aborted(false),
   _has_timed_out(false),
   _draining_satb_buffers(false),
--- a/src/hotspot/share/gc/g1/g1ConcurrentMark.hpp	Fri Aug 03 09:57:10 2018 +0100
+++ b/src/hotspot/share/gc/g1/g1ConcurrentMark.hpp	Fri Aug 03 11:06:10 2018 +0200
@@ -519,7 +519,7 @@
   }
 
   // Attempts to steal an object from the task queues of other tasks
-  bool try_stealing(uint worker_id, int* hash_seed, G1TaskQueueEntry& task_entry);
+  bool try_stealing(uint worker_id, G1TaskQueueEntry& task_entry);
 
   G1ConcurrentMark(G1CollectedHeap* g1h,
                    G1RegionToSpaceMapper* prev_bitmap_storage,
@@ -685,8 +685,6 @@
   // it was decreased).
   size_t                      _real_refs_reached_limit;
 
-  // Used by the work stealing
-  int                         _hash_seed;
   // If true, then the task has aborted for some reason
   bool                        _has_aborted;
   // Set when the task aborts because it has met its time quota
--- a/src/hotspot/share/gc/g1/g1FullGCMarker.cpp	Fri Aug 03 09:57:10 2018 +0100
+++ b/src/hotspot/share/gc/g1/g1FullGCMarker.cpp	Fri Aug 03 11:06:10 2018 +0200
@@ -46,15 +46,14 @@
 void G1FullGCMarker::complete_marking(OopQueueSet* oop_stacks,
                                       ObjArrayTaskQueueSet* array_stacks,
                                       ParallelTaskTerminator* terminator) {
-  int hash_seed = 17;
   do {
     drain_stack();
     ObjArrayTask steal_array;
-    if (array_stacks->steal(_worker_id, &hash_seed, steal_array)) {
+    if (array_stacks->steal(_worker_id, steal_array)) {
       follow_array_chunk(objArrayOop(steal_array.obj()), steal_array.index());
     } else {
       oop steal_oop;
-      if (oop_stacks->steal(_worker_id, &hash_seed, steal_oop)) {
+      if (oop_stacks->steal(_worker_id, steal_oop)) {
         follow_object(steal_oop);
       }
     }
--- a/src/hotspot/share/gc/g1/g1ParScanThreadState.cpp	Fri Aug 03 09:57:10 2018 +0100
+++ b/src/hotspot/share/gc/g1/g1ParScanThreadState.cpp	Fri Aug 03 11:06:10 2018 +0200
@@ -47,7 +47,6 @@
     _age_table(false),
     _tenuring_threshold(g1h->g1_policy()->tenuring_threshold()),
     _scanner(g1h, this),
-    _hash_seed(17),
     _worker_id(worker_id),
     _stack_trim_upper_threshold(GCDrainStackTargetSize * 2 + 1),
     _stack_trim_lower_threshold(GCDrainStackTargetSize),
--- a/src/hotspot/share/gc/g1/g1ParScanThreadState.hpp	Fri Aug 03 09:57:10 2018 +0100
+++ b/src/hotspot/share/gc/g1/g1ParScanThreadState.hpp	Fri Aug 03 11:06:10 2018 +0200
@@ -57,7 +57,6 @@
   uint              _tenuring_threshold;
   G1ScanEvacuatedObjClosure  _scanner;
 
-  int  _hash_seed;
   uint _worker_id;
 
   // Upper and lower threshold to start and end work queue draining.
--- a/src/hotspot/share/gc/g1/g1ParScanThreadState.inline.hpp	Fri Aug 03 09:57:10 2018 +0100
+++ b/src/hotspot/share/gc/g1/g1ParScanThreadState.inline.hpp	Fri Aug 03 11:06:10 2018 +0200
@@ -140,7 +140,7 @@
 
 void G1ParScanThreadState::steal_and_trim_queue(RefToScanQueueSet *task_queues) {
   StarTask stolen_task;
-  while (task_queues->steal(_worker_id, &_hash_seed, stolen_task)) {
+  while (task_queues->steal(_worker_id, stolen_task)) {
     assert(verify_task(stolen_task), "sanity");
     dispatch_reference(stolen_task);
 
--- a/src/hotspot/share/gc/parallel/pcTasks.cpp	Fri Aug 03 09:57:10 2018 +0100
+++ b/src/hotspot/share/gc/parallel/pcTasks.cpp	Fri Aug 03 11:06:10 2018 +0200
@@ -183,13 +183,12 @@
 
   oop obj = NULL;
   ObjArrayTask task;
-  int random_seed = 17;
   do {
-    while (ParCompactionManager::steal_objarray(which, &random_seed, task)) {
+    while (ParCompactionManager::steal_objarray(which,  task)) {
       cm->follow_contents((objArrayOop)task.obj(), task.index());
       cm->follow_marking_stacks();
     }
-    while (ParCompactionManager::steal(which, &random_seed, obj)) {
+    while (ParCompactionManager::steal(which, obj)) {
       cm->follow_contents(obj);
       cm->follow_marking_stacks();
     }
@@ -217,10 +216,9 @@
   guarantee(cm->region_stack()->is_empty(), "Not empty");
 
   size_t region_index = 0;
-  int random_seed = 17;
 
   while(true) {
-    if (ParCompactionManager::steal(which, &random_seed, region_index)) {
+    if (ParCompactionManager::steal(which, region_index)) {
       PSParallelCompact::fill_and_update_region(cm, region_index);
       cm->drain_region_stacks();
     } else {
--- a/src/hotspot/share/gc/parallel/psCompactionManager.hpp	Fri Aug 03 09:57:10 2018 +0100
+++ b/src/hotspot/share/gc/parallel/psCompactionManager.hpp	Fri Aug 03 11:06:10 2018 +0200
@@ -159,9 +159,9 @@
   // Access function for compaction managers
   static ParCompactionManager* gc_thread_compaction_manager(uint index);
 
-  static bool steal(int queue_num, int* seed, oop& t);
-  static bool steal_objarray(int queue_num, int* seed, ObjArrayTask& t);
-  static bool steal(int queue_num, int* seed, size_t& region);
+  static bool steal(int queue_num, oop& t);
+  static bool steal_objarray(int queue_num, ObjArrayTask& t);
+  static bool steal(int queue_num, size_t& region);
 
   // Process tasks remaining on any marking stack
   void follow_marking_stacks();
--- a/src/hotspot/share/gc/parallel/psCompactionManager.inline.hpp	Fri Aug 03 09:57:10 2018 +0100
+++ b/src/hotspot/share/gc/parallel/psCompactionManager.inline.hpp	Fri Aug 03 11:06:10 2018 +0200
@@ -37,16 +37,16 @@
 #include "utilities/debug.hpp"
 #include "utilities/globalDefinitions.hpp"
 
-inline bool ParCompactionManager::steal(int queue_num, int* seed, oop& t) {
-  return stack_array()->steal(queue_num, seed, t);
+inline bool ParCompactionManager::steal(int queue_num, oop& t) {
+  return stack_array()->steal(queue_num, t);
 }
 
-inline bool ParCompactionManager::steal_objarray(int queue_num, int* seed, ObjArrayTask& t) {
-  return _objarray_queues->steal(queue_num, seed, t);
+inline bool ParCompactionManager::steal_objarray(int queue_num, ObjArrayTask& t) {
+  return _objarray_queues->steal(queue_num, t);
 }
 
-inline bool ParCompactionManager::steal(int queue_num, int* seed, size_t& region) {
-  return region_array()->steal(queue_num, seed, region);
+inline bool ParCompactionManager::steal(int queue_num, size_t& region) {
+  return region_array()->steal(queue_num, region);
 }
 
 inline void ParCompactionManager::push(oop obj) {
--- a/src/hotspot/share/gc/parallel/psPromotionManager.hpp	Fri Aug 03 09:57:10 2018 +0100
+++ b/src/hotspot/share/gc/parallel/psPromotionManager.hpp	Fri Aug 03 11:06:10 2018 +0200
@@ -159,7 +159,7 @@
   static PSPromotionManager* gc_thread_promotion_manager(uint index);
   static PSPromotionManager* vm_thread_promotion_manager();
 
-  static bool steal_depth(int queue_num, int* seed, StarTask& t);
+  static bool steal_depth(int queue_num, StarTask& t);
 
   PSPromotionManager();
 
--- a/src/hotspot/share/gc/parallel/psPromotionManager.inline.hpp	Fri Aug 03 09:57:10 2018 +0100
+++ b/src/hotspot/share/gc/parallel/psPromotionManager.inline.hpp	Fri Aug 03 11:06:10 2018 +0200
@@ -322,8 +322,8 @@
   }
 }
 
-inline bool PSPromotionManager::steal_depth(int queue_num, int* seed, StarTask& t) {
-  return stack_array_depth()->steal(queue_num, seed, t);
+inline bool PSPromotionManager::steal_depth(int queue_num, StarTask& t) {
+  return stack_array_depth()->steal(queue_num, t);
 }
 
 #if TASKQUEUE_STATS
--- a/src/hotspot/share/gc/parallel/psTasks.cpp	Fri Aug 03 09:57:10 2018 +0100
+++ b/src/hotspot/share/gc/parallel/psTasks.cpp	Fri Aug 03 11:06:10 2018 +0200
@@ -141,10 +141,9 @@
   guarantee(pm->stacks_empty(),
             "stacks should be empty at this point");
 
-  int random_seed = 17;
   while(true) {
     StarTask p;
-    if (PSPromotionManager::steal_depth(which, &random_seed, p)) {
+    if (PSPromotionManager::steal_depth(which, p)) {
       TASKQUEUE_STATS_ONLY(pm->record_steal(p));
       pm->process_popped_location_depth(p);
       pm->drain_stacks_depth(true);
--- a/src/hotspot/share/gc/shared/taskqueue.cpp	Fri Aug 03 09:57:10 2018 +0100
+++ b/src/hotspot/share/gc/shared/taskqueue.cpp	Fri Aug 03 11:06:10 2018 +0200
@@ -111,24 +111,6 @@
 #endif // ASSERT
 #endif // TASKQUEUE_STATS
 
-int TaskQueueSetSuper::randomParkAndMiller(int *seed0) {
-  const int a =      16807;
-  const int m = 2147483647;
-  const int q =     127773;  /* m div a */
-  const int r =       2836;  /* m mod a */
-  assert(sizeof(int) == 4, "I think this relies on that");
-  int seed = *seed0;
-  int hi   = seed / q;
-  int lo   = seed % q;
-  int test = a * lo - r * hi;
-  if (test > 0)
-    seed = test;
-  else
-    seed = test + m;
-  *seed0 = seed;
-  return seed;
-}
-
 ParallelTaskTerminator::
 ParallelTaskTerminator(uint n_threads, TaskQueueSetSuper* queue_set) :
   _n_threads(n_threads),
--- a/src/hotspot/share/gc/shared/taskqueue.hpp	Fri Aug 03 09:57:10 2018 +0100
+++ b/src/hotspot/share/gc/shared/taskqueue.hpp	Fri Aug 03 11:06:10 2018 +0200
@@ -26,6 +26,7 @@
 #define SHARE_VM_GC_SHARED_TASKQUEUE_HPP
 
 #include "memory/allocation.hpp"
+#include "memory/padded.hpp"
 #include "oops/oopsHierarchy.hpp"
 #include "utilities/ostream.hpp"
 #include "utilities/stack.hpp"
@@ -298,12 +299,30 @@
   template<typename Fn> void iterate(Fn fn);
 
 private:
+  DEFINE_PAD_MINUS_SIZE(0, DEFAULT_CACHE_LINE_SIZE, 0);
   // Element array.
   volatile E* _elems;
+
+  DEFINE_PAD_MINUS_SIZE(1, DEFAULT_CACHE_LINE_SIZE, sizeof(E*));
+  // Queue owner local variables. Not to be accessed by other threads.
+
+  static const uint InvalidQueueId = uint(-1);
+  uint _last_stolen_queue_id; // The id of the queue we last stole from
+
+  int _seed; // Current random seed used for selecting a random queue during stealing.
+
+  DEFINE_PAD_MINUS_SIZE(2, DEFAULT_CACHE_LINE_SIZE, sizeof(uint) + sizeof(int));
+public:
+  int next_random_queue_id();
+
+  void set_last_stolen_queue_id(uint id)     { _last_stolen_queue_id = id; }
+  uint last_stolen_queue_id() const          { return _last_stolen_queue_id; }
+  bool is_last_stolen_queue_id_valid() const { return _last_stolen_queue_id != InvalidQueueId; }
+  void invalidate_last_stolen_queue_id()     { _last_stolen_queue_id = InvalidQueueId; }
 };
 
 template<class E, MEMFLAGS F, unsigned int N>
-GenericTaskQueue<E, F, N>::GenericTaskQueue() {
+GenericTaskQueue<E, F, N>::GenericTaskQueue() : _last_stolen_queue_id(InvalidQueueId), _seed(17 /* random number */) {
   assert(sizeof(Age) == sizeof(size_t), "Depends on this.");
 }
 
@@ -348,8 +367,6 @@
 };
 
 class TaskQueueSetSuper {
-protected:
-  static int randomParkAndMiller(int* seed0);
 public:
   // Returns "true" if some TaskQueue in the set contains a task.
   virtual bool peek() = 0;
@@ -367,22 +384,19 @@
   uint _n;
   T** _queues;
 
-  bool steal_best_of_2(uint queue_num, int* seed, E& t);
+  bool steal_best_of_2(uint queue_num, E& t);
 
 public:
-  GenericTaskQueueSet(int n);
+  GenericTaskQueueSet(uint n);
   ~GenericTaskQueueSet();
 
   void register_queue(uint i, T* q);
 
   T* queue(uint n);
 
-  // The thread with queue number "queue_num" (and whose random number seed is
-  // at "seed") is trying to steal a task from some other queue.  (It may try
-  // several queues, according to some configuration parameter.)  If some steal
-  // succeeds, returns "true" and sets "t" to the stolen task, otherwise returns
-  // false.
-  bool steal(uint queue_num, int* seed, E& t);
+  // Try to steal a task from some other queue than queue_num. It may perform several attempts at doing so.
+  // Returns if stealing succeeds, and sets "t" to the stolen task.
+  bool steal(uint queue_num, E& t);
 
   bool peek();
 
--- a/src/hotspot/share/gc/shared/taskqueue.inline.hpp	Fri Aug 03 09:57:10 2018 +0100
+++ b/src/hotspot/share/gc/shared/taskqueue.inline.hpp	Fri Aug 03 11:06:10 2018 +0200
@@ -34,10 +34,10 @@
 #include "utilities/stack.inline.hpp"
 
 template <class T, MEMFLAGS F>
-inline GenericTaskQueueSet<T, F>::GenericTaskQueueSet(int n) : _n(n) {
+inline GenericTaskQueueSet<T, F>::GenericTaskQueueSet(uint n) : _n(n) {
   typedef T* GenericTaskQueuePtr;
   _queues = NEW_C_HEAP_ARRAY(GenericTaskQueuePtr, n, F);
-  for (int i = 0; i < n; i++) {
+  for (uint i = 0; i < n; i++) {
     _queues[i] = NULL;
   }
 }
@@ -227,18 +227,71 @@
   return resAge == oldAge;
 }
 
+inline int randomParkAndMiller(int *seed0) {
+  const int a =      16807;
+  const int m = 2147483647;
+  const int q =     127773;  /* m div a */
+  const int r =       2836;  /* m mod a */
+  STATIC_ASSERT(sizeof(int) == 4);
+  int seed = *seed0;
+  int hi   = seed / q;
+  int lo   = seed % q;
+  int test = a * lo - r * hi;
+  if (test > 0) {
+    seed = test;
+  } else {
+    seed = test + m;
+  }
+  *seed0 = seed;
+  return seed;
+}
+
+template<class E, MEMFLAGS F, unsigned int N>
+int GenericTaskQueue<E, F, N>::next_random_queue_id() {
+  return randomParkAndMiller(&_seed);
+}
+
 template<class T, MEMFLAGS F> bool
-GenericTaskQueueSet<T, F>::steal_best_of_2(uint queue_num, int* seed, E& t) {
+GenericTaskQueueSet<T, F>::steal_best_of_2(uint queue_num, E& t) {
   if (_n > 2) {
+    T* const local_queue = _queues[queue_num];
     uint k1 = queue_num;
-    while (k1 == queue_num) k1 = TaskQueueSetSuper::randomParkAndMiller(seed) % _n;
+
+    if (local_queue->is_last_stolen_queue_id_valid()) {
+      k1 = local_queue->last_stolen_queue_id();
+      assert(k1 != queue_num, "Should not be the same");
+    } else {
+      while (k1 == queue_num) {
+        k1 = local_queue->next_random_queue_id() % _n;
+      }
+    }
+
     uint k2 = queue_num;
-    while (k2 == queue_num || k2 == k1) k2 = TaskQueueSetSuper::randomParkAndMiller(seed) % _n;
+    while (k2 == queue_num || k2 == k1) {
+      k2 = local_queue->next_random_queue_id() % _n;
+    }
     // Sample both and try the larger.
     uint sz1 = _queues[k1]->size();
     uint sz2 = _queues[k2]->size();
-    if (sz2 > sz1) return _queues[k2]->pop_global(t);
-    else return _queues[k1]->pop_global(t);
+
+    uint sel_k = 0;
+    bool suc = false;
+
+    if (sz2 > sz1) {
+      sel_k = k2;
+      suc = _queues[k2]->pop_global(t);
+    } else if (sz1 > 0) {
+      sel_k = k1;
+      suc = _queues[k1]->pop_global(t);
+    }
+
+    if (suc) {
+      local_queue->set_last_stolen_queue_id(sel_k);
+    } else {
+      local_queue->invalidate_last_stolen_queue_id();
+    }
+
+    return suc;
   } else if (_n == 2) {
     // Just try the other one.
     uint k = (queue_num + 1) % 2;
@@ -250,10 +303,10 @@
 }
 
 template<class T, MEMFLAGS F> bool
-GenericTaskQueueSet<T, F>::steal(uint queue_num, int* seed, E& t) {
+GenericTaskQueueSet<T, F>::steal(uint queue_num, E& t) {
   for (uint i = 0; i < 2 * _n; i++) {
     TASKQUEUE_STATS_ONLY(queue(queue_num)->stats.record_steal_attempt());
-    if (steal_best_of_2(queue_num, seed, t)) {
+    if (steal_best_of_2(queue_num, t)) {
       TASKQUEUE_STATS_ONLY(queue(queue_num)->stats.record_steal());
       return true;
     }