power: Add LowPowerMode1 service

Add a new service to limit the platform TDP while handles are still held. This
feature is currently limited to the Steam Deck.
This commit is contained in:
Vicki Pfau 2024-08-21 22:35:57 -07:00
parent e7d2d63ac8
commit 65a81cee47
10 changed files with 666 additions and 56 deletions

View file

@ -206,6 +206,42 @@
</interface>
<!--
com.steampowered.SteamOSManager1.LowPowerMode1
@short_description: Interface for handling a low power mode.
-->
<interface name="com.steampowered.SteamOSManager1.LowPowerMode1">
<!--
EnterDownloadMode:
Enter a low power mode for downloads and get a handle that will keep
this mode active until all obtained handles are dropped.
@identifier: A human-readable string to identify who has obtained this handle.
@handle: A file handle that must be retained to keep download mode
active. When all handles are closed, low power mode is exited.
-->
<method name="EnterDownloadMode">
<arg type="s" name="identifier" direction="in"/>
<arg type="h" name="handle" direction="out"/>
</method>
<!--
ListDownloadModeHandles
Get a list of all of the currently open download mode handles.
@identifiers: A dict of all open download mode handles, as a pair of
the identifier passed to EnterDownloadMode and the number of handles
requested for that identifier.
-->
<method name="ListDownloadModeHandles">
<arg type="a{su}" name="identifiers" direction="out"/>
</method>
</interface>
<!--
com.steampowered.SteamOSManager1.Manager2
@short_description: Interface for basic manager control.

View file

@ -31,6 +31,7 @@ systemd = "jupiter-fan-control.service"
[tdp_limit]
method = "gpu_hwmon"
download_mode_limit = 6
[tdp_limit.range]
min = 3

View file

