Skip to content
Snippets Groups Projects
  1. Sep 25, 2024
  2. Sep 09, 2024
    • Tejun Heo's avatar
      scx_qmap: Implement highpri boosting · 2d285d56
      Tejun Heo authored
      
      Implement a silly boosting mechanism for nice -20 tasks. The only purpose is
      demonstrating and testing scx_bpf_dispatch_from_dsq(). The boosting only
      works within SHARED_DSQ and makes only minor differences with increased
      dispatch batch (-b).
      
      This exercises moving tasks to a user DSQ and all local DSQs from
      ops.dispatch() and BPF timerfn.
      
      v2: - Updated to use scx_bpf_dispatch_from_dsq_set_{slice|vtime}().
      
          - Drop the workaround for the iterated tasks not being trusted by the
            verifier. The issue is fixed from BPF side.
      
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      Cc: Daniel Hodges <hodges.daniel.scott@gmail.com>
      Cc: David Vernet <void@manifault.com>
      Cc: Changwoo Min <multics69@gmail.com>
      Cc: Andrea Righi <andrea.righi@linux.dev>
      Cc: Dan Schatzberg <schatzberg.dan@gmail.com>
      2d285d56
  3. Sep 04, 2024
    • 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
  4. Jul 12, 2024
    • Tejun Heo's avatar
      sched_ext/scx_qmap: Pick idle CPU for direct dispatch on !wakeup enqueues · 1edab907
      Tejun Heo authored
      
      Because there was no way to directly dispatch to the local DSQ of a remote
      CPU from ops.enqueue(), scx_qmap skipped looking for an idle CPU on !wakeup
      enqueues. This restriction was removed and sched_ext now allows
      SCX_DSQ_LOCAL_ON verdicts for direct dispatches.
      
      Factor out pick_direct_dispatch_cpu() from ops.select_cpu() and use it to
      direct dispatch from ops.enqueue() on !wakeup enqueues.
      
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      Acked-by: default avatarDavid Vernet <void@manifault.com>
      Cc: Dan Schatzberg <schatzberg.dan@gmail.com>
      Cc: Changwoo Min <changwoo@igalia.com>
      Cc: Andrea Righi <righi.andrea@gmail.com>
      1edab907
  5. Jul 09, 2024
  6. Jun 21, 2024
    • Tejun Heo's avatar
      sched_ext: Add cpuperf support · d86adb4f
      Tejun Heo authored
      
      sched_ext currently does not integrate with schedutil. When schedutil is the
      governor, frequencies are left unregulated and usually get stuck close to
      the highest performance level from running RT tasks.
      
      Add CPU performance monitoring and scaling support by integrating into
      schedutil. The following kfuncs are added:
      
      - scx_bpf_cpuperf_cap(): Query the relative performance capacity of
        different CPUs in the system.
      
      - scx_bpf_cpuperf_cur(): Query the current performance level of a CPU
        relative to its max performance.
      
      - scx_bpf_cpuperf_set(): Set the current target performance level of a CPU.
      
      This gives direct control over CPU performance setting to the BPF scheduler.
      The only changes on the schedutil side are accounting for the utilization
      factor from sched_ext and disabling frequency holding heuristics as it may
      not apply well to sched_ext schedulers which may have a lot weaker
      connection between tasks and their current / last CPU.
      
      With cpuperf support added, there is no reason to block uclamp. Enable while
      at it.
      
      A toy implementation of cpuperf is added to scx_qmap as a demonstration of
      the feature.
      
      v2: Ignore cpu_util_cfs_boost() when scx_switched_all() in sugov_get_util()
          to avoid factoring in stale util metric. (Christian)
      
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      Reviewed-by: default avatarDavid Vernet <dvernet@meta.com>
      Cc: Rafael J. Wysocki <rafael.j.wysocki@intel.com>
      Cc: Viresh Kumar <viresh.kumar@linaro.org>
      Cc: Christian Loehle <christian.loehle@arm.com>
      d86adb4f
  7. Jun 18, 2024
    • Tejun Heo's avatar
      sched_ext: Implement core-sched support · 7b0888b7
      Tejun Heo authored
      
      The core-sched support is composed of the following parts:
      
      - task_struct->scx.core_sched_at is added. This is a timestamp which can be
        used to order tasks. Depending on whether the BPF scheduler implements
        custom ordering, it tracks either global FIFO ordering of all tasks or
        local-DSQ ordering within the dispatched tasks on a CPU.
      
      - prio_less() is updated to call scx_prio_less() when comparing SCX tasks.
        scx_prio_less() calls ops.core_sched_before() if available or uses the
        core_sched_at timestamp. For global FIFO ordering, the BPF scheduler
        doesn't need to do anything. Otherwise, it should implement
        ops.core_sched_before() which reflects the ordering.
      
      - When core-sched is enabled, balance_scx() balances all SMT siblings so
        that they all have tasks dispatched if necessary before pick_task_scx() is
        called. pick_task_scx() picks between the current task and the first
        dispatched task on the local DSQ based on availability and the
        core_sched_at timestamps. Note that FIFO ordering is expected among the
        already dispatched tasks whether running or on the local DSQ, so this path
        always compares core_sched_at instead of calling into
        ops.core_sched_before().
      
      qmap_core_sched_before() is added to scx_qmap. It scales the
      distances from the heads of the queues to compare the tasks across different
      priority queues and seems to behave as expected.
      
      v3: Fixed build error when !CONFIG_SCHED_SMT reported by Andrea Righi.
      
      v2: Sched core added the const qualifiers to prio_less task arguments.
          Explicitly drop them for ops.core_sched_before() task arguments. BPF
          enforces access control through the verifier, so the qualifier isn't
          actually operative and only gets in the way when interacting with
          various helpers.
      
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      Reviewed-by: default avatarDavid Vernet <dvernet@meta.com>
      Reviewed-by: default avatarJosh Don <joshdon@google.com>
      Cc: Andrea Righi <andrea.righi@canonical.com>
      7b0888b7
    • Tejun Heo's avatar
      sched_ext: Implement sched_ext_ops.cpu_online/offline() · 60c27fb5
      Tejun Heo authored
      
      Add ops.cpu_online/offline() which are invoked when CPUs come online and
      offline respectively. As the enqueue path already automatically bypasses
      tasks to the local dsq on a deactivated CPU, BPF schedulers are guaranteed
      to see tasks only on CPUs which are between online() and offline().
      
      If the BPF scheduler doesn't implement ops.cpu_online/offline(), the
      scheduler is automatically exited with SCX_ECODE_RESTART |
      SCX_ECODE_RSN_HOTPLUG. Userspace can implement CPU hotpplug support
      trivially by simply reinitializing and reloading the scheduler.
      
      scx_qmap is updated to print out online CPUs on hotplug events. Other
      schedulers are updated to restart based on ecode.
      
      v3: - The previous implementation added @reason to
            sched_class.rq_on/offline() to distinguish between CPU hotplug events
            and topology updates. This was buggy and fragile as the methods are
            skipped if the current state equals the target state. Instead, add
            scx_rq_[de]activate() which are directly called from
            sched_cpu_de/activate(). This also allows ops.cpu_on/offline() to
            sleep which can be useful.
      
          - ops.dispatch() could be called on a CPU that the BPF scheduler was
            told to be offline. The dispatch patch is updated to bypass in such
            cases.
      
      v2: - To accommodate lock ordering change between scx_cgroup_rwsem and
            cpus_read_lock(), CPU hotplug operations are put into its own SCX_OPI
            block and enabled eariler during scx_ope_enable() so that
            cpus_read_lock() can be dropped before acquiring scx_cgroup_rwsem.
      
          - Auto exit with ECODE added.
      
      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>
      60c27fb5
    • David Vernet's avatar
      sched_ext: Implement sched_ext_ops.cpu_acquire/release() · 245254f7
      David Vernet authored
      
      Scheduler classes are strictly ordered and when a higher priority class has
      tasks to run, the lower priority ones lose access to the CPU. Being able to
      monitor and act on these events are necessary for use cases includling
      strict core-scheduling and latency management.
      
      This patch adds two operations ops.cpu_acquire() and .cpu_release(). The
      former is invoked when a CPU becomes available to the BPF scheduler and the
      opposite for the latter. This patch also implements
      scx_bpf_reenqueue_local() which can be called from .cpu_release() to trigger
      requeueing of all tasks in the local dsq of the CPU so that the tasks can be
      reassigned to other available CPUs.
      
      scx_pair is updated to use .cpu_acquire/release() along with
      %SCX_KICK_WAIT to make the pair scheduling guarantee strict even when a CPU
      is preempted by a higher priority scheduler class.
      
      scx_qmap is updated to use .cpu_acquire/release() to empty the local
      dsq of a preempted CPU. A similar approach can be adopted by BPF schedulers
      that want to have a tight control over latency.
      
      v4: Use the new SCX_KICK_IDLE to wake up a CPU after re-enqueueing.
      
      v3: Drop the const qualifier from scx_cpu_release_args.task. BPF enforces
          access control through the verifier, so the qualifier isn't actually
          operative and only gets in the way when interacting with various
          helpers.
      
      v2: Add p->scx.kf_mask annotation to allow calling scx_bpf_reenqueue_local()
          from ops.cpu_release() nested inside ops.init() and other sleepable
          operations.
      
      Signed-off-by: default avatarDavid Vernet <dvernet@meta.com>
      Reviewed-by: default avatarTejun Heo <tj@kernel.org>
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      Acked-by: default avatarJosh Don <joshdon@google.com>
      Acked-by: default avatarHao Luo <haoluo@google.com>
      Acked-by: default avatarBarret Rhoden <brho@google.com>
      245254f7
    • Tejun Heo's avatar
      sched_ext: Make watchdog handle ops.dispatch() looping stall · 0922f54f
      Tejun Heo authored
      
      The dispatch path retries if the local DSQ is still empty after
      ops.dispatch() either dispatched or consumed a task. This is both out of
      necessity and for convenience. It has to retry because the dispatch path
      might lose the tasks to dequeue while the rq lock is released while trying
      to migrate tasks across CPUs, and the retry mechanism makes ops.dispatch()
      implementation easier as it only needs to make some forward progress each
      iteration.
      
      However, this makes it possible for ops.dispatch() to stall CPUs by
      repeatedly dispatching ineligible tasks. If all CPUs are stalled that way,
      the watchdog or sysrq handler can't run and the system can't be saved. Let's
      address the issue by breaking out of the dispatch loop after 32 iterations.
      
      It is unlikely but not impossible for ops.dispatch() to legitimately go over
      the iteration limit. We want to come back to the dispatch path in such cases
      as not doing so risks stalling the CPU by idling with runnable tasks
      pending. As the previous task is still current in balance_scx(),
      resched_curr() doesn't do anything - it will just get cleared. Let's instead
      use scx_kick_bpf() which will trigger reschedule after switching to the next
      task which will likely be the idle task.
      
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      Reviewed-by: default avatarDavid Vernet <dvernet@meta.com>
      0922f54f
    • Tejun Heo's avatar
      sched_ext: Print debug dump after an error exit · 07814a94
      Tejun Heo authored
      
      If a BPF scheduler triggers an error, the scheduler is aborted and the
      system is reverted to the built-in scheduler. In the process, a lot of
      information which may be useful for figuring out what happened can be lost.
      
      This patch adds debug dump which captures information which may be useful
      for debugging including runqueue and runnable thread states at the time of
      failure. The following shows a debug dump after triggering the watchdog:
      
        root@test ~# os/work/tools/sched_ext/build/bin/scx_qmap -t 100
        stats  : enq=1 dsp=0 delta=1 deq=0
        stats  : enq=90 dsp=90 delta=0 deq=0
        stats  : enq=156 dsp=156 delta=0 deq=0
        stats  : enq=218 dsp=218 delta=0 deq=0
        stats  : enq=255 dsp=255 delta=0 deq=0
        stats  : enq=271 dsp=271 delta=0 deq=0
        stats  : enq=284 dsp=284 delta=0 deq=0
        stats  : enq=293 dsp=293 delta=0 deq=0
      
        DEBUG DUMP
        ================================================================================
      
        kworker/u32:12[320] triggered exit kind 1026:
          runnable task stall (stress[1530] failed to run for 6.841s)
      
        Backtrace:
          scx_watchdog_workfn+0x136/0x1c0
          process_scheduled_works+0x2b5/0x600
          worker_thread+0x269/0x360
          kthread+0xeb/0x110
          ret_from_fork+0x36/0x40
          ret_from_fork_asm+0x1a/0x30
      
        QMAP FIFO[0]:
        QMAP FIFO[1]:
        QMAP FIFO[2]: 1436
        QMAP FIFO[3]:
        QMAP FIFO[4]:
      
        CPU states
        ----------
      
        CPU 0   : nr_run=1 ops_qseq=244
      	    curr=swapper/0[0] class=idle_sched_class
      
          QMAP: dsp_idx=1 dsp_cnt=0
      
          R stress[1530] -6841ms
      	scx_state/flags=3/0x1 ops_state/qseq=2/20
      	sticky/holding_cpu=-1/-1 dsq_id=(n/a)
      	cpus=ff
      
            QMAP: force_local=0
      
            asm_sysvec_apic_timer_interrupt+0x16/0x20
      
        CPU 2   : nr_run=2 ops_qseq=142
      	    curr=swapper/2[0] class=idle_sched_class
      
          QMAP: dsp_idx=1 dsp_cnt=0
      
          R sshd[1703] -5905ms
      	scx_state/flags=3/0x9 ops_state/qseq=2/88
      	sticky/holding_cpu=-1/-1 dsq_id=(n/a)
      	cpus=ff
      
            QMAP: force_local=1
      
            __x64_sys_ppoll+0xf6/0x120
            do_syscall_64+0x7b/0x150
            entry_SYSCALL_64_after_hwframe+0x76/0x7e
      
          R fish[1539] -4141ms
      	scx_state/flags=3/0x9 ops_state/qseq=2/124
      	sticky/holding_cpu=-1/-1 dsq_id=(n/a)
      	cpus=ff
      
            QMAP: force_local=1
      
            futex_wait+0x60/0xe0
            do_futex+0x109/0x180
            __x64_sys_futex+0x117/0x190
            do_syscall_64+0x7b/0x150
            entry_SYSCALL_64_after_hwframe+0x76/0x7e
      
        CPU 3   : nr_run=2 ops_qseq=162
      	    curr=kworker/u32:12[320] class=ext_sched_class
      
          QMAP: dsp_idx=1 dsp_cnt=0
      
         *R kworker/u32:12[320] +0ms
      	scx_state/flags=3/0xd ops_state/qseq=0/0
      	sticky/holding_cpu=-1/-1 dsq_id=(n/a)
      	cpus=ff
      
            QMAP: force_local=0
      
            scx_dump_state+0x613/0x6f0
            scx_ops_error_irq_workfn+0x1f/0x40
            irq_work_run_list+0x82/0xd0
            irq_work_run+0x14/0x30
            __sysvec_irq_work+0x40/0x140
            sysvec_irq_work+0x60/0x70
            asm_sysvec_irq_work+0x16/0x20
            scx_watchdog_workfn+0x15f/0x1c0
            process_scheduled_works+0x2b5/0x600
            worker_thread+0x269/0x360
            kthread+0xeb/0x110
            ret_from_fork+0x36/0x40
            ret_from_fork_asm+0x1a/0x30
      
          R kworker/3:2[1436] +0ms
      	scx_state/flags=3/0x9 ops_state/qseq=2/160
      	sticky/holding_cpu=-1/-1 dsq_id=(n/a)
      	cpus=08
      
            QMAP: force_local=0
      
            kthread+0xeb/0x110
            ret_from_fork+0x36/0x40
            ret_from_fork_asm+0x1a/0x30
      
        CPU 7   : nr_run=0 ops_qseq=76
      	    curr=swapper/7[0] class=idle_sched_class
      
      
        ================================================================================
      
        EXIT: runnable task stall (stress[1530] failed to run for 6.841s)
      
      It shows that CPU 3 was running the watchdog when it triggered the error
      condition and the scx_qmap thread has been queued on CPU 0 for over 5
      seconds but failed to run. It also prints out scx_qmap specific information
      - e.g. which tasks are queued on each FIFO and so on using the dump_*() ops.
      This dump has proved pretty useful for developing and debugging BPF
      schedulers.
      
      Debug dump is generated automatically when the BPF scheduler exits due to an
      error. The debug buffer used in such cases is determined by
      sched_ext_ops.exit_dump_len and defaults to 32k. If the debug dump overruns
      the available buffer, the output is truncated and marked accordingly.
      
      Debug dump output can also be read through the sched_ext_dump tracepoint.
      When read through the tracepoint, there is no length limit.
      
      SysRq-D can be used to trigger debug dump at any time while a BPF scheduler
      is loaded. This is non-destructive - the scheduler keeps running afterwards.
      The output can be read through the sched_ext_dump tracepoint.
      
      v2: - The size of exit debug dump buffer can now be customized using
            sched_ext_ops.exit_dump_len.
      
          - sched_ext_ops.dump*() added to enable dumping of BPF scheduler
            specific information.
      
          - Tracpoint output and SysRq-D triggering added.
      
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      Reviewed-by: default avatarDavid Vernet <dvernet@meta.com>
      07814a94
    • Tejun Heo's avatar
      sched_ext: Allow BPF schedulers to disallow specific tasks from joining SCHED_EXT · 7bb6f081
      Tejun Heo authored
      
      BPF schedulers might not want to schedule certain tasks - e.g. kernel
      threads. This patch adds p->scx.disallow which can be set by BPF schedulers
      in such cases. The field can be changed anytime and setting it in
      ops.prep_enable() guarantees that the task can never be scheduled by
      sched_ext.
      
      scx_qmap is updated with the -d option to disallow a specific PID:
      
        # echo $$
        1092
        # grep -E '(policy)|(ext\.enabled)' /proc/self/sched
        policy                                       :                    0
        ext.enabled                                  :                    0
        # ./set-scx 1092
        # grep -E '(policy)|(ext\.enabled)' /proc/self/sched
        policy                                       :                    7
        ext.enabled                                  :                    0
      
      Run "scx_qmap -p -d 1092" in another terminal.
      
        # cat /sys/kernel/sched_ext/nr_rejected
        1
        # grep -E '(policy)|(ext\.enabled)' /proc/self/sched
        policy                                       :                    0
        ext.enabled                                  :                    0
        # ./set-scx 1092
        setparam failed for 1092 (Permission denied)
      
      - v4: Refreshed on top of tip:sched/core.
      
      - v3: Update description to reflect /sys/kernel/sched_ext interface change.
      
      - v2: Use atomic_long_t instead of atomic64_t for scx_kick_cpus_pnt_seqs to
            accommodate 32bit archs.
      
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      Suggested-by: default avatarBarret Rhoden <brho@google.com>
      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>
      7bb6f081
    • David Vernet's avatar
      sched_ext: Implement runnable task stall watchdog · 8a010b81
      David Vernet authored
      
      The most common and critical way that a BPF scheduler can misbehave is by
      failing to run runnable tasks for too long. This patch implements a
      watchdog.
      
      * All tasks record when they become runnable.
      
      * A watchdog work periodically scans all runnable tasks. If any task has
        stayed runnable for too long, the BPF scheduler is aborted.
      
      * scheduler_tick() monitors whether the watchdog itself is stuck. If so, the
        BPF scheduler is aborted.
      
      Because the watchdog only scans the tasks which are currently runnable and
      usually very infrequently, the overhead should be negligible.
      scx_qmap is updated so that it can be told to stall user and/or
      kernel tasks.
      
      A detected task stall looks like the following:
      
       sched_ext: BPF scheduler "qmap" errored, disabling
       sched_ext: runnable task stall (dbus-daemon[953] failed to run for 6.478s)
          scx_check_timeout_workfn+0x10e/0x1b0
          process_one_work+0x287/0x560
          worker_thread+0x234/0x420
          kthread+0xe9/0x100
          ret_from_fork+0x1f/0x30
      
      A detected watchdog stall:
      
       sched_ext: BPF scheduler "qmap" errored, disabling
       sched_ext: runnable task stall (watchdog failed to check in for 5.001s)
          scheduler_tick+0x2eb/0x340
          update_process_times+0x7a/0x90
          tick_sched_timer+0xd8/0x130
          __hrtimer_run_queues+0x178/0x3b0
          hrtimer_interrupt+0xfc/0x390
          __sysvec_apic_timer_interrupt+0xb7/0x2b0
          sysvec_apic_timer_interrupt+0x90/0xb0
          asm_sysvec_apic_timer_interrupt+0x1b/0x20
          default_idle+0x14/0x20
          arch_cpu_idle+0xf/0x20
          default_idle_call+0x50/0x90
          do_idle+0xe8/0x240
          cpu_startup_entry+0x1d/0x20
          kernel_init+0x0/0x190
          start_kernel+0x0/0x392
          start_kernel+0x324/0x392
          x86_64_start_reservations+0x2a/0x2c
          x86_64_start_kernel+0x104/0x109
          secondary_startup_64_no_verify+0xce/0xdb
      
      Note that this patch exposes scx_ops_error[_type]() in kernel/sched/ext.h to
      inline scx_notify_sched_tick().
      
      v4: - While disabling, cancel_delayed_work_sync(&scx_watchdog_work) was
            being called before forward progress was guaranteed and thus could
            lead to system lockup. Relocated.
      
          - While enabling, it was comparing msecs against jiffies without
            conversion leading to spurious load failures on lower HZ kernels.
            Fixed.
      
          - runnable list management is now used by core bypass logic and moved to
            the patch implementing sched_ext core.
      
      v3: - bpf_scx_init_member() was incorrectly comparing ops->timeout_ms
            against SCX_WATCHDOG_MAX_TIMEOUT which is in jiffies without
            conversion leading to spurious load failures in lower HZ kernels.
            Fixed.
      
      v2: - Julia Lawall noticed that the watchdog code was mixing msecs and
            jiffies. Fix by using jiffies for everything.
      
      Signed-off-by: default avatarDavid Vernet <dvernet@meta.com>
      Reviewed-by: default avatarTejun Heo <tj@kernel.org>
      Signed-off-by: default avatarTejun Heo <tj@kernel.org>
      Acked-by: default avatarJosh Don <joshdon@google.com>
      Acked-by: default avatarHao Luo <haoluo@google.com>
      Acked-by: default avatarBarret Rhoden <brho@google.com>
      Cc: Julia Lawall <julia.lawall@inria.fr>
      8a010b81
    • Tejun Heo's avatar
      sched_ext: Add scx_simple and scx_example_qmap example schedulers · 2a52ca7c
      Tejun Heo authored
      
      Add two simple example BPF schedulers - simple and qmap.
      
      * simple: In terms of scheduling, it behaves identical to not having any
        operation implemented at all. The two operations it implements are only to
        improve visibility and exit handling. On certain homogeneous
        configurations, this actually can perform pretty well.
      
      * qmap: A fixed five level priority scheduler to demonstrate queueing PIDs
        on BPF maps for scheduling. While not very practical, this is useful as a
        simple example and will be used to demonstrate different features.
      
      v7: - Compat helpers stripped out in prepartion of upstreaming as the
            upstreamed patchset will be the baselinfe. Utility macros that can be
            used to implement compat features are kept.
      
          - Explicitly disable map autoattach on struct_ops to avoid trying to
            attach twice while maintaining compatbility with older libbpf.
      
      v6: - Common header files reorganized and cleaned up. Compat helpers are
            added to demonstrate how schedulers can maintain backward
            compatibility with older kernels while making use of newly added
            features.
      
          - simple_select_cpu() added to keep track of the number of local
            dispatches. This is needed because the default ops.select_cpu()
            implementation is updated to dispatch directly and won't call
            ops.enqueue().
      
          - Updated to reflect the sched_ext API changes. Switching all tasks is
            the default behavior now and scx_qmap supports partial switching when
            `-p` is specified.
      
          - tools/sched_ext/Kconfig dropped. This will be included in the doc
            instead.
      
      v5: - Improve Makefile. Build artifects are now collected into a separate
            dir which change be changed. Install and help targets are added and
            clean actually cleans everything.
      
          - MEMBER_VPTR() improved to improve access to structs. ARRAY_ELEM_PTR()
            and RESIZEABLE_ARRAY() are added to support resizable arrays in .bss.
      
          - Add scx_common.h which provides common utilities to user code such as
            SCX_BUG[_ON]() and RESIZE_ARRAY().
      
          - Use SCX_BUG[_ON]() to simplify error handling.
      
      v4: - Dropped _example prefix from scheduler names.
      
      v3: - Rename scx_example_dummy to scx_example_simple and restructure a bit
            to ease later additions. Comment updates.
      
          - Added declarations for BPF inline iterators. In the future, hopefully,
            these will be consolidated into a generic BPF header so that they
            don't need to be replicated here.
      
      v2: - Updated with the generic BPF cpumask helpers.
      
      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>
      2a52ca7c
Loading