hotspot/src/share/vm/gc_implementation/parallelScavenge/gcTaskManager.hpp
changeset 11174 fccee5238e70
parent 7397 5b173b4ca846
child 13195 be27e1b6a4b9
equal deleted inserted replaced
11173:af2bc14f35f8 11174:fccee5238e70
    43 class NoopGCTask;
    43 class NoopGCTask;
    44 class BarrierGCTask;
    44 class BarrierGCTask;
    45 class ReleasingBarrierGCTask;
    45 class ReleasingBarrierGCTask;
    46 class NotifyingBarrierGCTask;
    46 class NotifyingBarrierGCTask;
    47 class WaitForBarrierGCTask;
    47 class WaitForBarrierGCTask;
       
    48 class IdleGCTask;
    48 // A free list of Monitor*'s.
    49 // A free list of Monitor*'s.
    49 class MonitorSupply;
    50 class MonitorSupply;
    50 
    51 
    51 // Forward declarations of classes referenced in this file via pointer.
    52 // Forward declarations of classes referenced in this file via pointer.
    52 class GCTaskThread;
    53 class GCTaskThread;
    62   public:
    63   public:
    63     enum kind {
    64     enum kind {
    64       unknown_task,
    65       unknown_task,
    65       ordinary_task,
    66       ordinary_task,
    66       barrier_task,
    67       barrier_task,
    67       noop_task
    68       noop_task,
       
    69       idle_task
    68     };
    70     };
    69     static const char* to_string(kind value);
    71     static const char* to_string(kind value);
    70   };
    72   };
    71 private:
    73 private:
    72   // Instance state.
    74   // Instance state.
   105   bool is_barrier_task() const {
   107   bool is_barrier_task() const {
   106     return kind()==Kind::barrier_task;
   108     return kind()==Kind::barrier_task;
   107   }
   109   }
   108   bool is_noop_task() const {
   110   bool is_noop_task() const {
   109     return kind()==Kind::noop_task;
   111     return kind()==Kind::noop_task;
       
   112   }
       
   113   bool is_idle_task() const {
       
   114     return kind()==Kind::idle_task;
   110   }
   115   }
   111   void print(const char* message) const PRODUCT_RETURN;
   116   void print(const char* message) const PRODUCT_RETURN;
   112 protected:
   117 protected:
   113   // Constructors: Only create subclasses.
   118   // Constructors: Only create subclasses.
   114   //     An ordinary GCTask.
   119   //     An ordinary GCTask.
   151   //     These just examine the state of the queue.
   156   //     These just examine the state of the queue.
   152   bool is_empty() const {
   157   bool is_empty() const {
   153     assert(((insert_end() == NULL && remove_end() == NULL) ||
   158     assert(((insert_end() == NULL && remove_end() == NULL) ||
   154             (insert_end() != NULL && remove_end() != NULL)),
   159             (insert_end() != NULL && remove_end() != NULL)),
   155            "insert_end and remove_end don't match");
   160            "insert_end and remove_end don't match");
       
   161     assert((insert_end() != NULL) || (_length == 0), "Not empty");
   156     return insert_end() == NULL;
   162     return insert_end() == NULL;
   157   }
   163   }
   158   uint length() const {
   164   uint length() const {
   159     return _length;
   165     return _length;
   160   }
   166   }
   202   // Methods.
   208   // Methods.
   203   void initialize();
   209   void initialize();
   204   GCTask* remove();                     // Remove from remove end.
   210   GCTask* remove();                     // Remove from remove end.
   205   GCTask* remove(GCTask* task);         // Remove from the middle.
   211   GCTask* remove(GCTask* task);         // Remove from the middle.
   206   void print(const char* message) const PRODUCT_RETURN;
   212   void print(const char* message) const PRODUCT_RETURN;
       
   213   // Debug support
       
   214   void verify_length() const PRODUCT_RETURN;
   207 };
   215 };
   208 
   216 
   209 // A GCTaskQueue that can be synchronized.
   217 // A GCTaskQueue that can be synchronized.
   210 // This "has-a" GCTaskQueue and a mutex to do the exclusion.
   218 // This "has-a" GCTaskQueue and a mutex to do the exclusion.
   211 class SynchronizedGCTaskQueue : public CHeapObj {
   219 class SynchronizedGCTaskQueue : public CHeapObj {
   283   virtual ~NotifyDoneClosure() {
   291   virtual ~NotifyDoneClosure() {
   284     // Nothing to do.
   292     // Nothing to do.
   285   }
   293   }
   286 };
   294 };
   287 
   295 
       
   296 // Dynamic number of GC threads
       
   297 //
       
   298 //  GC threads wait in get_task() for work (i.e., a task) to perform.
       
   299 // When the number of GC threads was static, the number of tasks
       
   300 // created to do a job was equal to or greater than the maximum
       
   301 // number of GC threads (ParallelGCThreads).  The job might be divided
       
   302 // into a number of tasks greater than the number of GC threads for
       
   303 // load balancing (i.e., over partitioning).  The last task to be
       
   304 // executed by a GC thread in a job is a work stealing task.  A
       
   305 // GC  thread that gets a work stealing task continues to execute
       
   306 // that task until the job is done.  In the static number of GC theads
       
   307 // case, tasks are added to a queue (FIFO).  The work stealing tasks are
       
   308 // the last to be added.  Once the tasks are added, the GC threads grab
       
   309 // a task and go.  A single thread can do all the non-work stealing tasks
       
   310 // and then execute a work stealing and wait for all the other GC threads
       
   311 // to execute their work stealing task.
       
   312 //  In the dynamic number of GC threads implementation, idle-tasks are
       
   313 // created to occupy the non-participating or "inactive" threads.  An
       
   314 // idle-task makes the GC thread wait on a barrier that is part of the
       
   315 // GCTaskManager.  The GC threads that have been "idled" in a IdleGCTask
       
   316 // are released once all the active GC threads have finished their work
       
   317 // stealing tasks.  The GCTaskManager does not wait for all the "idled"
       
   318 // GC threads to resume execution. When those GC threads do resume
       
   319 // execution in the course of the thread scheduling, they call get_tasks()
       
   320 // as all the other GC threads do.  Because all the "idled" threads are
       
   321 // not required to execute in order to finish a job, it is possible for
       
   322 // a GC thread to still be "idled" when the next job is started.  Such
       
   323 // a thread stays "idled" for the next job.  This can result in a new
       
   324 // job not having all the expected active workers.  For example if on
       
   325 // job requests 4 active workers out of a total of 10 workers so the
       
   326 // remaining 6 are "idled", if the next job requests 6 active workers
       
   327 // but all 6 of the "idled" workers are still idle, then the next job
       
   328 // will only get 4 active workers.
       
   329 //  The implementation for the parallel old compaction phase has an
       
   330 // added complication.  In the static case parold partitions the chunks
       
   331 // ready to be filled into stacks, one for each GC thread.  A GC thread
       
   332 // executing a draining task (drains the stack of ready chunks)
       
   333 // claims a stack according to it's id (the unique ordinal value assigned
       
   334 // to each GC thread).  In the dynamic case not all GC threads will
       
   335 // actively participate so stacks with ready to fill chunks can only be
       
   336 // given to the active threads.  An initial implementation chose stacks
       
   337 // number 1-n to get the ready chunks and required that GC threads
       
   338 // 1-n be the active workers.  This was undesirable because it required
       
   339 // certain threads to participate.  In the final implementation a
       
   340 // list of stacks equal in number to the active workers are filled
       
   341 // with ready chunks.  GC threads that participate get a stack from
       
   342 // the task (DrainStacksCompactionTask), empty the stack, and then add it to a
       
   343 // recycling list at the end of the task.  If the same GC thread gets
       
   344 // a second task, it gets a second stack to drain and returns it.  The
       
   345 // stacks are added to a recycling list so that later stealing tasks
       
   346 // for this tasks can get a stack from the recycling list.  Stealing tasks
       
   347 // use the stacks in its work in a way similar to the draining tasks.
       
   348 // A thread is not guaranteed to get anything but a stealing task and
       
   349 // a thread that only gets a stealing task has to get a stack. A failed
       
   350 // implementation tried to have the GC threads keep the stack they used
       
   351 // during a draining task for later use in the stealing task but that didn't
       
   352 // work because as noted a thread is not guaranteed to get a draining task.
       
   353 //
       
   354 // For PSScavenge and ParCompactionManager the GC threads are
       
   355 // held in the GCTaskThread** _thread array in GCTaskManager.
       
   356 
       
   357 
   288 class GCTaskManager : public CHeapObj {
   358 class GCTaskManager : public CHeapObj {
   289  friend class ParCompactionManager;
   359  friend class ParCompactionManager;
   290  friend class PSParallelCompact;
   360  friend class PSParallelCompact;
   291  friend class PSScavenge;
   361  friend class PSScavenge;
   292  friend class PSRefProcTaskExecutor;
   362  friend class PSRefProcTaskExecutor;
   293  friend class RefProcTaskExecutor;
   363  friend class RefProcTaskExecutor;
       
   364  friend class GCTaskThread;
       
   365  friend class IdleGCTask;
   294 private:
   366 private:
   295   // Instance state.
   367   // Instance state.
   296   NotifyDoneClosure*        _ndc;               // Notify on completion.
   368   NotifyDoneClosure*        _ndc;               // Notify on completion.
   297   const uint                _workers;           // Number of workers.
   369   const uint                _workers;           // Number of workers.
   298   Monitor*                  _monitor;           // Notification of changes.
   370   Monitor*                  _monitor;           // Notification of changes.
   299   SynchronizedGCTaskQueue*  _queue;             // Queue of tasks.
   371   SynchronizedGCTaskQueue*  _queue;             // Queue of tasks.
   300   GCTaskThread**            _thread;            // Array of worker threads.
   372   GCTaskThread**            _thread;            // Array of worker threads.
       
   373   uint                      _active_workers;    // Number of active workers.
   301   uint                      _busy_workers;      // Number of busy workers.
   374   uint                      _busy_workers;      // Number of busy workers.
   302   uint                      _blocking_worker;   // The worker that's blocking.
   375   uint                      _blocking_worker;   // The worker that's blocking.
   303   bool*                     _resource_flag;     // Array of flag per threads.
   376   bool*                     _resource_flag;     // Array of flag per threads.
   304   uint                      _delivered_tasks;   // Count of delivered tasks.
   377   uint                      _delivered_tasks;   // Count of delivered tasks.
   305   uint                      _completed_tasks;   // Count of completed tasks.
   378   uint                      _completed_tasks;   // Count of completed tasks.
   306   uint                      _barriers;          // Count of barrier tasks.
   379   uint                      _barriers;          // Count of barrier tasks.
   307   uint                      _emptied_queue;     // Times we emptied the queue.
   380   uint                      _emptied_queue;     // Times we emptied the queue.
   308   NoopGCTask*               _noop_task;         // The NoopGCTask instance.
   381   NoopGCTask*               _noop_task;         // The NoopGCTask instance.
   309   uint                      _noop_tasks;        // Count of noop tasks.
   382   uint                      _noop_tasks;        // Count of noop tasks.
       
   383   WaitForBarrierGCTask*     _idle_inactive_task;// Task for inactive workers
       
   384   volatile uint             _idle_workers;      // Number of idled workers
   310 public:
   385 public:
   311   // Factory create and destroy methods.
   386   // Factory create and destroy methods.
   312   static GCTaskManager* create(uint workers) {
   387   static GCTaskManager* create(uint workers) {
   313     return new GCTaskManager(workers);
   388     return new GCTaskManager(workers);
   314   }
   389   }
   322   }
   397   }
   323   // Accessors.
   398   // Accessors.
   324   uint busy_workers() const {
   399   uint busy_workers() const {
   325     return _busy_workers;
   400     return _busy_workers;
   326   }
   401   }
       
   402   volatile uint idle_workers() const {
       
   403     return _idle_workers;
       
   404   }
   327   //     Pun between Monitor* and Mutex*
   405   //     Pun between Monitor* and Mutex*
   328   Monitor* monitor() const {
   406   Monitor* monitor() const {
   329     return _monitor;
   407     return _monitor;
   330   }
   408   }
   331   Monitor * lock() const {
   409   Monitor * lock() const {
   332     return _monitor;
   410     return _monitor;
       
   411   }
       
   412   WaitForBarrierGCTask* idle_inactive_task() {
       
   413     return _idle_inactive_task;
   333   }
   414   }
   334   // Methods.
   415   // Methods.
   335   //     Add the argument task to be run.
   416   //     Add the argument task to be run.
   336   void add_task(GCTask* task);
   417   void add_task(GCTask* task);
   337   //     Add a list of tasks.  Removes task from the argument list.
   418   //     Add a list of tasks.  Removes task from the argument list.
   348   void release_all_resources();
   429   void release_all_resources();
   349   //     Ask if a particular worker should release its resources.
   430   //     Ask if a particular worker should release its resources.
   350   bool should_release_resources(uint which); // Predicate.
   431   bool should_release_resources(uint which); // Predicate.
   351   //     Note the release of resources by the argument worker.
   432   //     Note the release of resources by the argument worker.
   352   void note_release(uint which);
   433   void note_release(uint which);
       
   434   //     Create IdleGCTasks for inactive workers and start workers
       
   435   void task_idle_workers();
       
   436   //     Release the workers in IdleGCTasks
       
   437   void release_idle_workers();
   353   // Constants.
   438   // Constants.
   354   //     A sentinel worker identifier.
   439   //     A sentinel worker identifier.
   355   static uint sentinel_worker() {
   440   static uint sentinel_worker() {
   356     return (uint) -1;                   // Why isn't there a max_uint?
   441     return (uint) -1;                   // Why isn't there a max_uint?
   357   }
   442   }
   373   ~GCTaskManager();
   458   ~GCTaskManager();
   374   // Accessors.
   459   // Accessors.
   375   uint workers() const {
   460   uint workers() const {
   376     return _workers;
   461     return _workers;
   377   }
   462   }
       
   463   void set_active_workers(uint v) {
       
   464     assert(v <= _workers, "Trying to set more workers active than there are");
       
   465     _active_workers = MIN2(v, _workers);
       
   466     assert(v != 0, "Trying to set active workers to 0");
       
   467     _active_workers = MAX2(1U, _active_workers);
       
   468   }
       
   469   // Sets the number of threads that will be used in a collection
       
   470   void set_active_gang();
       
   471 
   378   NotifyDoneClosure* notify_done_closure() const {
   472   NotifyDoneClosure* notify_done_closure() const {
   379     return _ndc;
   473     return _ndc;
   380   }
   474   }
   381   SynchronizedGCTaskQueue* queue() const {
   475   SynchronizedGCTaskQueue* queue() const {
   382     return _queue;
   476     return _queue;
   455     _noop_tasks += 1;
   549     _noop_tasks += 1;
   456   }
   550   }
   457   void reset_noop_tasks() {
   551   void reset_noop_tasks() {
   458     _noop_tasks = 0;
   552     _noop_tasks = 0;
   459   }
   553   }
       
   554   void increment_idle_workers() {
       
   555     _idle_workers++;
       
   556   }
       
   557   void decrement_idle_workers() {
       
   558     _idle_workers--;
       
   559   }
   460   // Other methods.
   560   // Other methods.
   461   void initialize();
   561   void initialize();
       
   562 
       
   563  public:
       
   564   // Return true if all workers are currently active.
       
   565   bool all_workers_active() { return workers() == active_workers(); }
       
   566   uint active_workers() const {
       
   567     return _active_workers;
       
   568   }
   462 };
   569 };
   463 
   570 
   464 //
   571 //
   465 // Some exemplary GCTasks.
   572 // Some exemplary GCTasks.
   466 //
   573 //
   473 public:
   580 public:
   474   // Factory create and destroy methods.
   581   // Factory create and destroy methods.
   475   static NoopGCTask* create();
   582   static NoopGCTask* create();
   476   static NoopGCTask* create_on_c_heap();
   583   static NoopGCTask* create_on_c_heap();
   477   static void destroy(NoopGCTask* that);
   584   static void destroy(NoopGCTask* that);
       
   585 
       
   586   virtual char* name() { return (char *)"noop task"; }
   478   // Methods from GCTask.
   587   // Methods from GCTask.
   479   void do_it(GCTaskManager* manager, uint which) {
   588   void do_it(GCTaskManager* manager, uint which) {
   480     // Nothing to do.
   589     // Nothing to do.
   481   }
   590   }
   482 protected:
   591 protected:
   516     GCTask(GCTask::Kind::barrier_task) {
   625     GCTask(GCTask::Kind::barrier_task) {
   517     // Nothing to do.
   626     // Nothing to do.
   518   }
   627   }
   519   // Destructor-like method.
   628   // Destructor-like method.
   520   void destruct();
   629   void destruct();
       
   630 
       
   631   virtual char* name() { return (char *)"barrier task"; }
   521   // Methods.
   632   // Methods.
   522   //     Wait for this to be the only task running.
   633   //     Wait for this to be the only task running.
   523   void do_it_internal(GCTaskManager* manager, uint which);
   634   void do_it_internal(GCTaskManager* manager, uint which);
   524 };
   635 };
   525 
   636 
   584 // A WaitForBarrierGCTask is a BarrierGCTask
   695 // A WaitForBarrierGCTask is a BarrierGCTask
   585 // with a method you can call to wait until
   696 // with a method you can call to wait until
   586 // the BarrierGCTask is done.
   697 // the BarrierGCTask is done.
   587 // This may cover many of the uses of NotifyingBarrierGCTasks.
   698 // This may cover many of the uses of NotifyingBarrierGCTasks.
   588 class WaitForBarrierGCTask : public BarrierGCTask {
   699 class WaitForBarrierGCTask : public BarrierGCTask {
       
   700   friend class GCTaskManager;
       
   701   friend class IdleGCTask;
   589 private:
   702 private:
   590   // Instance state.
   703   // Instance state.
   591   Monitor*   _monitor;                  // Guard and notify changes.
   704   Monitor*      _monitor;                  // Guard and notify changes.
   592   bool       _should_wait;              // true=>wait, false=>proceed.
   705   volatile bool _should_wait;              // true=>wait, false=>proceed.
   593   const bool _is_c_heap_obj;            // Was allocated on the heap.
   706   const bool    _is_c_heap_obj;            // Was allocated on the heap.
   594 public:
   707 public:
   595   virtual char* name() { return (char *) "waitfor-barrier-task"; }
   708   virtual char* name() { return (char *) "waitfor-barrier-task"; }
   596 
   709 
   597   // Factory create and destroy methods.
   710   // Factory create and destroy methods.
   598   static WaitForBarrierGCTask* create();
   711   static WaitForBarrierGCTask* create();
   599   static WaitForBarrierGCTask* create_on_c_heap();
   712   static WaitForBarrierGCTask* create_on_c_heap();
   600   static void destroy(WaitForBarrierGCTask* that);
   713   static void destroy(WaitForBarrierGCTask* that);
   601   // Methods.
   714   // Methods.
   602   void     do_it(GCTaskManager* manager, uint which);
   715   void     do_it(GCTaskManager* manager, uint which);
   603   void     wait_for();
   716   void     wait_for(bool reset);
       
   717   void set_should_wait(bool value) {
       
   718     _should_wait = value;
       
   719   }
   604 protected:
   720 protected:
   605   // Constructor.  Clients use factory, but there might be subclasses.
   721   // Constructor.  Clients use factory, but there might be subclasses.
   606   WaitForBarrierGCTask(bool on_c_heap);
   722   WaitForBarrierGCTask(bool on_c_heap);
   607   // Destructor-like method.
   723   // Destructor-like method.
   608   void destruct();
   724   void destruct();
   611     return _monitor;
   727     return _monitor;
   612   }
   728   }
   613   bool should_wait() const {
   729   bool should_wait() const {
   614     return _should_wait;
   730     return _should_wait;
   615   }
   731   }
   616   void set_should_wait(bool value) {
       
   617     _should_wait = value;
       
   618   }
       
   619   bool is_c_heap_obj() {
   732   bool is_c_heap_obj() {
   620     return _is_c_heap_obj;
   733     return _is_c_heap_obj;
   621   }
   734   }
       
   735 };
       
   736 
       
   737 // Task that is used to idle a GC task when fewer than
       
   738 // the maximum workers are wanted.
       
   739 class IdleGCTask : public GCTask {
       
   740   const bool    _is_c_heap_obj;            // Was allocated on the heap.
       
   741  public:
       
   742   bool is_c_heap_obj() {
       
   743     return _is_c_heap_obj;
       
   744   }
       
   745   // Factory create and destroy methods.
       
   746   static IdleGCTask* create();
       
   747   static IdleGCTask* create_on_c_heap();
       
   748   static void destroy(IdleGCTask* that);
       
   749 
       
   750   virtual char* name() { return (char *)"idle task"; }
       
   751   // Methods from GCTask.
       
   752   virtual void do_it(GCTaskManager* manager, uint which);
       
   753 protected:
       
   754   // Constructor.
       
   755   IdleGCTask(bool on_c_heap) :
       
   756     GCTask(GCTask::Kind::idle_task),
       
   757     _is_c_heap_obj(on_c_heap) {
       
   758     // Nothing to do.
       
   759   }
       
   760   // Destructor-like method.
       
   761   void destruct();
   622 };
   762 };
   623 
   763 
   624 class MonitorSupply : public AllStatic {
   764 class MonitorSupply : public AllStatic {
   625 private:
   765 private:
   626   // State.
   766   // State.