< prev index next >

src/share/vm/gc/shared/workgroup.hpp

Print this page

        

@@ -23,11 +23,15 @@
  */
 
 #ifndef SHARE_VM_GC_SHARED_WORKGROUP_HPP
 #define SHARE_VM_GC_SHARED_WORKGROUP_HPP
 
-#include "runtime/thread.inline.hpp"
+#include "memory/allocation.hpp"
+#include "runtime/globals.hpp"
+#include "runtime/thread.hpp"
+#include "utilities/debug.hpp"
+#include "utilities/globalDefinitions.hpp"
 
 // Task class hierarchy:
 //   AbstractGangTask
 //
 // Gang/Group class hierarchy:

@@ -41,12 +45,12 @@
 //     YieldingFlexibleGangWorker   (defined in another file)
 
 // Forward declarations of classes defined here
 
 class AbstractGangWorker;
-class GangWorker;
-class WorkData;
+class Semaphore;
+class WorkGang;
 
 // An abstract task to be worked on by a gang.
 // You subclass this to supply your own work() method
 class AbstractGangTask VALUE_OBJ_CLASS_SPEC {
   const char* _name;

@@ -60,10 +64,37 @@
 
   // Debugging accessor for the name.
   const char* name() const { return _name; }
 };
 
+struct WorkData {
+  AbstractGangTask* _task;
+  uint              _worker_id;
+  WorkData(AbstractGangTask* task, uint worker_id) : _task(task), _worker_id(worker_id) {}
+};
+
+// Interface to handle the synchronization between the coordinator thread and the worker threads,
+// when a task is dispatched out to the worker threads.
+class GangTaskDispatcher : public CHeapObj<mtGC> {
+ public:
+  virtual ~GangTaskDispatcher() {}
+
+  // Coordinator API.
+
+  // Distributes the task out to num_workers workers.
+  // Returns when the task has been completed by all workers.
+  virtual void coordinator_execute_on_workers(AbstractGangTask* task, uint num_workers) = 0;
+
+  // Worker API.
+
+  // Waits for a task to become available to the worker.
+  // Returns when the worker has been assigned a task.
+  virtual WorkData worker_wait_for_task() = 0;
+
+  // Signal to the coordinator that the worker is done with the assigned task.
+  virtual void     worker_done_with_task() = 0;
+};
 
 // The work gang is the collection of workers to execute tasks.
 // The number of workers run for a task is "_active_workers"
 // while "_total_workers" is the number of available of workers.
 class AbstractWorkGang : public CHeapObj<mtInternal> {

@@ -89,12 +120,10 @@
       _active_workers(UseDynamicNumberOfGCThreads ? 1U : workers),
       _are_GC_task_threads(are_GC_task_threads),
       _are_ConcurrentGC_threads(are_ConcurrentGC_threads)
   { }
 
-  virtual AbstractGangWorker* allocate_worker(uint which) = 0;
-
   // Initialize workers in the gang.  Return true if initialization succeeded.
   bool initialize_workers();
 
   bool are_GC_task_threads()      const { return _are_GC_task_threads; }
   bool are_ConcurrentGC_threads() const { return _are_ConcurrentGC_threads; }

@@ -129,103 +158,48 @@
   // Printing
   void print_worker_threads_on(outputStream *st) const;
   void print_worker_threads() const {
     print_worker_threads_on(tty);
   }
+
+ protected:
+  virtual AbstractGangWorker* allocate_worker(uint which) = 0;
 };
 
 // An class representing a gang of workers.
 class WorkGang: public AbstractWorkGang {
-private:
+  // To get access to the GangTaskDispatcher instance.
+  friend class GangWorker;
+
   // Never deleted.
   ~WorkGang();
+
+  GangTaskDispatcher* const _dispatcher;
+  GangTaskDispatcher* dispatcher()const {
+    return _dispatcher;
+  }
+
 public:
   WorkGang(const char* name,
            uint workers,
            bool are_GC_task_threads,
            bool are_ConcurrentGC_threads);
 
   // Run a task, returns when the task is done.
   virtual void run_task(AbstractGangTask* task);
-  void run_task(AbstractGangTask* task, uint no_of_parallel_workers);
-
-  // Return true if more workers should be applied to the task.
-  virtual bool needs_more_workers() const {
-    return _started_workers < _active_workers;
-  }
 
 protected:
-  // The monitor which protects these data,
-  // and notifies of changes in it.
-  Monitor*  _monitor;
-  // The task for this gang.
-  AbstractGangTask* _task;
-  // A sequence number for the current task.
-  int _sequence_number;
-  // The number of started workers.
-  uint _started_workers;
-  // The number of finished workers.
-  uint _finished_workers;
-
-public:
   virtual AbstractGangWorker* allocate_worker(uint which);
 
-  // Accessors for fields
-  Monitor* monitor() const {
-    return _monitor;
-  }
-  AbstractGangTask* task() const {
-    return _task;
-  }
-  int sequence_number() const {
-    return _sequence_number;
-  }
-  uint started_workers() const {
-    return _started_workers;
-  }
-  uint finished_workers() const {
-    return _finished_workers;
-  }
-  // Predicates.
-  bool is_idle() const {
-    return (task() == NULL);
-  }
-  // Return the Ith gang worker.
-  GangWorker* gang_worker(uint i) const;
-
-protected:
-  friend class GangWorker;
-  // Note activation and deactivation of workers.
-  // These methods should only be called with the mutex held.
-  void internal_worker_poll(WorkData* data) const;
-  void internal_note_start();
-  void internal_note_finish();
-};
-
-class WorkData: public StackObj {
-  // This would be a struct, but I want accessor methods.
 private:
-  AbstractGangTask* _task;
-  int               _sequence_number;
-public:
-  // Constructor and destructor
-  WorkData() {
-    _task            = NULL;
-    _sequence_number = 0;
-  }
-  ~WorkData() {
-  }
-  AbstractGangTask* task()               const { return _task; }
-  void set_task(AbstractGangTask* value)       { _task = value; }
-  int sequence_number()                  const { return _sequence_number; }
-  void set_sequence_number(int value)          { _sequence_number = value; }
+  void print_worker_started_task(AbstractGangTask* task, uint worker_id);
+  void print_worker_finished_task(AbstractGangTask* task, uint worker_id);
 };
 
 // Several instances of this class run in parallel as workers for a gang.
 class AbstractGangWorker: public WorkerThread {
 public:
-  // Constructors and destructor.
   AbstractGangWorker(AbstractWorkGang* gang, uint id);
 
   // The only real method: run a task for the gang.
   virtual void run();
   // Predicate for Thread

@@ -250,34 +224,20 @@
 
 protected:
   virtual void loop();
 
 private:
+  WorkData wait_for_task();
+  void run_task(WorkData work);
+  void signal_task_done();
+
+  void print_task_started(WorkData data);
+  void print_task_done(WorkData data);
+
   WorkGang* gang() const { return (WorkGang*)_gang; }
 };
 
-// Dynamic number of worker threads
-//
-// This type of work gang is used to run different numbers of
-// worker threads at different times.  The
-// number of workers run for a task is "_active_workers"
-// instead of "_total_workers" in a WorkGang.  The method
-// "needs_more_workers()" returns true until "_active_workers"
-// have been started and returns false afterwards.  The
-// implementation of "needs_more_workers()" in WorkGang always
-// returns true so that all workers are started.  The method
-// "loop()" in GangWorker was modified to ask "needs_more_workers()"
-// in its loop to decide if it should start working on a task.
-// A worker in "loop()" waits for notification on the WorkGang
-// monitor and execution of each worker as it checks for work
-// is serialized via the same monitor.  The "needs_more_workers()"
-// call is serialized and additionally the calculation for the
-// "part" (effectively the worker id for executing the task) is
-// serialized to give each worker a unique "part".  Workers that
-// are not needed for this tasks (i.e., "_active_workers" have
-// been started before it, continue to wait for work.
-
 // A class that acts as a synchronisation barrier. Workers enter
 // the barrier and must wait until all other workers have entered
 // before any of them may leave.
 
 class WorkGangBarrierSync : public StackObj {
< prev index next >