@ -16,8 +16,9 @@ use steamos_manager::power::{CPUScalingGovernor, GPUPerformanceLevel, GPUPowerPr
use steamos_manager::proxy::{
AmbientLightSensor1Proxy, BatteryChargeLimit1Proxy, CpuScaling1Proxy, FactoryReset1Proxy,
FanControl1Proxy, GpuPerformanceLevel1Proxy, GpuPowerProfile1Proxy, HdmiCec1Proxy,
Manager2Proxy, PerformanceProfile1Proxy, Storage1Proxy, TdpLimit1Proxy, UpdateBios1Proxy,
UpdateDock1Proxy, WifiDebug1Proxy, WifiDebugDump1Proxy, WifiPowerManagement1Proxy,
LowPowerMode1Proxy, Manager2Proxy, PerformanceProfile1Proxy, Storage1Proxy, TdpLimit1Proxy,
UpdateBios1Proxy, UpdateDock1Proxy, WifiDebug1Proxy, WifiDebugDump1Proxy,
WifiPowerManagement1Proxy,
};
use steamos_manager::wifi::{WifiBackend, WifiDebugMode, WifiPowerManagement};
use zbus::fdo::{IntrospectableProxy, PropertiesProxy};
@ -170,6 +171,9 @@ enum Commands {
state: HdmiCecState,
},
/// List active low power download mode handles
ListLowPowerDownloadModeHandles,
/// Update the BIOS, if possible
UpdateBios,
@ -468,6 +472,13 @@ async fn main() -> Result<()> {
Err(_) => println!("Got unknown value {state} from backend"),
}
}
Commands::ListLowPowerDownloadModeHandles => {
let proxy = LowPowerMode1Proxy::new(&conn).await?;
let handles: HashMap<String, u32> = proxy.list_download_mode_handles().await?;
for (identifier, count) in handles.into_iter().sorted() {
println!("{identifier}: {count}");
}
}
Commands::UpdateBios => {
let proxy = UpdateBios1Proxy::new(&conn).await?;
let _ = proxy.update_bios().await?;

View file

@ -20,8 +20,8 @@ use crate::daemon::{channel, Daemon, DaemonCommand, DaemonContext};
use crate::job::{JobManager, JobManagerService};
use crate::manager::user::create_interfaces;
use crate::path;
use crate::power::TdpManagerService;
use crate::udev::UdevMonitor;
use crate::Service;
#[derive(Copy, Clone, Default, Deserialize, Debug)]
#[serde(default)]
@ -104,7 +104,7 @@ pub(crate) type Command = DaemonCommand<()>;
async fn create_connections(
channel: Sender<Command>,
) -> Result<(Connection, Connection, impl Service)> {
) -> Result<(Connection, Connection, JobManagerService, TdpManagerService)> {
let system = Connection::system().await?;
let connection = Builder::session()?
.name("com.steampowered.SteamOSManager1")?
@ -113,10 +113,14 @@ async fn create_connections(
let (jm_tx, rx) = unbounded_channel();
let job_manager = JobManager::new(connection.clone()).await?;
let service = JobManagerService::new(job_manager, rx, system.clone());
create_interfaces(connection.clone(), system.clone(), channel, jm_tx).await?;
let jm_service = JobManagerService::new(job_manager, rx, system.clone());
Ok((connection, system, service))
let (tdp_tx, rx) = unbounded_channel();
let tdp_service = TdpManagerService::new(rx, &system).await?;
create_interfaces(connection.clone(), system.clone(), channel, jm_tx, tdp_tx).await?;
Ok((connection, system, jm_service, tdp_service))
}
pub async fn daemon() -> Result<()> {
@ -127,7 +131,7 @@ pub async fn daemon() -> Result<()> {
let subscriber = Registry::default().with(stdout_log);
let (tx, rx) = channel::<UserContext>();
let (session, system, mirror_service) = match create_connections(tx).await {
let (session, system, mirror_service, tdp_service) = match create_connections(tx).await {
Ok(c) => c,
Err(e) => {
let _guard = tracing::subscriber::set_default(subscriber);
@ -140,6 +144,7 @@ pub async fn daemon() -> Result<()> {
let mut daemon = Daemon::new(subscriber, system, rx).await?;
daemon.add_service(mirror_service);
daemon.add_service(tdp_service);
daemon.run(context).await
}

View file

@ -15,7 +15,7 @@ use tokio::sync::oneshot;
use tracing::{error, info};
use zbus::object_server::SignalEmitter;
use zbus::zvariant::{self, Fd};
use zbus::{fdo, interface, Connection};
use zbus::{fdo, interface, proxy, Connection};
use crate::daemon::root::{Command, RootCommand};
use crate::daemon::DaemonCommand;
@ -91,6 +91,15 @@ impl SteamOSManager {
}
}
#[proxy(
interface = "com.steampowered.SteamOSManager1.RootManager",
default_service = "com.steampowered.SteamOSManager1",
default_path = "/com/steampowered/SteamOSManager1"
)]
pub(crate) trait RootManager {
fn set_tdp_limit(&self, limit: u32) -> zbus::Result<()>;
}
#[interface(name = "com.steampowered.SteamOSManager1.RootManager")]
impl SteamOSManager {
async fn prepare_factory_reset(&self, kind: u32) -> fdo::Result<u32> {

View file

@ -6,13 +6,14 @@
* SPDX-License-Identifier: MIT
*/
use anyhow::Result;
use anyhow::{Error, Result};
use std::collections::HashMap;
use tokio::sync::mpsc::{Sender, UnboundedSender};
use tokio::sync::oneshot;
use tracing::error;
use zbus::object_server::SignalEmitter;
use zbus::proxy::{Builder, CacheProperties};
use zbus::zvariant::Fd;
use zbus::{fdo, interface, zvariant, Connection, ObjectServer, Proxy};
use crate::cec::{HdmiCecControl, HdmiCecState};
@ -28,7 +29,7 @@ use crate::power::{
get_available_cpu_scaling_governors, get_available_gpu_performance_levels,
get_available_gpu_power_profiles, get_available_platform_profiles, get_cpu_scaling_governor,
get_gpu_clocks, get_gpu_clocks_range, get_gpu_performance_level, get_gpu_power_profile,
get_max_charge_level, get_platform_profile, tdp_limit_manager, TdpLimitManager,
get_max_charge_level, get_platform_profile, tdp_limit_manager, TdpManagerCommand,
};
use crate::wifi::{
get_wifi_backend, get_wifi_power_management_state, list_wifi_interfaces, WifiBackend,
@ -131,14 +132,17 @@ struct GpuPowerProfile1 {
}
struct TdpLimit1 {
proxy: Proxy<'static>,
manager: Box<dyn TdpLimitManager>,
manager: UnboundedSender<TdpManagerCommand>,
}
struct HdmiCec1 {
hdmi_cec: HdmiCecControl<'static>,
}
struct LowPowerMode1 {
manager: UnboundedSender<TdpManagerCommand>,
}
struct Manager2 {
proxy: Proxy<'static>,
channel: Sender<Command>,
@ -146,7 +150,7 @@ struct Manager2 {
struct PerformanceProfile1 {
proxy: Proxy<'static>,
tdp_limit_manager: Option<Box<dyn TdpLimitManager>>,
tdp_limit_manager: UnboundedSender<TdpManagerCommand>,
}
struct Storage1 {
@ -444,6 +448,39 @@ impl HdmiCec1 {
}
}
#[interface(name = "com.steampowered.SteamOSManager1.LowPowerMode1")]
impl LowPowerMode1 {
async fn enter_download_mode(&self, identifier: &str) -> fdo::Result<Fd> {
let (tx, rx) = oneshot::channel();
self.manager
.send(TdpManagerCommand::EnterDownloadMode(
identifier.to_string(),
tx,
))
.map_err(|_| {
fdo::Error::Failed(String::from("Failed to obtain download mode handle"))
})?;
Ok(rx
.await
.map_err(to_zbus_fdo_error)?
.map_err(to_zbus_fdo_error)?
.ok_or(fdo::Error::Failed(String::from(
"Download mode not configured",
)))?
.into())
}
async fn list_download_mode_handles(&self) -> fdo::Result<HashMap<String, u32>> {
let (tx, rx) = oneshot::channel();
self.manager
.send(TdpManagerCommand::ListDownloadModeHandles(tx))
.map_err(|_| {
fdo::Error::Failed(String::from("Failed to obtain download mode handle list"))
})?;
rx.await.map_err(to_zbus_fdo_error)
}
}
#[interface(name = "com.steampowered.SteamOSManager1.Manager2")]
impl Manager2 {
async fn reload_config(&self) -> fdo::Result<()> {
@ -499,22 +536,25 @@ impl PerformanceProfile1 {
#[zbus(connection)] connection: &Connection,
) -> zbus::Result<()> {
let _: () = self.proxy.call("SetPerformanceProfile", &(profile)).await?;
if self.tdp_limit_manager.is_some() {
let connection = connection.clone();
let manager = tdp_limit_manager().await.map_err(to_zbus_error)?;
let proxy = self.proxy.clone();
let manager = self.tdp_limit_manager.clone();
let _ = manager.send(TdpManagerCommand::UpdateDownloadMode);
tokio::spawn(async move {
if manager.is_active().await.map_err(to_zbus_error)? {
let tdp_limit = TdpLimit1 { proxy, manager };
connection.object_server().at(MANAGER_PATH, tdp_limit).await
let (tx, rx) = oneshot::channel();
manager.send(TdpManagerCommand::IsActive(tx))?;
Ok::<(), Error>(if rx.await?? {
let tdp_limit = TdpLimit1 { manager };
connection
.object_server()
.at(MANAGER_PATH, tdp_limit)
.await?;
} else {
connection
.object_server()
.remove::<TdpLimit1, _>(MANAGER_PATH)
.await
}
.await?;
})
});
}
Ok(())
}
@ -551,30 +591,56 @@ impl Storage1 {
impl TdpLimit1 {
#[zbus(property(emits_changed_signal = "false"))]
async fn tdp_limit(&self) -> u32 {
self.manager.get_tdp_limit().await.unwrap_or(0)
let (tx, rx) = oneshot::channel();
if self
.manager
.send(TdpManagerCommand::GetTdpLimit(tx))
.is_err()
{
return 0;
}
rx.await.unwrap_or(Ok(0)).unwrap_or(0)
}
#[zbus(property)]
async fn set_tdp_limit(&self, limit: u32) -> zbus::Result<()> {
self.proxy.call("SetTdpLimit", &(limit)).await
self.manager
.send(TdpManagerCommand::SetTdpLimit(limit))
.map_err(|_| zbus::Error::Failure(String::from("Failed to set TDP limit")))
}
#[zbus(property(emits_changed_signal = "const"))]
async fn tdp_limit_min(&self) -> u32 {
self.manager
.get_tdp_limit_range()
.await
.map(|r| *r.start())
.unwrap_or(0)
let (tx, rx) = oneshot::channel();
if self
.manager
.send(TdpManagerCommand::GetTdpLimitRange(tx))
.is_err()
{
return 0;
}
if let Ok(range) = rx.await {
range.map(|r| *r.start()).unwrap_or(0)
} else {
0
}
}
#[zbus(property(emits_changed_signal = "const"))]
async fn tdp_limit_max(&self) -> u32 {
self.manager
.get_tdp_limit_range()
.await
.map(|r| *r.end())
.unwrap_or(0)
let (tx, rx) = oneshot::channel();
if self
.manager
.send(TdpManagerCommand::GetTdpLimitRange(tx))
.is_err()
{
return 0;
}
if let Ok(range) = rx.await {
range.map(|r| *r.end()).unwrap_or(0)
} else {
0
}
}
}
@ -663,6 +729,7 @@ async fn create_config_interfaces(
proxy: &Proxy<'static>,
object_server: &ObjectServer,
job_manager: &UnboundedSender<JobManagerCommand>,
tdp_manager: &UnboundedSender<TdpManagerCommand>,
) -> Result<()> {
let Some(config) = platform_config().await? else {
return Ok(());
@ -676,7 +743,7 @@ async fn create_config_interfaces(
};
let performance_profile = PerformanceProfile1 {
proxy: proxy.clone(),
tdp_limit_manager: tdp_limit_manager().await.ok(),
tdp_limit_manager: tdp_manager.clone(),
};
let storage = Storage1 {
proxy: proxy.clone(),
@ -700,16 +767,22 @@ async fn create_config_interfaces(
}
if let Ok(manager) = tdp_limit_manager().await {
let low_power_mode = LowPowerMode1 {
manager: tdp_manager.clone(),
};
if config
.tdp_limit
.as_ref()
.and_then(|config| config.download_mode_limit)
.is_some()
{
object_server.at(MANAGER_PATH, low_power_mode).await?;
}
if manager.is_active().await? {
object_server
.at(
MANAGER_PATH,
TdpLimit1 {
proxy: proxy.clone(),
manager,
},
)
.await?;
let tdp_limit = TdpLimit1 {
manager: tdp_manager.clone(),
};
object_server.at(MANAGER_PATH, tdp_limit).await?;
}
}
@ -756,6 +829,7 @@ pub(crate) async fn create_interfaces(
system: Connection,
daemon: Sender<Command>,
job_manager: UnboundedSender<JobManagerCommand>,
tdp_manager: UnboundedSender<TdpManagerCommand>,
) -> Result<()> {
let proxy = Builder::<Proxy>::new(&system)
.destination("com.steampowered.SteamOSManager1")?
@ -800,7 +874,7 @@ pub(crate) async fn create_interfaces(
let object_server = session.object_server();
object_server.at(MANAGER_PATH, manager).await?;
create_config_interfaces(&proxy, object_server, &job_manager).await?;
create_config_interfaces(&proxy, object_server, &job_manager, &tdp_manager).await?;
if device_type().await.unwrap_or_default() == DeviceType::SteamDeck {
object_server.at(MANAGER_PATH, als).await?;
@ -867,6 +941,7 @@ mod test {
use crate::systemd::test::{MockManager, MockUnit};
use crate::{path, power, testing};
use std::num::NonZeroU32;
use std::os::unix::fs::PermissionsExt;
use std::time::Duration;
use tokio::fs::{set_permissions, write};
@ -892,6 +967,7 @@ mod test {
tdp_limit: Some(TdpLimitConfig {
method: TdpLimitingMethod::GpuHwmon,
range: Some(RangeConfig::new(3, 15)),
download_mode_limit: NonZeroU32::new(6),
}),
gpu_clocks: Some(RangeConfig::new(200, 1600)),
battery_charge_limit: Some(BatteryChargeLimitConfig {
@ -910,6 +986,7 @@ mod test {
let mut handle = testing::start();
let (tx_ctx, _rx_ctx) = channel::<UserContext>();
let (tx_job, _rx_job) = unbounded_channel::<JobManagerCommand>();
let (tx_tdp, _rx_tdp) = unbounded_channel::<TdpManagerCommand>();
if let Some(ref mut config) = platform_config {
config.set_test_paths();
@ -945,7 +1022,14 @@ mod test {
.process_cb
.set(|_, _| Ok((0, String::from("Interface wlan0"))));
power::test::create_nodes().await?;
create_interfaces(connection.clone(), connection.clone(), tx_ctx, tx_job).await?;
create_interfaces(
connection.clone(),
connection.clone(),
tx_ctx,
tx_job,
tx_tdp,
)
.await?;
sleep(Duration::from_millis(1)).await;
@ -1083,6 +1167,13 @@ mod test {
.unwrap());
}
#[tokio::test]
async fn interface_missing_tdp_limit1() {
let test = start(None).await.expect("start");
assert!(test_interface_missing::<TdpLimit1>(&test.connection).await);
}
#[tokio::test]
async fn interface_matches_hdmi_cec1() {
let test = start(all_config()).await.expect("start");
@ -1092,6 +1183,22 @@ mod test {
.unwrap());
}
#[tokio::test]
async fn interface_matches_low_power_mode1() {
let test = start(all_config()).await.expect("start");
assert!(test_interface_matches::<LowPowerMode1>(&test.connection)
.await
.unwrap());
}
#[tokio::test]
async fn interface_missing_low_power_mode1() {
let test = start(None).await.expect("start");
assert!(test_interface_missing::<LowPowerMode1>(&test.connection).await);
}
#[tokio::test]
async fn interface_matches_manager2() {
let test = start(all_config()).await.expect("start");

View file

@ -11,6 +11,7 @@ use nix::unistd::{access, AccessFlags};
use serde::de::Error;
use serde::{Deserialize, Deserializer};
use std::io::ErrorKind;
use std::num::NonZeroU32;
use std::os::unix::fs::MetadataExt;
use std::path::PathBuf;
use strum::VariantNames;
@ -128,6 +129,7 @@ pub(crate) struct TdpLimitConfig {
#[serde(deserialize_with = "de_tdp_limiter_method")]
pub method: TdpLimitingMethod,
pub range: Option<RangeConfig<u32>>,
pub download_mode_limit: Option<NonZeroU32>,
}
#[derive(Clone, Default, Deserialize, Debug)]

View file

@ -10,16 +10,27 @@ use async_trait::async_trait;
use lazy_static::lazy_static;
use num_enum::TryFromPrimitive;
use regex::Regex;
use std::collections::hash_map::Entry;
use std::collections::HashMap;
use std::num::NonZeroU32;
use std::ops::RangeInclusive;
use std::os::fd::OwnedFd;
use std::path::{Path, PathBuf};
use std::str::FromStr;
use strum::{Display, EnumString, VariantNames};
use tokio::fs::{self, try_exists, File};
use tokio::io::{AsyncBufReadExt, AsyncWriteExt, BufReader};
use tracing::{error, warn};
use tokio::io::{AsyncBufReadExt, AsyncWriteExt, BufReader, Interest};
use tokio::net::unix::pipe;
use tokio::sync::mpsc::UnboundedReceiver;
use tokio::sync::oneshot;
use tokio::task::JoinSet;
use tracing::{debug, error, warn};
use zbus::Connection;
use crate::hardware::{device_type, DeviceType};
use crate::manager::root::RootManagerProxy;
use crate::platform::platform_config;
use crate::Service;
use crate::{path, write_synced};
const HWMON_PREFIX: &str = "/sys/class/hwmon";
@ -125,6 +136,26 @@ pub(crate) async fn tdp_limit_manager() -> Result<Box<dyn TdpLimitManager>> {
})
}
pub(crate) struct TdpManagerService {
proxy: RootManagerProxy<'static>,
channel: UnboundedReceiver<TdpManagerCommand>,
download_set: JoinSet<String>,
download_handles: HashMap<String, u32>,
download_mode_limit: Option<NonZeroU32>,
previous_limit: Option<NonZeroU32>,
manager: Box<dyn TdpLimitManager>,
}
pub(crate) enum TdpManagerCommand {
SetTdpLimit(u32),
GetTdpLimit(oneshot::Sender<Result<u32>>),
GetTdpLimitRange(oneshot::Sender<Result<RangeInclusive<u32>>>),
IsActive(oneshot::Sender<Result<bool>>),
UpdateDownloadMode,
EnterDownloadMode(String, oneshot::Sender<Result<Option<OwnedFd>>>),
ListDownloadModeHandles(oneshot::Sender<HashMap<String, u32>>),
}
async fn read_gpu_sysfs_contents<S: AsRef<Path>>(suffix: S) -> Result<String> {
// Read a given suffix for the GPU
let base = find_hwmon(GPU_HWMON_NAME).await?;
@ -606,15 +637,209 @@ pub(crate) async fn set_platform_profile(name: &str, profile: &str) -> Result<()
.map_err(|message| anyhow!("Error writing to sysfs: {message}"))
}
impl TdpManagerService {
pub async fn new(
channel: UnboundedReceiver<TdpManagerCommand>,
connection: &Connection,
) -> Result<TdpManagerService> {
let config = platform_config().await?;
let config = config
.as_ref()
.and_then(|config| config.tdp_limit.as_ref())
.ok_or(anyhow!("No TDP limit configured"))?;
let manager = tdp_limit_manager().await?;
let proxy = RootManagerProxy::new(connection).await?;
Ok(TdpManagerService {
proxy,
channel,
download_set: JoinSet::new(),
download_handles: HashMap::new(),
previous_limit: None,
download_mode_limit: config.download_mode_limit,
manager,
})
}
async fn update_download_mode(&mut self) -> Result<()> {
if !self.manager.is_active().await? {
return Ok(());
}
let Some(download_mode_limit) = self.download_mode_limit else {
return Ok(());
};
let Some(current_limit) = NonZeroU32::new(self.manager.get_tdp_limit().await?) else {
// If current_limit is 0 then the interface is broken, likely because TDP limiting
// isn't possible with the current power profile or system, so we should just ignore
// it for now.
return Ok(());
};
if self.download_handles.is_empty() {
if let Some(previous_limit) = self.previous_limit {
debug!("Leaving download mode, setting TDP to {previous_limit}");
self.set_tdp_limit(previous_limit.get()).await?;
self.previous_limit = None;
}
} else {
if self.previous_limit.is_none() {
debug!("Entering download mode, caching TDP limit of {current_limit}");
self.previous_limit = Some(current_limit);
}
if current_limit != download_mode_limit {
self.set_tdp_limit(download_mode_limit.get()).await?;
}
}
Ok(())
}
async fn get_download_mode_handle(
&mut self,
identifier: impl AsRef<str>,
) -> Result<Option<OwnedFd>> {
if self.download_mode_limit.is_none() {
return Ok(None);
}
let (send, recv) = pipe::pipe()?;
let identifier = identifier.as_ref().to_string();
self.download_handles
.entry(identifier.clone())
.and_modify(|count| *count += 1)
.or_insert(1);
self.download_set
.spawn(TdpManagerService::wait_on_handle(recv, identifier));
self.update_download_mode().await?;
Ok(Some(send.into_blocking_fd()?))
}
async fn wait_on_handle(recv: pipe::Receiver, identifier: String) -> String {
loop {
let mut buf = [0; 1024];
let read = match recv.ready(Interest::READABLE).await {
Ok(r) if r.is_read_closed() => break,
Ok(r) if r.is_readable() => recv.try_read(&mut buf),
Err(e) => Err(e),
Ok(e) => {
warn!("Download mode handle received unexpected event: {e:?}");
break;
}
};
if let Err(e) = read {
warn!("Download mode handle received unexpected error: {e:?}");
break;
}
}
identifier
}
async fn set_tdp_limit(&self, limit: u32) -> Result<()> {
Ok(self
.proxy
.set_tdp_limit(limit)
.await
.inspect_err(|e| error!("Failed to set TDP limit: {e}"))?)
}
async fn handle_command(&mut self, command: TdpManagerCommand) -> Result<()> {
match command {
TdpManagerCommand::SetTdpLimit(limit) => {
if self.download_handles.is_empty() {
self.set_tdp_limit(limit).await?;
}
}
TdpManagerCommand::GetTdpLimit(reply) => {
let _ = reply.send(self.manager.get_tdp_limit().await);
}
TdpManagerCommand::GetTdpLimitRange(reply) => {
let _ = reply.send(self.manager.get_tdp_limit_range().await);
}
TdpManagerCommand::IsActive(reply) => {
let _ = reply.send(self.manager.is_active().await);
}
TdpManagerCommand::UpdateDownloadMode => {
self.update_download_mode().await?;
}
TdpManagerCommand::EnterDownloadMode(identifier, reply) => {
let fd = self.get_download_mode_handle(identifier).await;
let _ = reply.send(fd);
}
TdpManagerCommand::ListDownloadModeHandles(reply) => {
let _ = reply.send(self.download_handles.clone());
}
}
Ok(())
}
}
impl Service for TdpManagerService {
const NAME: &'static str = "tdp-manager";
async fn run(&mut self) -> Result<()> {
loop {
if self.download_set.is_empty() {
let message = match self.channel.recv().await {
None => bail!("TDP manager service channel broke"),
Some(message) => message,
};
let _ = self
.handle_command(message)
.await
.inspect_err(|e| error!("Failed to handle command: {e}"));
} else {
tokio::select! {
message = self.channel.recv() => {
let message = match message {
None => bail!("TDP manager service channel broke"),
Some(message) => message,
};
let _ = self.handle_command(message)
.await
.inspect_err(|e| error!("Failed to handle command: {e}"));
},
identifier = self.download_set.join_next() => {
match identifier {
None => (),
Some(Ok(identifier)) => {
match self.download_handles.entry(identifier) {
Entry::Occupied(e) if e.get() == &1 => {
e.remove();
if self.download_handles.is_empty() {
if let Err(e) = self.update_download_mode().await {
error!("Failed to update download mode: {e}");
}
}
},
Entry::Occupied(mut e) => *e.get_mut() -= 1,
Entry::Vacant(_) => (),
}
}
Some(Err(e)) => warn!("Failed to get closed download mode handle: {e}"),
}
},
}
}
}
}
}
#[cfg(test)]
pub(crate) mod test {
use super::*;
use crate::error::to_zbus_fdo_error;
use crate::hardware::test::fake_model;
use crate::hardware::SteamDeckVariant;
use crate::platform::{BatteryChargeLimitConfig, PlatformConfig, RangeConfig, TdpLimitConfig};
use crate::{enum_roundtrip, testing};
use anyhow::anyhow;
use std::time::Duration;
use tokio::fs::{create_dir_all, read_to_string, remove_dir, write};
use tokio::sync::mpsc::{channel, unbounded_channel, Sender};
use tokio::time::sleep;
use zbus::{fdo, interface};
pub async fn setup() -> Result<()> {
// Use hwmon5 just as a test. We needed a subfolder of HWMON_PREFIX
@ -792,6 +1017,7 @@ CCLK_RANGE in Core0:
platform_config.tdp_limit = Some(TdpLimitConfig {
method: TdpLimitingMethod::GpuHwmon,
range: Some(RangeConfig { min: 3, max: 15 }),
download_mode_limit: None,
});
handle.test.platform_config.replace(Some(platform_config));
let manager = tdp_limit_manager().await.unwrap();
@ -815,6 +1041,7 @@ CCLK_RANGE in Core0:
platform_config.tdp_limit = Some(TdpLimitConfig {
method: TdpLimitingMethod::GpuHwmon,
range: Some(RangeConfig { min: 3, max: 15 }),
download_mode_limit: None,
});
handle.test.platform_config.replace(Some(platform_config));
let manager = tdp_limit_manager().await.unwrap();
@ -1360,4 +1587,187 @@ CCLK_RANGE in Core0:
&["a", "b", "c"]
);
}
struct MockTdpLimit {
queue: Sender<()>,
}
#[interface(name = "com.steampowered.SteamOSManager1.RootManager")]
impl MockTdpLimit {
async fn set_tdp_limit(&mut self, limit: u32) -> fdo::Result<()> {
let hwmon = path(HWMON_PREFIX);
write(
hwmon.join("hwmon5").join(TDP_LIMIT1),
format!("{limit}000000\n"),
)
.await
.expect("write");
self.queue.send(()).await.map_err(to_zbus_fdo_error)?;
Ok(())
}
}
#[tokio::test]
async fn test_low_power_lock() {
let mut h = testing::start();
setup().await.expect("setup");
let connection = h.new_dbus().await.expect("new_dbus");
let (tx, rx) = unbounded_channel();
let (fin_tx, fin_rx) = oneshot::channel();
let (start_tx, start_rx) = oneshot::channel();
let (reply_tx, mut reply_rx) = channel(1);
let iface = MockTdpLimit { queue: reply_tx };
let mut platform_config = PlatformConfig::default();
platform_config.tdp_limit = Some(TdpLimitConfig {
method: TdpLimitingMethod::GpuHwmon,
range: Some(RangeConfig { min: 3, max: 15 }),
download_mode_limit: NonZeroU32::new(6),
});
h.test.platform_config.replace(Some(platform_config));
let manager = tdp_limit_manager().await.unwrap();
connection
.request_name("com.steampowered.SteamOSManager1")
.await
.expect("reserve_name");
let object_server = connection.object_server();
object_server
.at("/com/steampowered/SteamOSManager1", iface)
.await
.expect("at");
let mut service = TdpManagerService::new(rx, &connection)
.await
.expect("service");
let task = tokio::spawn(async move {
start_tx.send(()).unwrap();
tokio::select! {
r = service.run() => r,
_ = fin_rx => Ok(()),
}
});
start_rx.await.expect("start_rx");
sleep(Duration::from_millis(1)).await;
tx.send(TdpManagerCommand::SetTdpLimit(15)).unwrap();
reply_rx.recv().await;
assert_eq!(manager.get_tdp_limit().await.unwrap(), 15);
let (os_tx, os_rx) = oneshot::channel();
tx.send(TdpManagerCommand::ListDownloadModeHandles(os_tx))
.unwrap();
assert!(os_rx.await.unwrap().is_empty());
let (h_tx, h_rx) = oneshot::channel();
tx.send(TdpManagerCommand::EnterDownloadMode(
String::from("test"),
h_tx,
))
.unwrap();
{
let _h = h_rx.await.unwrap().expect("result").expect("handle");
reply_rx.recv().await;
assert_eq!(manager.get_tdp_limit().await.unwrap(), 6);
let (os_tx, os_rx) = oneshot::channel();
tx.send(TdpManagerCommand::ListDownloadModeHandles(os_tx))
.unwrap();
assert_eq!(os_rx.await.unwrap(), [(String::from("test"), 1u32)].into());
tx.send(TdpManagerCommand::SetTdpLimit(15)).unwrap();
assert!(tokio::select! {
_ = reply_rx.recv() => false,
_ = sleep(Duration::from_millis(2)) => true,
});
assert_eq!(manager.get_tdp_limit().await.unwrap(), 6);
}
reply_rx.recv().await;
assert_eq!(manager.get_tdp_limit().await.unwrap(), 15);
tx.send(TdpManagerCommand::SetTdpLimit(12)).unwrap();
reply_rx.recv().await;
assert_eq!(manager.get_tdp_limit().await.unwrap(), 12);
let (os_tx, os_rx) = oneshot::channel();
tx.send(TdpManagerCommand::ListDownloadModeHandles(os_tx))
.unwrap();
assert!(os_rx.await.unwrap().is_empty());
fin_tx.send(()).expect("fin");
task.await.expect("exit").expect("exit2");
}
#[tokio::test]
async fn test_disabled_low_power_lock() {
let mut h = testing::start();
setup().await.expect("setup");
let connection = h.new_dbus().await.expect("new_dbus");
let (tx, rx) = unbounded_channel();
let (fin_tx, fin_rx) = oneshot::channel();
let (start_tx, start_rx) = oneshot::channel();
let (reply_tx, mut reply_rx) = channel(1);
let iface = MockTdpLimit { queue: reply_tx };
let mut platform_config = PlatformConfig::default();
platform_config.tdp_limit = Some(TdpLimitConfig {
method: TdpLimitingMethod::GpuHwmon,
range: Some(RangeConfig { min: 3, max: 15 }),
download_mode_limit: None,
});
h.test.platform_config.replace(Some(platform_config));
let manager = tdp_limit_manager().await.unwrap();
connection
.request_name("com.steampowered.SteamOSManager1")
.await
.expect("reserve_name");
let object_server = connection.object_server();
object_server
.at("/com/steampowered/SteamOSManager1", iface)
.await
.expect("at");
let mut service = TdpManagerService::new(rx, &connection)
.await
.expect("service");
let task = tokio::spawn(async move {
start_tx.send(()).unwrap();
tokio::select! {
r = service.run() => r,
_ = fin_rx => Ok(()),
}
});
start_rx.await.expect("start_rx");
sleep(Duration::from_millis(1)).await;
tx.send(TdpManagerCommand::SetTdpLimit(15)).unwrap();
reply_rx.recv().await;
assert_eq!(manager.get_tdp_limit().await.unwrap(), 15);
let (os_tx, os_rx) = oneshot::channel();
tx.send(TdpManagerCommand::ListDownloadModeHandles(os_tx))
.unwrap();
assert!(os_rx.await.unwrap().is_empty());
let (h_tx, h_rx) = oneshot::channel();
tx.send(TdpManagerCommand::EnterDownloadMode(
String::from("test"),
h_tx,
))
.unwrap();
let h = h_rx.await.unwrap().expect("result");
assert!(h.is_none());
fin_tx.send(()).expect("fin");
task.await.expect("exit").expect("exit2");
}
}

View file

@ -0,0 +1,27 @@
//! # D-Bus interface proxy for: `com.steampowered.SteamOSManager1.LowPowerMode1`
//!
//! This code was generated by `zbus-xmlgen` `4.1.0` from D-Bus introspection data.
//! Source: `com.steampowered.SteamOSManager1.xml`.
//!
//! You may prefer to adapt it, instead of using it verbatim.
//!
//! More information can be found in the [Writing a client proxy] section of the zbus
//! documentation.
//!
//!
//! [Writing a client proxy]: https://dbus2.github.io/zbus/client.html
//! [D-Bus standard interfaces]: https://dbus.freedesktop.org/doc/dbus-specification.html#standard-interfaces,
use zbus::proxy;
#[proxy(
interface = "com.steampowered.SteamOSManager1.LowPowerMode1",
default_service = "com.steampowered.SteamOSManager1",
default_path = "/com/steampowered/SteamOSManager1",
assume_defaults = true
)]
pub trait LowPowerMode1 {
/// EnterDownloadMode method
fn enter_download_mode(&self, identifier: &str) -> zbus::Result<zbus::zvariant::OwnedFd>;
/// ListDownloadModeHandles method
fn list_download_mode_handles(&self) -> zbus::Result<std::collections::HashMap<String, u32>>;
}

View file

@ -22,6 +22,7 @@ mod fan_control1;
mod gpu_performance_level1;
mod gpu_power_profile1;
mod hdmi_cec1;
mod low_power_mode1;
mod manager2;
mod performance_profile1;
mod storage1;
@ -39,6 +40,7 @@ pub use crate::proxy::fan_control1::FanControl1Proxy;
pub use crate::proxy::gpu_performance_level1::GpuPerformanceLevel1Proxy;
pub use crate::proxy::gpu_power_profile1::GpuPowerProfile1Proxy;
pub use crate::proxy::hdmi_cec1::HdmiCec1Proxy;
pub use crate::proxy::low_power_mode1::LowPowerMode1Proxy;
pub use crate::proxy::manager2::Manager2Proxy;
pub use crate::proxy::performance_profile1::PerformanceProfile1Proxy;
pub use crate::proxy::storage1::Storage1Proxy;