[Date Prev][Date Next][Thread Prev][Thread Next][Date Index][Thread Index]

[Xen-devel] [PATCH] xen/sched: rework credit2 run-queue allocation



Currently the memory for each run-queue of the credit2 scheduler is
allocated at the scheduler's init function: for each cpu in the system
a struct csched2_runqueue_data is being allocated, even if the
current scheduler only handles one physical cpu or is configured to
work with a single run-queue. As each struct contains 4 cpumasks this
sums up to rather large memory sizes pretty fast.

Rework the memory allocation for run-queues to be done only when
needed, i.e. when adding a physical cpu to the scheduler requiring a
new run-queue.

In fact this fixes a bug in credit2 related to run-queue handling:
cpu_to_runqueue() will return the first free or matching run-queue,
which ever is found first. So in case a cpu is removed from credit2
this could result in e.g. run-queue 0 becoming free, so when another
cpu is added it will in any case be assigned to that free run-queue,
even if it would have found another run-queue matching later.

Signed-off-by: Juergen Gross <jgross@xxxxxxxx>
---
 xen/common/sched/credit2.c | 372 ++++++++++++++++++++++-----------------------
 1 file changed, 184 insertions(+), 188 deletions(-)

diff --git a/xen/common/sched/credit2.c b/xen/common/sched/credit2.c
index 256c1c01fc..49a3c553f1 100644
--- a/xen/common/sched/credit2.c
+++ b/xen/common/sched/credit2.c
@@ -466,8 +466,12 @@ custom_param("credit2_runqueue", parse_credit2_runqueue);
 struct csched2_runqueue_data {
     spinlock_t lock;           /* Lock for this runqueue                     */
 
+    struct list_head rql;      /* List of runqueues                          */
     struct list_head runq;     /* Ordered list of runnable vms               */
+    unsigned int refcnt;       /* How many CPUs reference this runqueue      */
+                               /* (including not yet active ones)            */
     unsigned int nr_cpus;      /* How many CPUs are sharing this runqueue    */
+                               /* (only active ones)                         */
     int id;                    /* ID of this runqueue (-1 if invalid)        */
 
     int load;                  /* Instantaneous load (num of non-idle units) */
@@ -495,8 +499,8 @@ struct csched2_private {
     unsigned int load_window_shift;    /* Lenght of load decaying window     */
     unsigned int ratelimit_us;         /* Rate limiting for this scheduler   */
 
-    cpumask_t active_queues;           /* Runqueues with (maybe) active cpus */
-    struct csched2_runqueue_data *rqd; /* Data of the various runqueues      */
+    unsigned int active_queues;        /* Number of active runqueues         */
+    struct list_head rql;              /* List of runqueues                  */
 
     cpumask_t initialized;             /* CPUs part of this scheduler        */
     struct list_head sdom;             /* List of domains (for debug key)    */
@@ -507,7 +511,7 @@ struct csched2_private {
  */
 struct csched2_pcpu {
     cpumask_t sibling_mask;            /* Siblings in the same runqueue      */
-    int runq_id;
+    struct csched2_runqueue_data *rqd; /* Runqueue for this CPU              */
 };
 
 /*
@@ -585,14 +589,13 @@ static inline struct csched2_dom *csched2_dom(const 
struct domain *d)
 /* CPU to runq_id macro */
 static inline int c2r(unsigned int cpu)
 {
-    return csched2_pcpu(cpu)->runq_id;
+    return csched2_pcpu(cpu)->rqd->id;
 }
 
 /* CPU to runqueue struct macro */
-static inline struct csched2_runqueue_data *c2rqd(const struct scheduler *ops,
-                                                  unsigned int cpu)
+static inline struct csched2_runqueue_data *c2rqd(unsigned int cpu)
 {
-    return &csched2_priv(ops)->rqd[c2r(cpu)];
+    return csched2_pcpu(cpu)->rqd;
 }
 
 /* Does the domain of this unit have a cap? */
@@ -803,36 +806,6 @@ static inline struct csched2_unit * runq_elem(struct 
list_head *elem)
     return list_entry(elem, struct csched2_unit, runq_elem);
 }
 
-static void activate_runqueue(struct csched2_private *prv, int rqi)
-{
-    struct csched2_runqueue_data *rqd;
-
-    rqd = prv->rqd + rqi;
-
-    BUG_ON(!cpumask_empty(&rqd->active));
-
-    rqd->max_weight = 1;
-    rqd->id = rqi;
-    INIT_LIST_HEAD(&rqd->svc);
-    INIT_LIST_HEAD(&rqd->runq);
-    spin_lock_init(&rqd->lock);
-
-    __cpumask_set_cpu(rqi, &prv->active_queues);
-}
-
-static void deactivate_runqueue(struct csched2_private *prv, int rqi)
-{
-    struct csched2_runqueue_data *rqd;
-
-    rqd = prv->rqd + rqi;
-
-    BUG_ON(!cpumask_empty(&rqd->active));
-
-    rqd->id = -1;
-
-    __cpumask_clear_cpu(rqi, &prv->active_queues);
-}
-
 static inline bool same_node(unsigned int cpua, unsigned int cpub)
 {
     return cpu_to_node(cpua) == cpu_to_node(cpub);
@@ -849,51 +822,71 @@ static inline bool same_core(unsigned int cpua, unsigned 
int cpub)
            cpu_to_core(cpua) == cpu_to_core(cpub);
 }
 
-static unsigned int
-cpu_to_runqueue(const struct csched2_private *prv, unsigned int cpu)
+static struct csched2_runqueue_data *
+cpu_add_to_runqueue(struct csched2_private *prv, unsigned int cpu)
 {
-    const struct csched2_runqueue_data *rqd;
-    unsigned int rqi;
+    struct csched2_runqueue_data *rqd, *rqd_new;
+    struct list_head *rqd_ins;
+    unsigned long flags;
+    int rqi = 0;
+    bool rqi_unused = false, rqd_valid = false;
+
+    rqd_new = xzalloc(struct csched2_runqueue_data);
 
-    for ( rqi = 0; rqi < nr_cpu_ids; rqi++ )
+    write_lock_irqsave(&prv->lock, flags);
+
+    rqd_ins = &prv->rql;
+    list_for_each_entry ( rqd, &prv->rql, rql )
     {
         unsigned int peer_cpu;
 
-        /*
-         * As soon as we come across an uninitialized runqueue, use it.
-         * In fact, either:
-         *  - we are initializing the first cpu, and we assign it to
-         *    runqueue 0. This is handy, especially if we are dealing
-         *    with the boot cpu (if credit2 is the default scheduler),
-         *    as we would not be able to use cpu_to_socket() and similar
-         *    helpers anyway (they're result of which is not reliable yet);
-         *  - we have gone through all the active runqueues, and have not
-         *    found anyone whose cpus' topology matches the one we are
-         *    dealing with, so activating a new runqueue is what we want.
-         */
-        if ( prv->rqd[rqi].id == -1 )
-            break;
+        /* Remember first unused queue index. */
+        if ( !rqi_unused && rqd->id > rqi )
+            rqi_unused = true;
 
-        rqd = prv->rqd + rqi;
-        BUG_ON(cpumask_empty(&rqd->active));
-
-        peer_cpu = cpumask_first(&rqd->active);
+        peer_cpu = rqd->pick_bias;
         BUG_ON(cpu_to_socket(cpu) == XEN_INVALID_SOCKET_ID ||
                cpu_to_socket(peer_cpu) == XEN_INVALID_SOCKET_ID);
 
-        if (opt_runqueue == OPT_RUNQUEUE_CPU)
-            continue;
         if ( opt_runqueue == OPT_RUNQUEUE_ALL ||
              (opt_runqueue == OPT_RUNQUEUE_CORE && same_core(peer_cpu, cpu)) ||
              (opt_runqueue == OPT_RUNQUEUE_SOCKET && same_socket(peer_cpu, 
cpu)) ||
              (opt_runqueue == OPT_RUNQUEUE_NODE && same_node(peer_cpu, cpu)) )
+        {
+            rqd_valid = true;
             break;
+        }
+
+        if ( !rqi_unused )
+        {
+            rqi++;
+            rqd_ins = &rqd->rql;
+        }
+    }
+
+    if ( !rqd_valid )
+    {
+        if ( !rqd_new )
+        {
+            rqd = ERR_PTR(-ENOMEM);
+            goto out;
+        }
+        rqd = rqd_new;
+        rqd_new = NULL;
+
+        list_add(&rqd->rql, rqd_ins);
+        rqd->pick_bias = cpu;
+        rqd->id = rqi;
     }
 
-    /* We really expect to be able to assign each cpu to a runqueue. */
-    BUG_ON(rqi >= nr_cpu_ids);
+    rqd->refcnt++;
+
+ out:
+    write_unlock_irqrestore(&prv->lock, flags);
+
+    xfree(rqd_new);
 
-    return rqi;
+    return rqd;
 }
 
 /* Find the domain with the highest weight. */
@@ -971,13 +964,13 @@ _runq_assign(struct csched2_unit *svc, struct 
csched2_runqueue_data *rqd)
 }
 
 static void
-runq_assign(const struct scheduler *ops, const struct sched_unit *unit)
+runq_assign(const struct sched_unit *unit)
 {
     struct csched2_unit *svc = unit->priv;
 
     ASSERT(svc->rqd == NULL);
 
-    _runq_assign(svc, c2rqd(ops, sched_unit_master(unit)));
+    _runq_assign(svc, c2rqd(sched_unit_master(unit)));
 }
 
 static void
@@ -998,11 +991,11 @@ _runq_deassign(struct csched2_unit *svc)
 }
 
 static void
