1. 04 Sep, 2024 13 commits
    • Tejun Heo's avatar
      sched_ext: Add a cgroup scheduler which uses flattened hierarchy · a4103eac
      Tejun Heo authored
      This patch adds scx_flatcg example scheduler which implements hierarchical
      weight-based cgroup CPU control by flattening the cgroup hierarchy into a
      single layer by compounding the active weight share at each level.
      
      This flattening of hierarchy can bring a substantial performance gain when
      the cgroup hierarchy is nested multiple levels. in a simple benchmark using
      wrk[8] on apache serving a CGI script calculating sha1sum of a small file,
      it outperforms CFS by ~3% with CPU controller disabled and by ~10% with two
      apache instances competing with 2:1 weight ratio nested four level deep.
      
      However, the gain comes at the cost of not being able to properly handle
      thundering herd of cgroups. For example, if many cgroups which are nested
      behind a low priority parent cgroup wake up around the same time, they may
      be able to consume more CPU cycles than they are entitled to. In many use
      cases, this isn't a real concern especially given the performance gain.
      Also, there are ways to mitigate the problem further by e.g. introducing an
      extra scheduling layer on cgroup delegation boundaries.
      
      v5: - Updated to specify SCX_OPS_HAS_CGROUP_WEIGHT instead of
            SCX_OPS_KNOB_CGROUP_WEIGHT.
      
      v4: - Revert reference counted kptr for cgv_node as the change caused easily
            reproducible stalls.
      
      v3: - Updated to reflect the core API changes including ops.init/exit_task()
            and direct dispatch from ops.select_cpu(). Fixes and improvements
            including additional statistics.
      
          - Use reference counted kptr for cgv_node instead of xchg'ing against
            stash location.
      
          - Dropped '-p' option.
      
      v2: - Use SCX_BUG[_ON]() to simplify error handling.
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      Reviewed-by: default avatarDavid Vernet <dvernet@meta.com>
      Acked-by: default avatarJosh Don <joshdon@google.com>
      Acked-by: default avatarHao Luo <haoluo@google.com>
      Acked-by: default avatarBarret Rhoden <brho@google.com>
      a4103eac
    • Tejun Heo's avatar
      sched_ext: Add cgroup support · 81951366
      Tejun Heo authored
      Add sched_ext_ops operations to init/exit cgroups, and track task migrations
      and config changes. A BPF scheduler may not implement or implement only
      subset of cgroup features. The implemented features can be indicated using
      %SCX_OPS_HAS_CGOUP_* flags. If cgroup configuration makes use of features
      that are not implemented, a warning is triggered.
      
      While a BPF scheduler is being enabled and disabled, relevant cgroup
      operations are locked out using scx_cgroup_rwsem. This avoids situations
      like task prep taking place while the task is being moved across cgroups,
      making things easier for BPF schedulers.
      
      v7: - cgroup interface file visibility toggling is dropped in favor just
            warning messages. Dynamically changing interface visiblity caused more
            confusion than helping.
      
      v6: - Updated to reflect the removal of SCX_KF_SLEEPABLE.
      
          - Updated to use CONFIG_GROUP_SCHED_WEIGHT and fixes for
            !CONFIG_FAIR_GROUP_SCHED && CONFIG_EXT_GROUP_SCHED.
      
      v5: - Flipped the locking order between scx_cgroup_rwsem and
            cpus_read_lock() to avoid locking order conflict w/ cpuset. Better
            documentation around locking.
      
          - sched_move_task() takes an early exit if the source and destination
            are identical. This triggered the warning in scx_cgroup_can_attach()
            as it left p->scx.cgrp_moving_from uncleared. Updated the cgroup
            migration path so that ops.cgroup_prep_move() is skipped for identity
            migrations so that its invocations always match ops.cgroup_move()
            one-to-one.
      
      v4: - Example schedulers moved into their own patches.
      
          - Fix build failure when !CONFIG_CGROUP_SCHED, reported by Andrea Righi.
      
      v3: - Make scx_example_pair switch all tasks by default.
      
          - Convert to BPF inline iterators.
      
          - scx_bpf_task_cgroup() is added to determine the current cgroup from
            CPU controller's POV. This allows BPF schedulers to accurately track
            CPU cgroup membership.
      
          - scx_example_flatcg added. This demonstrates flattened hierarchy
            implementation of CPU cgroup control and shows significant performance
            improvement when cgroups which are nested multiple levels are under
            competition.
      
      v2: - Build fixes for different CONFIG combinations.
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      Reviewed-by: default avatarDavid Vernet <dvernet@meta.com>
      Acked-by: default avatarJosh Don <joshdon@google.com>
      Acked-by: default avatarHao Luo <haoluo@google.com>
      Acked-by: default avatarBarret Rhoden <brho@google.com>
      Reported-by: default avatarkernel test robot <lkp@intel.com>
      Cc: Andrea Righi <andrea.righi@canonical.com>
      81951366
    • Tejun Heo's avatar
      sched: Introduce CONFIG_GROUP_SCHED_WEIGHT · e179e80c
      Tejun Heo authored
      sched_ext will soon add cgroup cpu.weigh support. The cgroup interface code
      is currently gated behind CONFIG_FAIR_GROUP_SCHED. As the fair class and/or
      SCX may implement the feature, put the interface code behind the new
      CONFIG_CGROUP_SCHED_WEIGHT which is selected by CONFIG_FAIR_GROUP_SCHED.
      This allows either sched class to enable the itnerface code without ading
      more complex CONFIG tests.
      
      When !CONFIG_FAIR_GROUP_SCHED, a dummy version of sched_group_set_shares()
      is added to support later CONFIG_CGROUP_SCHED_WEIGHT &&
      !CONFIG_FAIR_GROUP_SCHED builds.
      
      No functional changes.
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      e179e80c
    • Tejun Heo's avatar
      sched: Make cpu_shares_read_u64() use tg_weight() · 41082c1d
      Tejun Heo authored
      Move tg_weight() upward and make cpu_shares_read_u64() use it too. This
      makes the weight retrieval shared between cgroup v1 and v2 paths and will be
      used to implement cgroup support for sched_ext.
      
      No functional changes.
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      41082c1d
    • Tejun Heo's avatar
      sched: Expose css_tg() · 859dc4ec
      Tejun Heo authored
      A new BPF extensible sched_class will use css_tg() in the init and exit
      paths to visit all task_groups by walking cgroups.
      
      v4: __setscheduler_prio() is already exposed. Dropped from this patch.
      
      v3: Dropped SCHED_CHANGE_BLOCK() as upstream is adding more generic cleanup
          mechanism.
      
      v2: Expose SCHED_CHANGE_BLOCK() too and update the description.
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      Reviewed-by: default avatarDavid Vernet <dvernet@meta.com>
      Acked-by: default avatarJosh Don <joshdon@google.com>
      Acked-by: default avatarHao Luo <haoluo@google.com>
      Acked-by: default avatarBarret Rhoden <brho@google.com>
      859dc4ec
    • Tejun Heo's avatar
      sched_ext: TASK_DEAD tasks must be switched into SCX on ops_enable · a8532fac
      Tejun Heo authored
      During scx_ops_enable(), SCX needs to invoke the sleepable ops.init_task()
      on every task. To do this, it does get_task_struct() on each iterated task,
      drop the lock and then call ops.init_task().
      
      However, a TASK_DEAD task may already have lost all its usage count and be
      waiting for RCU grace period to be freed. If get_task_struct() is called on
      such task, use-after-free can happen. To avoid such situations,
      scx_ops_enable() skips initialization of TASK_DEAD tasks, which seems safe
      as they are never going to be scheduled again.
      
      Unfortunately, a racing sched_setscheduler(2) can grab the task before the
      task is unhashed and then continue to e.g. move the task from RT to SCX
      after TASK_DEAD is set and ops_enable skipped the task. As the task hasn't
      gone through scx_ops_init_task(), scx_ops_enable_task() called from
      switching_to_scx() triggers the following warning:
      
        sched_ext: Invalid task state transition 0 -> 3 for stress-ng-race-[2872]
        WARNING: CPU: 6 PID: 2367 at kernel/sched/ext.c:3327 scx_ops_enable_task+0x18f/0x1f0
        ...
        RIP: 0010:scx_ops_enable_task+0x18f/0x1f0
        ...
         switching_to_scx+0x13/0xa0
         __sched_setscheduler+0x84e/0xa50
         do_sched_setscheduler+0x104/0x1c0
         __x64_sys_sched_setscheduler+0x18/0x30
         do_syscall_64+0x7b/0x140
         entry_SYSCALL_64_after_hwframe+0x76/0x7e
      
      As in the ops_disable path, it just doesn't seem like a good idea to leave
      any task in an inconsistent state, even when the task is dead. The root
      cause is ops_enable not being able to tell reliably whether a task is truly
      dead (no one else is looking at it and it's about to be freed) and was
      testing TASK_DEAD instead. Fix it by testing the task's usage count
      directly.
      
      - ops_init no longer ignores TASK_DEAD tasks. As now all users iterate all
        tasks, @include_dead is removed from scx_task_iter_next_locked() along
        with dead task filtering.
      
      - tryget_task_struct() is added. Tasks are skipped iff tryget_task_struct()
        fails.
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      Cc: David Vernet <void@manifault.com>
      Cc: Peter Zijlstra <peterz@infradead.org>
      a8532fac
    • Tejun Heo's avatar
      sched_ext: TASK_DEAD tasks must be switched out of SCX on ops_disable · 61eeb9a9
      Tejun Heo authored
      scx_ops_disable_workfn() only switches !TASK_DEAD tasks out of SCX while
      calling scx_ops_exit_task() on all tasks including dead ones. This can leave
      a dead task on SCX but with SCX_TASK_NONE state, which is inconsistent.
      
      If another task was in the process of changing the TASK_DEAD task's
      scheduling class and grabs the rq lock after scx_ops_disable_workfn() is
      done with the task, the task ends up calling scx_ops_disable_task() on the
      dead task which is in an inconsistent state triggering a warning:
      
        WARNING: CPU: 6 PID: 3316 at kernel/sched/ext.c:3411 scx_ops_disable_task+0x12c/0x160
        ...
        RIP: 0010:scx_ops_disable_task+0x12c/0x160
        ...
        Call Trace:
         <TASK>
         check_class_changed+0x2c/0x70
         __sched_setscheduler+0x8a0/0xa50
         do_sched_setscheduler+0x104/0x1c0
         __x64_sys_sched_setscheduler+0x18/0x30
         do_syscall_64+0x7b/0x140
         entry_SYSCALL_64_after_hwframe+0x76/0x7e
        RIP: 0033:0x7f140d70ea5b
      
      There is no reason to leave dead tasks on SCX when unloading the BPF
      scheduler. Fix by making scx_ops_disable_workfn() eject all tasks including
      the dead ones from SCX.
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      61eeb9a9
    • Tejun Heo's avatar
      sched_ext: Remove sched_class->switch_class() · 37cb049e
      Tejun Heo authored
      With sched_ext converted to use put_prev_task() for class switch detection,
      there's no user of switch_class() left. Drop it.
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      Cc: Peter Zijlstra <peterz@infradead.org>
      37cb049e
    • Tejun Heo's avatar
      sched_ext: Remove switch_class_scx() · f422316d
      Tejun Heo authored
      Now that put_prev_task_scx() is called with @next on task switches, there's
      no reason to use sched_class.switch_class(). Rename switch_class_scx() to
      switch_class() and call it from put_prev_task_scx().
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      f422316d
    • Tejun Heo's avatar
      sched_ext: Relocate functions in kernel/sched/ext.c · 65aaf905
      Tejun Heo authored
      Relocate functions to ease the removal of switch_class_scx(). No functional
      changes.
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      65aaf905
    • Tejun Heo's avatar
      sched_ext: Unify regular and core-sched pick task paths · 753e2836
      Tejun Heo authored
      Because the BPF scheduler's dispatch path is invoked from balance(),
      sched_ext needs to invoke balance_one() on all sibling rq's before picking
      the next task for core-sched.
      
      Before the recent pick_next_task() updates, sched_ext couldn't share pick
      task between regular and core-sched paths because pick_next_task() depended
      on put_prev_task() being called on the current task. Tasks currently running
      on sibling rq's can't be put when one rq is trying to pick the next task, so
      pick_task_scx() had to have a separate mechanism to pick between a sibling
      rq's current task and the first task in its local DSQ.
      
      However, with the preceding updates, pick_next_task_scx() no longer depends
      on the current task being put and can compare the current task and the next
      in line statelessly, and the pick task logic should be shareable between
      regular and core-sched paths.
      
      Unify regular and core-sched pick task paths:
      
      - There's no reason to distinguish local and sibling picks anymore. @local
        is removed from balance_one().
      
      - pick_next_task_scx() is turned into pick_task_scx() by dropping the
        put_prev_set_next_task() call.
      
      - The old pick_task_scx() is dropped.
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      753e2836
    • Tejun Heo's avatar
      sched_ext: Replace SCX_TASK_BAL_KEEP with SCX_RQ_BAL_KEEP · 8b1451f2
      Tejun Heo authored
      SCX_TASK_BAL_KEEP is used by balance_one() to tell pick_next_task_scx() to
      keep running the current task. It's not really a task property. Replace it
      with SCX_RQ_BAL_KEEP which resides in rq->scx.flags and is a better fit for
      the usage. Also, the existing clearing rule is unnecessarily strict and
      makes it difficult to use with core-sched. Just clear it on entry to
      balance_one().
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      8b1451f2
    • Tejun Heo's avatar
      sched_ext: Don't call put_prev_task_scx() before picking the next task · 7c65ae81
      Tejun Heo authored
      fd03c5b8 ("sched: Rework pick_next_task()") changed the definition of
      pick_next_task() from:
      
        pick_next_task() := pick_task() + set_next_task(.first = true)
      
      to:
      
        pick_next_task(prev) := pick_task() + put_prev_task() + set_next_task(.first = true)
      
      making invoking put_prev_task() pick_next_task()'s responsibility. This
      reordering allows pick_task() to be shared between regular and core-sched
      paths and put_prev_task() to know the next task.
      
      sched_ext depended on put_prev_task_scx() enqueueing the current task before
      pick_next_task_scx() is called. While pulling sched/core changes,
      70cc76aa0d80 ("Merge branch 'tip/sched/core' into for-6.12") added an
      explicit put_prev_task_scx() call for SCX tasks in pick_next_task_scx()
      before picking the first task as a workaround.
      
      Clean it up and adopt the conventions that other sched classes are
      following.
      
      The operation of keeping running the current task was spread and required
      the task to be put on the local DSQ before picking:
      
        - balance_one() used SCX_TASK_BAL_KEEP to indicate that the task is still
          runnable, hasn't exhausted its slice, and thus should keep running.
      
        - put_prev_task_scx() enqueued the task to local DSQ if SCX_TASK_BAL_KEEP
          is set. It also called do_enqueue_task() with SCX_ENQ_LAST if it is the
          only runnable task. do_enqueue_task() in turn decided whether to use the
          local DSQ depending on SCX_OPS_ENQ_LAST.
      
      Consolidate the logic in balance_one() as it always knows whether it is
      going to keep the current task. balance_one() now considers all conditions
      where the current task should be kept and uses SCX_TASK_BAL_KEEP to tell
      pick_next_task_scx() to keep the current task instead of picking one from
      the local DSQ. Accordingly, SCX_ENQ_LAST handling is removed from
      put_prev_task_scx() and do_enqueue_task() and pick_next_task_scx() is
      updated to pick the current task if SCX_TASK_BAL_KEEP is set.
      
      The workaround put_prev_task[_scx]() calls are replaced with
      put_prev_set_next_task().
      
      This causes two behavior changes observable from the BPF scheduler:
      
      - When a task keep running, it no longer goes through enqueue/dequeue cycle
        and thus ops.stopping/running() transitions. The new behavior is better
        and all the existing schedulers should be able to handle the new behavior.
      
      - The BPF scheduler cannot keep executing the current task by enqueueing
        SCX_ENQ_LAST task to the local DSQ. If SCX_OPS_ENQ_LAST is specified, the
        BPF scheduler is responsible for resuming execution after each
        SCX_ENQ_LAST. SCX_OPS_ENQ_LAST is mostly useful for cases where scheduling
        decisions are not made on the local CPU - e.g. central or userspace-driven
        schedulin - and the new behavior is more logical and shouldn't pose any
        problems. SCX_OPS_ENQ_LAST demonstration from scx_qmap is dropped as it
        doesn't fit that well anymore and the last task handling is moved to the
        end of qmap_dispatch().
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      Cc: David Vernet <void@manifault.com>
      Cc: Andrea Righi <righi.andrea@gmail.com>
      Cc: Changwoo Min <multics69@gmail.com>
      Cc: Daniel Hodges <hodges.daniel.scott@gmail.com>
      Cc: Dan Schatzberg <schatzberg.dan@gmail.com>
      7c65ae81
  2. 03 Sep, 2024 11 commits
  3. 31 Aug, 2024 2 commits
    • Tejun Heo's avatar
      sched_ext: Use sched_clock_cpu() instead of rq_clock_task() in touch_core_sched() · 62607d03
      Tejun Heo authored
      Since 3cf78c5d ("sched_ext: Unpin and repin rq lock from
      balance_scx()"), sched_ext's balance path terminates rq_pin in the outermost
      function. This is simpler and in line with what other balance functions are
      doing but it loses control over rq->clock_update_flags which makes
      assert_clock_udpated() trigger if other CPUs pins the rq lock.
      
      The only place this matters is touch_core_sched() which uses the timestamp
      to order tasks from sibling rq's. Switch to sched_clock_cpu(). Later, it may
      be better to use per-core dispatch sequence number.
      
      v2: Use sched_clock_cpu() instead of ktime_get_ns() per David.
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      Fixes: 3cf78c5d ("sched_ext: Unpin and repin rq lock from balance_scx()")
      Acked-by: default avatarDavid Vernet <void@manifault.com>
      Cc: Peter Zijlstra <peterz@infradead.org>
      62607d03
    • Tejun Heo's avatar
      sched_ext: Use task_can_run_on_remote_rq() test in dispatch_to_local_dsq() · 0366017e
      Tejun Heo authored
      When deciding whether a task can be migrated to a CPU,
      dispatch_to_local_dsq() was open-coding p->cpus_allowed and scx_rq_online()
      tests instead of using task_can_run_on_remote_rq(). This had two problems.
      
      - It was missing is_migration_disabled() check and thus could try to migrate
        a task which shouldn't leading to assertion and scheduling failures.
      
      - It was testing p->cpus_ptr directly instead of using task_allowed_on_cpu()
        and thus failed to consider ISA compatibility.
      
      Update dispatch_to_local_dsq() to use task_can_run_on_remote_rq():
      
      - Move scx_ops_error() triggering into task_can_run_on_remote_rq().
      
      - When migration isn't allowed, fall back to the global DSQ instead of the
        source DSQ by returning DTL_INVALID. This is both simpler and an overall
        better behavior.
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      Cc: Peter Zijlstra <peterz@infradead.org>
      Acked-by: default avatarDavid Vernet <void@manifault.com>
      0366017e
  4. 28 Aug, 2024 1 commit
  5. 27 Aug, 2024 1 commit
    • Tejun Heo's avatar
      scx_central: Fix smatch checker warning · 59cfdf3f
      Tejun Heo authored
      ARRAY_ELEM_PTR() is an access macro used to help the BPF verifier not
      confused by offseted memory acceeses by yiedling a valid pointer or NULL in
      a way that's clear to the verifier. As such, the canonical usage involves
      checking NULL return from the macro. Note that in many cases, the NULL
      condition can never happen - they're there just to hint the verifier.
      
      In a bpf_loop in scx_central.bpf.c::central_dispatch(), the NULL check was
      incorrect in that there was another dereference of the pointer in addition
      to the NULL checked access. This worked as the pointer can never be NULL and
      the verifier could tell it would never be NULL in this case.
      
      However, this still looks wrong and trips smatch:
      
        ./tools/sched_ext/scx_central.bpf.c:205 ____central_dispatch()
        error: we previously assumed 'gimme' could be null (see line 201)
      
        ./tools/sched_ext/scx_central.bpf.c
            195
            196                         if (!scx_bpf_dispatch_nr_slots())
            197                                 break;
            198
            199                         /* central's gimme is never set */
            200                         gimme = ARRAY_ELEM_PTR(cpu_gimme_task, cpu, nr_cpu_ids);
            201                         if (gimme && !*gimme)
      				      ^^^^^
        If gimme is NULL
      
            202                                 continue;
            203
            204                         if (dispatch_to_cpu(cpu))
        --> 205                                 *gimme = false;
      
      Fix the NULL check so that there are no derefs if NULL. This doesn't change
      actual behavior.
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      Reported-by: default avatarDan Carpenter <dan.carpenter@linaro.org>
      Link: http://lkml.kernel.org/r/<955e1c3c-ace2-4a1d-b246-15b8196038a3@stanley.mountain>
      59cfdf3f
  6. 20 Aug, 2024 2 commits
    • Yipeng Zou's avatar
      sched_ext: Allow dequeue_task_scx to fail · 9ad2861b
      Yipeng Zou authored
      Since dequeue_task() allowed to fail, there is a compile error:
      
      kernel/sched/ext.c:3630:19: error: initialization of ‘bool (*)(struct rq*, struct task_struct *, int)’ {aka ‘_Bool (*)(struct rq *, struct task_struct *, int)’} from incompatible pointer type ‘void (*)(struct rq*, struct task_struct *, int)’
        3630 |  .dequeue_task  = dequeue_task_scx,
             |                   ^~~~~~~~~~~~~~~~
      
      Allow dequeue_task_scx to fail too.
      
      Fixes: 863ccdbb ("sched: Allow sched_class::dequeue_task() to fail")
      Signed-off-by: default avatarYipeng Zou <zouyipeng@huawei.com>
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      9ad2861b
    • Tejun Heo's avatar
      Merge branch 'tip/sched/core' into for-6.12 · 5ac99857
      Tejun Heo authored
      To receive 863ccdbb ("sched: Allow sched_class::dequeue_task() to fail")
      which makes sched_class.dequeue_task() return bool instead of void. This
      leads to compile breakage and will be fixed by a follow-up patch.
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      5ac99857
  7. 17 Aug, 2024 10 commits