-runq_deassign(const struct scheduler *ops, const struct sched_unit *unit)
+runq_deassign(const struct sched_unit *unit)
 {
     struct csched2_unit *svc = unit->priv;
 
-    ASSERT(svc->rqd == c2rqd(ops, sched_unit_master(unit)));
+    ASSERT(svc->rqd == c2rqd(sched_unit_master(unit)));
 
     _runq_deassign(svc);
 }
@@ -1271,12 +1264,11 @@ update_load(const struct scheduler *ops,
         update_svc_load(ops, svc, change, now);
 }
 
-static void
-runq_insert(const struct scheduler *ops, struct csched2_unit *svc)
+static void runq_insert(struct csched2_unit *svc)
 {
     struct list_head *iter;
     unsigned int cpu = sched_unit_master(svc->unit);
-    struct list_head * runq = &c2rqd(ops, cpu)->runq;
+    struct list_head *runq = &c2rqd(cpu)->runq;
     int pos = 0;
 
     ASSERT(spin_is_locked(get_sched_res(cpu)->schedule_lock));
@@ -1365,7 +1357,7 @@ static inline bool is_preemptable(const struct 
csched2_unit *svc,
 static s_time_t tickle_score(const struct scheduler *ops, s_time_t now,
                              const struct csched2_unit *new, unsigned int cpu)
 {
-    struct csched2_runqueue_data *rqd = c2rqd(ops, cpu);
+    struct csched2_runqueue_data *rqd = c2rqd(cpu);
     struct csched2_unit * cur = csched2_unit(curr_on_cpu(cpu));
     const struct csched2_private *prv = csched2_priv(ops);
     s_time_t score;
@@ -1441,7 +1433,7 @@ runq_tickle(const struct scheduler *ops, struct 
csched2_unit *new, s_time_t now)
     s_time_t max = 0;
     struct sched_unit *unit = new->unit;
     unsigned int bs, cpu = sched_unit_master(unit);
-    struct csched2_runqueue_data *rqd = c2rqd(ops, cpu);
+    struct csched2_runqueue_data *rqd = c2rqd(cpu);
     const cpumask_t *online = cpupool_domain_master_cpumask(unit->domain);
     cpumask_t mask;
 
@@ -1617,10 +1609,9 @@ runq_tickle(const struct scheduler *ops, struct 
csched2_unit *new, s_time_t now)
 /*
  * Credit-related code
  */
-static void reset_credit(const struct scheduler *ops, int cpu, s_time_t now,
-                         struct csched2_unit *snext)
+static void reset_credit(int cpu, s_time_t now, struct csched2_unit *snext)
 {
-    struct csched2_runqueue_data *rqd = c2rqd(ops, cpu);
+    struct csched2_runqueue_data *rqd = c2rqd(cpu);
     struct list_head *iter;
     int m;
 
@@ -1909,7 +1900,7 @@ unpark_parked_units(const struct scheduler *ops, struct 
list_head *units)
              * for the newly replenished budget.
              */
             ASSERT( svc->rqd != NULL );
-            ASSERT( c2rqd(ops, sched_unit_master(svc->unit)) == svc->rqd );
+            ASSERT( c2rqd(sched_unit_master(svc->unit)) == svc->rqd );
             __set_bit(__CSFLAG_delayed_runq_add, &svc->flags);
         }
         else if ( unit_runnable(svc->unit) )
@@ -1922,7 +1913,7 @@ unpark_parked_units(const struct scheduler *ops, struct 
list_head *units)
              */
             now = NOW();
             update_load(ops, svc->rqd, svc, 1, now);
-            runq_insert(ops, svc);
+            runq_insert(svc);
             runq_tickle(ops, svc, now);
         }
         list_del_init(&svc->parked_elem);
@@ -2087,7 +2078,7 @@ csched2_unit_sleep(const struct scheduler *ops, struct 
sched_unit *unit)
     }
     else if ( unit_on_runq(svc) )
     {
-        ASSERT(svc->rqd == c2rqd(ops, sched_unit_master(unit)));
+        ASSERT(svc->rqd == c2rqd(sched_unit_master(unit)));
         update_load(ops, svc->rqd, svc, -1, NOW());
         runq_remove(svc);
     }
@@ -2134,16 +2125,16 @@ csched2_unit_wake(const struct scheduler *ops, struct 
sched_unit *unit)
 
     /* Add into the new runqueue if necessary */
     if ( svc->rqd == NULL )
-        runq_assign(ops, unit);
+        runq_assign(unit);
     else
-        ASSERT(c2rqd(ops, sched_unit_master(unit)) == svc->rqd );
+        ASSERT(c2rqd(sched_unit_master(unit)) == svc->rqd );
 
     now = NOW();
 
     update_load(ops, svc->rqd, svc, 1, now);
 
     /* Put the UNIT on the runq */
-    runq_insert(ops, svc);
+    runq_insert(svc);
     runq_tickle(ops, svc, now);
 
 out:
@@ -2167,9 +2158,9 @@ csched2_context_saved(const struct scheduler *ops, struct 
sched_unit *unit)
     LIST_HEAD(were_parked);
 
     BUG_ON( !is_idle_unit(unit) &&
-            svc->rqd != c2rqd(ops, sched_unit_master(unit)));
+            svc->rqd != c2rqd(sched_unit_master(unit)));
     ASSERT(is_idle_unit(unit) ||
-           svc->rqd == c2rqd(ops, sched_unit_master(unit)));
+           svc->rqd == c2rqd(sched_unit_master(unit)));
 
     /* This unit is now eligible to be put on the runqueue again */
     __clear_bit(__CSFLAG_scheduled, &svc->flags);
@@ -2190,7 +2181,7 @@ csched2_context_saved(const struct scheduler *ops, struct 
sched_unit *unit)
     {
         ASSERT(!unit_on_runq(svc));
 
-        runq_insert(ops, svc);
+        runq_insert(svc);
         runq_tickle(ops, svc, now);
     }
     else if ( !is_idle_unit(unit) )
@@ -2206,13 +2197,13 @@ static struct sched_resource *
 csched2_res_pick(const struct scheduler *ops, const struct sched_unit *unit)
 {
     struct csched2_private *prv = csched2_priv(ops);
-    int i, min_rqi = -1, min_s_rqi = -1;
     unsigned int new_cpu, cpu = sched_unit_master(unit);
     struct csched2_unit *svc = csched2_unit(unit);
     s_time_t min_avgload = MAX_LOAD, min_s_avgload = MAX_LOAD;
     bool has_soft;
+    struct csched2_runqueue_data *rqd, *min_rqd = NULL, *min_s_rqd = NULL;
 
-    ASSERT(!cpumask_empty(&prv->active_queues));
+    ASSERT(!list_empty(&prv->rql));
 
     SCHED_STAT_CRANK(pick_resource);
 
@@ -2290,13 +2281,10 @@ csched2_res_pick(const struct scheduler *ops, const 
struct sched_unit *unit)
      * Find both runqueues in one pass.
      */
     has_soft = has_soft_affinity(unit);
-    for_each_cpu(i, &prv->active_queues)
+    list_for_each_entry ( rqd, &prv->rql, rql )
     {
-        struct csched2_runqueue_data *rqd;
         s_time_t rqd_avgload = MAX_LOAD;
 
-        rqd = prv->rqd + i;
-
         /*
          * If none of the cpus of this runqueue is in svc's hard-affinity,
          * skip the runqueue.
@@ -2339,18 +2327,18 @@ csched2_res_pick(const struct scheduler *ops, const 
struct sched_unit *unit)
             if ( cpumask_intersects(&mask, unit->cpu_soft_affinity) )
             {
                 min_s_avgload = rqd_avgload;
-                min_s_rqi = i;
+                min_s_rqd = rqd;
             }
         }
         /* In any case, keep the "hard-affinity minimum" updated too. */
         if ( rqd_avgload < min_avgload )
         {
             min_avgload = rqd_avgload;
-            min_rqi = i;
+            min_rqd = rqd;
         }
     }
 
-    if ( has_soft && min_s_rqi != -1 )
+    if ( has_soft && min_s_rqd )
     {
         /*
          * We have soft affinity, and we have a candidate runq, so go for it.
@@ -2370,9 +2358,9 @@ csched2_res_pick(const struct scheduler *ops, const 
struct sched_unit *unit)
         cpumask_and(cpumask_scratch_cpu(cpu), cpumask_scratch_cpu(cpu),
                     unit->cpu_soft_affinity);
         cpumask_and(cpumask_scratch_cpu(cpu), cpumask_scratch_cpu(cpu),
-                    &prv->rqd[min_s_rqi].active);
+                    &min_s_rqd->active);
     }
-    else if ( min_rqi != -1 )
+    else if ( min_rqd )
     {
         /*
          * Either we don't have soft-affinity, or we do, but we did not find
@@ -2384,7 +2372,7 @@ csched2_res_pick(const struct scheduler *ops, const 
struct sched_unit *unit)
          * with the cpus of the runq.
          */
         cpumask_and(cpumask_scratch_cpu(cpu), cpumask_scratch_cpu(cpu),
-                    &prv->rqd[min_rqi].active);
+                    &min_rqd->active);
     }
     else
     {
@@ -2393,14 +2381,13 @@ csched2_res_pick(const struct scheduler *ops, const 
struct sched_unit *unit)
          * contention).
          */
         new_cpu = get_fallback_cpu(svc);
-        min_rqi = c2r(new_cpu);
-        min_avgload = prv->rqd[min_rqi].b_avgload;
+        min_rqd = c2rqd(new_cpu);
+        min_avgload = min_rqd->b_avgload;
         goto out_up;
     }
 
-    new_cpu = cpumask_cycle(prv->rqd[min_rqi].pick_bias,
-                            cpumask_scratch_cpu(cpu));
-    prv->rqd[min_rqi].pick_bias = new_cpu;
+    new_cpu = cpumask_cycle(min_rqd->pick_bias, cpumask_scratch_cpu(cpu));
+    min_rqd->pick_bias = new_cpu;
     BUG_ON(new_cpu >= nr_cpu_ids);
 
  out_up:
@@ -2415,7 +2402,7 @@ csched2_res_pick(const struct scheduler *ops, const 
struct sched_unit *unit)
         } d;
         d.dom = unit->domain->domain_id;
         d.unit = unit->unit_id;
-        d.rq_id = min_rqi;
+        d.rq_id = min_rqd->id;
         d.b_avgload = min_avgload;
         d.new_cpu = new_cpu;
         __trace_var(TRC_CSCHED2_PICKED_CPU, 1,
@@ -2528,7 +2515,7 @@ static void migrate(const struct scheduler *ops,
         if ( on_runq )
         {
             update_load(ops, svc->rqd, NULL, 1, now);
-            runq_insert(ops, svc);
+            runq_insert(svc);
             runq_tickle(ops, svc, now);
             SCHED_STAT_CRANK(migrate_on_runq);
         }
@@ -2558,9 +2545,9 @@ static bool unit_is_migrateable(const struct csched2_unit 
*svc,
 static void balance_load(const struct scheduler *ops, int cpu, s_time_t now)
 {
     struct csched2_private *prv = csched2_priv(ops);
-    int i, max_delta_rqi;
     struct list_head *push_iter, *pull_iter;
     bool inner_load_updated = 0;
+    struct csched2_runqueue_data *rqd, *max_delta_rqd;
 
     balance_state_t st = { .best_push_svc = NULL, .best_pull_svc = NULL };
 
@@ -2572,22 +2559,22 @@ static void balance_load(const struct scheduler *ops, 
int cpu, s_time_t now)
      */
 
     ASSERT(spin_is_locked(get_sched_res(cpu)->schedule_lock));
-    st.lrqd = c2rqd(ops, cpu);
+    st.lrqd = c2rqd(cpu);
 
     update_runq_load(ops, st.lrqd, 0, now);
 
 retry:
-    max_delta_rqi = -1;
+    max_delta_rqd = NULL;
     if ( !read_trylock(&prv->lock) )
         return;
 
     st.load_delta = 0;
 
-    for_each_cpu(i, &prv->active_queues)
+    list_for_each_entry ( rqd, &prv->rql, rql )
     {
         s_time_t delta;
 
-        st.orqd = prv->rqd + i;
+        st.orqd = rqd;
 
         if ( st.orqd == st.lrqd
              || !spin_trylock(&st.orqd->lock) )
@@ -2602,7 +2589,7 @@ retry:
         if ( delta > st.load_delta )
         {
             st.load_delta = delta;
-            max_delta_rqi = i;
+            max_delta_rqd = rqd;
         }
 
         spin_unlock(&st.orqd->lock);
@@ -2610,7 +2597,7 @@ retry:
 
     /* Minimize holding the private scheduler lock. */
     read_unlock(&prv->lock);
-    if ( max_delta_rqi == -1 )
+    if ( !max_delta_rqd )
         goto out;
 
     {
@@ -2622,10 +2609,7 @@ retry:
         if ( st.orqd->b_avgload > load_max )
             load_max = st.orqd->b_avgload;
 
-        cpus_max = st.lrqd->nr_cpus;
-        i = st.orqd->nr_cpus;
-        if ( i > cpus_max )
-            cpus_max = i;
+        cpus_max = max(st.lrqd->nr_cpus, st.orqd->nr_cpus);
 
         if ( unlikely(tb_init_done) )
         {
@@ -2661,7 +2645,7 @@ retry:
      * meantime, try the process over again.  This can't deadlock
      * because if it doesn't get any other rqd locks, it will simply
      * give up and return. */
-    st.orqd = prv->rqd + max_delta_rqi;
+    st.orqd = max_delta_rqd;
     if ( !spin_trylock(&st.orqd->lock) )
         goto retry;
 
@@ -2752,7 +2736,7 @@ csched2_unit_migrate(
     ASSERT(cpumask_test_cpu(new_cpu, &csched2_priv(ops)->initialized));
     ASSERT(cpumask_test_cpu(new_cpu, unit->cpu_hard_affinity));
 
-    trqd = c2rqd(ops, new_cpu);
+    trqd = c2rqd(new_cpu);
 
     /*
      * Do the actual movement toward new_cpu, and update vc->processor.
@@ -2816,7 +2800,7 @@ csched2_dom_cntl(
                 struct csched2_unit *svc = csched2_unit(unit);
                 spinlock_t *lock = unit_schedule_lock(unit);
 
-                ASSERT(svc->rqd == c2rqd(ops, sched_unit_master(unit)));
+                ASSERT(svc->rqd == c2rqd(sched_unit_master(unit)));
 
                 svc->weight = sdom->weight;
                 update_max_weight(svc->rqd, svc->weight, old_weight);
@@ -2899,7 +2883,7 @@ csched2_dom_cntl(
                     if ( unit->is_running )
                     {
                         unsigned int cpu = sched_unit_master(unit);
-                        struct csched2_runqueue_data *rqd = c2rqd(ops, cpu);
+                        struct csched2_runqueue_data *rqd = c2rqd(cpu);
 
                         ASSERT(curr_on_cpu(cpu) == unit);
 
@@ -3094,7 +3078,7 @@ csched2_unit_insert(const struct scheduler *ops, struct 
sched_unit *unit)
     lock = unit_schedule_lock_irq(unit);
 
     /* Add unit to runqueue of initial processor */
-    runq_assign(ops, unit);
+    runq_assign(unit);
 
     unit_schedule_unlock_irq(lock, unit);
 
@@ -3127,7 +3111,7 @@ csched2_unit_remove(const struct scheduler *ops, struct 
sched_unit *unit)
     /* Remove from runqueue */
     lock = unit_schedule_lock_irq(unit);
 
-    runq_deassign(ops, unit);
+    runq_deassign(unit);
 
     unit_schedule_unlock_irq(lock, unit);
 
@@ -3141,7 +3125,7 @@ csched2_runtime(const struct scheduler *ops, int cpu,
 {
     s_time_t time, min_time;
     int rt_credit; /* Proposed runtime measured in credits */
-    struct csched2_runqueue_data *rqd = c2rqd(ops, cpu);
+    struct csched2_runqueue_data *rqd = c2rqd(cpu);
     struct list_head *runq = &rqd->runq;
     const struct csched2_private *prv = csched2_priv(ops);
 
@@ -3438,7 +3422,7 @@ static void csched2_schedule(
 
     BUG_ON(!cpumask_test_cpu(sched_cpu, &csched2_priv(ops)->initialized));
 
-    rqd = c2rqd(ops, sched_cpu);
+    rqd = c2rqd(sched_cpu);
     BUG_ON(!cpumask_test_cpu(sched_cpu, &rqd->active));
 
     ASSERT(spin_is_locked(get_sched_res(sched_cpu)->schedule_lock));
@@ -3552,7 +3536,7 @@ static void csched2_schedule(
          */
         if ( skipped_units == 0 && snext->credit <= CSCHED2_CREDIT_RESET )
         {
-            reset_credit(ops, sched_cpu, now, snext);
+            reset_credit(sched_cpu, now, snext);
             balance_load(ops, sched_cpu, now);
         }
 
@@ -3651,7 +3635,8 @@ csched2_dump(const struct scheduler *ops)
     struct list_head *iter_sdom;
     struct csched2_private *prv = csched2_priv(ops);
     unsigned long flags;
-    unsigned int i, j, loop;
+    unsigned int j, loop;
+    struct csched2_runqueue_data *rqd;
 
     /*
      * We need the private scheduler lock as we access global
@@ -3661,13 +3646,13 @@ csched2_dump(const struct scheduler *ops)
 
     printk("Active queues: %d\n"
            "\tdefault-weight     = %d\n",
-           cpumask_weight(&prv->active_queues),
+           prv->active_queues,
            CSCHED2_DEFAULT_WEIGHT);
-    for_each_cpu(i, &prv->active_queues)
+    list_for_each_entry ( rqd, &prv->rql, rql )
     {
         s_time_t fraction;
 
-        fraction = (prv->rqd[i].avgload * 100) >> prv->load_precision_shift;
+        fraction = (rqd->avgload * 100) >> prv->load_precision_shift;
 
         printk("Runqueue %d:\n"
                "\tncpus              = %u\n"
@@ -3676,21 +3661,21 @@ csched2_dump(const struct scheduler *ops)
                "\tpick_bias          = %u\n"
                "\tinstload           = %d\n"
                "\taveload            = %"PRI_stime" (~%"PRI_stime"%%)\n",
-               i,
-               prv->rqd[i].nr_cpus,
-               CPUMASK_PR(&prv->rqd[i].active),
-               prv->rqd[i].max_weight,
-               prv->rqd[i].pick_bias,
-               prv->rqd[i].load,
-               prv->rqd[i].avgload,
+               rqd->id,
+               rqd->nr_cpus,
+               CPUMASK_PR(&rqd->active),
+               rqd->max_weight,
+               rqd->pick_bias,
+               rqd->load,
+               rqd->avgload,
                fraction);
 
         printk("\tidlers: %*pb\n"
                "\ttickled: %*pb\n"
                "\tfully idle cores: %*pb\n",
-               CPUMASK_PR(&prv->rqd[i].idle),
-               CPUMASK_PR(&prv->rqd[i].tickled),
-               CPUMASK_PR(&prv->rqd[i].smt_idle));
+               CPUMASK_PR(&rqd->idle),
+               CPUMASK_PR(&rqd->tickled),
+               CPUMASK_PR(&rqd->smt_idle));
     }
 
     printk("Domain info:\n");
@@ -3722,16 +3707,15 @@ csched2_dump(const struct scheduler *ops)
         }
     }
 
-    for_each_cpu(i, &prv->active_queues)
+    list_for_each_entry ( rqd, &prv->rql, rql )
     {
-        struct csched2_runqueue_data *rqd = prv->rqd + i;
         struct list_head *iter, *runq = &rqd->runq;
         int loop = 0;
 
         /* We need the lock to scan the runqueue. */
         spin_lock(&rqd->lock);
 
-        printk("Runqueue %d:\n", i);
+        printk("Runqueue %d:\n", rqd->id);
 
         for_each_cpu(j, &rqd->active)
             dump_pcpu(ops, j);
@@ -3756,20 +3740,28 @@ csched2_dump(const struct scheduler *ops)
 static void *
 csched2_alloc_pdata(const struct scheduler *ops, int cpu)
 {
+    struct csched2_private *prv = csched2_priv(ops);
     struct csched2_pcpu *spc;
+    struct csched2_runqueue_data *rqd;
 
     spc = xzalloc(struct csched2_pcpu);
     if ( spc == NULL )
         return ERR_PTR(-ENOMEM);
 
-    /* Not in any runqueue yet */
-    spc->runq_id = -1;
+    rqd = cpu_add_to_runqueue(prv, cpu);
+    if ( IS_ERR(rqd) )
+    {
+        xfree(spc);
+        return rqd;
+    }
+
+    spc->rqd = rqd;
 
     return spc;
 }
 
 /* Returns the ID of the runqueue the cpu is assigned to. */
-static unsigned
+static struct csched2_runqueue_data *
 init_pdata(struct csched2_private *prv, struct csched2_pcpu *spc,
            unsigned int cpu)
 {
@@ -3779,18 +3771,23 @@ init_pdata(struct csched2_private *prv, struct 
csched2_pcpu *spc,
     ASSERT(rw_is_write_locked(&prv->lock));
     ASSERT(!cpumask_test_cpu(cpu, &prv->initialized));
     /* CPU data needs to be allocated, but still uninitialized. */
-    ASSERT(spc && spc->runq_id == -1);
+    ASSERT(spc);
 
-    /* Figure out which runqueue to put it in */
-    spc->runq_id = cpu_to_runqueue(prv, cpu);
+    rqd = spc->rqd;
 
-    rqd = prv->rqd + spc->runq_id;
+    ASSERT(rqd && !cpumask_test_cpu(cpu, &spc->rqd->active));
 
-    printk(XENLOG_INFO "Adding cpu %d to runqueue %d\n", cpu, spc->runq_id);
-    if ( ! cpumask_test_cpu(spc->runq_id, &prv->active_queues) )
+    printk(XENLOG_INFO "Adding cpu %d to runqueue %d\n", cpu, rqd->id);
+    if ( !rqd->nr_cpus )
     {
         printk(XENLOG_INFO " First cpu on runqueue, activating\n");
-        activate_runqueue(prv, spc->runq_id);
+
+        BUG_ON(!cpumask_empty(&rqd->active));
+        rqd->max_weight = 1;
+        INIT_LIST_HEAD(&rqd->svc);
+        INIT_LIST_HEAD(&rqd->runq);
+        spin_lock_init(&rqd->lock);
+        prv->active_queues++;
     }
 
     __cpumask_set_cpu(cpu, &spc->sibling_mask);
@@ -3814,7 +3811,7 @@ init_pdata(struct csched2_private *prv, struct 
csched2_pcpu *spc,
     if ( rqd->nr_cpus == 1 )
         rqd->pick_bias = cpu;
 
-    return spc->runq_id;
+    return rqd;
 }
 
 static void
@@ -3823,14 +3820,14 @@ csched2_init_pdata(const struct scheduler *ops, void 
*pdata, int cpu)
     struct csched2_private *prv = csched2_priv(ops);
     spinlock_t *old_lock;
     unsigned long flags;
-    unsigned rqi;
+    struct csched2_runqueue_data *rqd;
 
     write_lock_irqsave(&prv->lock, flags);
     old_lock = pcpu_schedule_lock(cpu);
 
-    rqi = init_pdata(prv, pdata, cpu);
+    rqd = init_pdata(prv, pdata, cpu);
     /* Move the scheduler lock to the new runq lock. */
-    get_sched_res(cpu)->schedule_lock = &prv->rqd[rqi].lock;
+    get_sched_res(cpu)->schedule_lock = &rqd->lock;
 
     /* _Not_ pcpu_schedule_unlock(): schedule_lock may have changed! */
     spin_unlock(old_lock);
@@ -3844,7 +3841,7 @@ csched2_switch_sched(struct scheduler *new_ops, unsigned 
int cpu,
 {
     struct csched2_private *prv = csched2_priv(new_ops);
     struct csched2_unit *svc = vdata;
-    unsigned rqi;
+    struct csched2_runqueue_data *rqd;
 
     ASSERT(pdata && svc && is_idle_unit(svc->unit));
 
@@ -3861,7 +3858,7 @@ csched2_switch_sched(struct scheduler *new_ops, unsigned 
int cpu,
 
     sched_idle_unit(cpu)->priv = vdata;
 
-    rqi = init_pdata(prv, pdata, cpu);
+    rqd = init_pdata(prv, pdata, cpu);
 
     /*
      * Now that we know what runqueue we'll go in, double check what's said
@@ -3869,11 +3866,11 @@ csched2_switch_sched(struct scheduler *new_ops, 
unsigned int cpu,
      * this scheduler, and so it's safe to have taken it /before/ our
      * private global lock.
      */
-    ASSERT(get_sched_res(cpu)->schedule_lock != &prv->rqd[rqi].lock);
+    ASSERT(get_sched_res(cpu)->schedule_lock != &rqd->lock);
 
     write_unlock(&prv->lock);
 
-    return &prv->rqd[rqi].lock;
+    return &rqd->lock;
 }
 
 static void
@@ -3899,16 +3896,16 @@ csched2_deinit_pdata(const struct scheduler *ops, void 
*pcpu, int cpu)
      *  2. init_pdata must have been called on this cpu, and deinit_pdata
      *     (us!) must not have been called on it already.
      */
-    ASSERT(spc && spc->runq_id != -1);
+    ASSERT(spc && spc->rqd);
     ASSERT(cpumask_test_cpu(cpu, &prv->initialized));
 
     /* Find the old runqueue and remove this cpu from it */
-    rqd = prv->rqd + spc->runq_id;
+    rqd = spc->rqd;
 
     /* No need to save IRQs here, they're already disabled */
     spin_lock(&rqd->lock);
 
-    printk(XENLOG_INFO "Removing cpu %d from runqueue %d\n", cpu, 
spc->runq_id);
+    printk(XENLOG_INFO "Removing cpu %d from runqueue %d\n", cpu, rqd->id);
 
     __cpumask_clear_cpu(cpu, &rqd->idle);
     __cpumask_clear_cpu(cpu, &rqd->smt_idle);
@@ -3923,13 +3920,13 @@ csched2_deinit_pdata(const struct scheduler *ops, void 
*pcpu, int cpu)
     if ( rqd->nr_cpus == 0 )
     {
         printk(XENLOG_INFO " No cpus left on runqueue, disabling\n");
-        deactivate_runqueue(prv, spc->runq_id);
+
+        BUG_ON(!cpumask_empty(&rqd->active));
+        prv->active_queues--;
     }
     else if ( rqd->pick_bias == cpu )
         rqd->pick_bias = cpumask_first(&rqd->active);
 
-    spc->runq_id = -1;
-
     spin_unlock(&rqd->lock);
 
     __cpumask_clear_cpu(cpu, &prv->initialized);
@@ -3942,18 +3939,29 @@ csched2_deinit_pdata(const struct scheduler *ops, void 
*pcpu, int cpu)
 static void
 csched2_free_pdata(const struct scheduler *ops, void *pcpu, int cpu)
 {
+    struct csched2_private *prv = csched2_priv(ops);
     struct csched2_pcpu *spc = pcpu;
+    struct csched2_runqueue_data *rqd;
+    unsigned long flags;
 
-    /*
-     * pcpu either points to a valid struct csched2_pcpu, or is NULL (if
-     * CPU bringup failed, and we're beeing called from CPU_UP_CANCELLED).
-     * xfree() does not really mind, but we want to be sure that either
-     * init_pdata has never been called, or deinit_pdata has been called
-     * already.
-     */
-    ASSERT(!pcpu || spc->runq_id == -1);
-    ASSERT(!cpumask_test_cpu(cpu, &csched2_priv(ops)->initialized));
+    if ( !spc )
+        return;
+
+    write_lock_irqsave(&prv->lock, flags);
+
+    rqd = spc->rqd;
+    ASSERT(rqd && rqd->refcnt);
+    ASSERT(!cpumask_test_cpu(cpu, &prv->initialized));
+
+    rqd->refcnt--;
+    if ( !rqd->refcnt )
+        list_del(&rqd->rql);
+    else
+        rqd = NULL;
+
+    write_unlock_irqrestore(&prv->lock, flags);
 
+    xfree(rqd);
     xfree(pcpu);
 }
 
@@ -3987,7 +3995,6 @@ csched2_global_init(void)
 static int
 csched2_init(struct scheduler *ops)
 {
-    int i;
     struct csched2_private *prv;
 
     printk("Initializing Credit2 scheduler\n");
@@ -4020,18 +4027,9 @@ csched2_init(struct scheduler *ops)
     ops->sched_data = prv;
 
     rwlock_init(&prv->lock);
+    INIT_LIST_HEAD(&prv->rql);
     INIT_LIST_HEAD(&prv->sdom);
 
-    /* Allocate all runqueues and mark them as un-initialized */
-    prv->rqd = xzalloc_array(struct csched2_runqueue_data, nr_cpu_ids);
-    if ( !prv->rqd )
-    {
-        xfree(prv);
-        return -ENOMEM;
-    }
-    for ( i = 0; i < nr_cpu_ids; i++ )
-        prv->rqd[i].id = -1;
-
     /* initialize ratelimit */
     prv->ratelimit_us = sched_ratelimit_us;
 
@@ -4049,8 +4047,6 @@ csched2_deinit(struct scheduler *ops)
 
     prv = csched2_priv(ops);
     ops->sched_data = NULL;
-    if ( prv )
-        xfree(prv->rqd);
     xfree(prv);
 }
 
-- 
2.16.4


_______________________________________________
Xen-devel mailing list
Xen-devel@xxxxxxxxxxxxxxxxxxxx
https://lists.xenproject.org/mailman/listinfo/xen-devel

 


Rackspace

Lists.xenproject.org is hosted with RackSpace, monitoring our
servers 24x7x365 and backed by RackSpace's Fanatical Support®.