Auto merge of #12706 - ehuss:cache-lock-mode, r=epage

Add new package cache lock modes

The way locking worked before this PR is that only one cargo could write to the package cache at once (otherwise it could cause corruption). However, it allowed cargo's to read from the package cache while running a build under the assumption that writers are append-only and won't affect reading. This allows multiple builds to run concurrently, only blocking on the part where it is not possible to run concurrently (downloading to the cache).

This introduces a new package cache locking strategy to support the ability to safely modify existing cache entries while other cargos are potentially reading from the cache. It has different locking modes:

- `MutateExclusive` (new) — Held when cargo wants to modify existing cache entries (such as being introduced for garbage collection in #12634), and ensures only one cargo has access to the cache during that time.
- `DownloadExclusive`  (renamed) — This is a more specialized name for the lock that was before this PR.   A caller should acquire this when downloading into the cache and doing resolution.  It ensures that only one cargo can append to the cache, but allows other cargos to concurrently read from the cache.
- `Shared` (new) — This is to preserve the old concurrent build behavior by allowing multiple concurrent cargos to hold this while a build is running when it is reading from the cache

**Reviewing suggestions:**
There are a few commits needed to help with testing which are first. The main commit has the following:
- `src/cargo/util/cache_lock.rs` is an abstraction around package cache locks, and is the heart of the change. It should have comments and notes which should guide what it is doing. The `CacheLocker` is stored in `Config` along with all our other global stuff.
- Every call to `config.acquire_package_cache_lock()` has been changed to explicitly state which lock mode it wants to lock the package cache in.
- `Context::compile` is the key point where the `Shared` lock is acquired, ensuring that no mutation is done while the cache is being read.
- `MutateExclusive` is not used in this PR, but is being added in preparation for #12634.
- The non-blocking `try_acquire_package_cache_lock` API is not used in this PR, but is being added in preparation for #12634 to allow automatic gc to skip running if another cargo is already running (to avoid unnecessary blocking).
- `src/cargo/util/flock.rs` has been updated with some code cleanup (removing unused stuff), adds support for non-blocking locks, and renames some functions to make their operation clearer.
- `tests/testsuite/cache_lock.rs` contains tests for all the different permutations of ways of acquiring locks.
This commit is contained in:
bors 2023-10-09 15:56:42 +00:00
commit b48c41aedb
29 changed files with 1239 additions and 228 deletions

View File

@ -236,6 +236,7 @@ fn substitute_macros(input: &str) -> String {
("[SKIPPING]", " Skipping"),
("[WAITING]", " Waiting"),
("[PUBLISHED]", " Published"),
("[BLOCKING]", " Blocking"),
];
let mut result = input.to_owned();
for &(pat, subst) in &macros {

View File

@ -13,6 +13,7 @@ use std::path::{Path, PathBuf};
use std::process::{Command, Output};
use std::str;
use std::sync::OnceLock;
use std::thread::JoinHandle;
use std::time::{self, Duration};
use anyhow::{bail, Result};
@ -1470,3 +1471,50 @@ pub fn symlink_supported() -> bool {
pub fn no_such_file_err_msg() -> String {
std::io::Error::from_raw_os_error(2).to_string()
}
/// Helper to retry a function `n` times.
///
/// The function should return `Some` when it is ready.
pub fn retry<F, R>(n: u32, mut f: F) -> R
where
F: FnMut() -> Option<R>,
{
let mut count = 0;
let start = std::time::Instant::now();
loop {
if let Some(r) = f() {
return r;
}
count += 1;
if count > n {
panic!(
"test did not finish within {n} attempts ({:?} total)",
start.elapsed()
);
}
sleep_ms(100);
}
}
#[test]
#[should_panic(expected = "test did not finish")]
fn retry_fails() {
retry(2, || None::<()>);
}
/// Helper that waits for a thread to finish, up to `n` tenths of a second.
pub fn thread_wait_timeout<T>(n: u32, thread: JoinHandle<T>) -> T {
retry(n, || thread.is_finished().then_some(()));
thread.join().unwrap()
}
/// Helper that runs some function, and waits up to `n` tenths of a second for
/// it to finish.
pub fn threaded_timeout<F, R>(n: u32, f: F) -> R
where
F: FnOnce() -> R + Send + 'static,
R: Send + 'static,
{
let thread = std::thread::spawn(|| f());
thread_wait_timeout(n, thread)
}

View File

@ -22,6 +22,7 @@ use cargo::core::Registry;
use cargo::core::SourceId;
use cargo::core::Workspace;
use cargo::sources::source::QueryKind;
use cargo::util::cache_lock::CacheLockMode;
use cargo::util::command_prelude::*;
use cargo::util::ToSemver;
use cargo::CargoResult;
@ -347,7 +348,7 @@ fn check_crates_io<'a>(
) -> CargoResult<()> {
let source_id = SourceId::crates_io(config)?;
let mut registry = PackageRegistry::new(config)?;
let _lock = config.acquire_package_cache_lock()?;
let _lock = config.acquire_package_cache_lock(CacheLockMode::DownloadExclusive)?;
registry.lock_patches();
config.shell().status(
STATUS,

View File

@ -7,6 +7,7 @@ use std::sync::{Arc, Mutex};
use crate::core::compiler::compilation::{self, UnitOutput};
use crate::core::compiler::{self, artifact, Unit};
use crate::core::PackageId;
use crate::util::cache_lock::CacheLockMode;
use crate::util::errors::CargoResult;
use crate::util::profile;
use anyhow::{bail, Context as _};
@ -132,6 +133,13 @@ impl<'a, 'cfg> Context<'a, 'cfg> {
///
/// [`ops::cargo_compile`]: ../../../ops/cargo_compile/index.html
pub fn compile(mut self, exec: &Arc<dyn Executor>) -> CargoResult<Compilation<'cfg>> {
// A shared lock is held during the duration of the build since rustc
// needs to read from the `src` cache, and we don't want other
// commands modifying the `src` cache while it is running.
let _lock = self
.bcx
.config
.acquire_package_cache_lock(CacheLockMode::Shared)?;
let mut queue = JobQueue::new(self.bcx);
let mut plan = BuildPlan::new();
let build_plan = self.bcx.build_config.build_plan;

View File

@ -37,6 +37,7 @@ use crate::core::compiler::BuildContext;
use crate::core::{Dependency, PackageId, Workspace};
use crate::sources::source::QueryKind;
use crate::sources::SourceConfigMap;
use crate::util::cache_lock::CacheLockMode;
use crate::util::{iter_join, CargoResult};
use anyhow::{bail, format_err, Context};
use serde::{Deserialize, Serialize};
@ -166,7 +167,7 @@ impl OnDiskReports {
let on_disk = serde_json::to_vec(&self).unwrap();
if let Err(e) = ws
.target_dir()
.open_rw(
.open_rw_exclusive_create(
FUTURE_INCOMPAT_FILE,
ws.config(),
"Future incompatibility report",
@ -190,7 +191,7 @@ impl OnDiskReports {
/// Loads the on-disk reports.
pub fn load(ws: &Workspace<'_>) -> CargoResult<OnDiskReports> {
let report_file = match ws.target_dir().open_ro(
let report_file = match ws.target_dir().open_ro_shared(
FUTURE_INCOMPAT_FILE,
ws.config(),
"Future incompatible report",
@ -297,7 +298,10 @@ fn render_report(per_package_reports: &[FutureIncompatReportPackage]) -> BTreeMa
/// This is best-effort - if an error occurs, `None` will be returned.
fn get_updates(ws: &Workspace<'_>, package_ids: &BTreeSet<PackageId>) -> Option<String> {
// This in general ignores all errors since this is opportunistic.
let _lock = ws.config().acquire_package_cache_lock().ok()?;
let _lock = ws
.config()
.acquire_package_cache_lock(CacheLockMode::DownloadExclusive)
.ok()?;
// Create a set of updated registry sources.
let map = SourceConfigMap::new(ws.config()).ok()?;
let mut package_ids: BTreeSet<_> = package_ids

View File

@ -166,7 +166,7 @@ impl Layout {
// For now we don't do any more finer-grained locking on the artifact
// directory, so just lock the entire thing for the duration of this
// compile.
let lock = dest.open_rw(".cargo-lock", ws.config(), "build directory")?;
let lock = dest.open_rw_exclusive_create(".cargo-lock", ws.config(), "build directory")?;
let root = root.into_path_unlocked();
let dest = dest.into_path_unlocked();
let deps = dest.join("deps");

View File

@ -24,7 +24,7 @@ use crate::core::resolver::{HasDevUnits, Resolve};
use crate::core::{Dependency, Manifest, PackageId, SourceId, Target};
use crate::core::{Summary, Workspace};
use crate::sources::source::{MaybePackage, SourceMap};
use crate::util::config::PackageCacheLock;
use crate::util::cache_lock::{CacheLock, CacheLockMode};
use crate::util::errors::{CargoResult, HttpNotSuccessful};
use crate::util::interning::InternedString;
use crate::util::network::http::http_handle_and_timeout;
@ -367,7 +367,7 @@ pub struct Downloads<'a, 'cfg> {
next_speed_check_bytes_threshold: Cell<u64>,
/// Global filesystem lock to ensure only one Cargo is downloading at a
/// time.
_lock: PackageCacheLock<'cfg>,
_lock: CacheLock<'cfg>,
}
struct Download<'cfg> {
@ -465,7 +465,9 @@ impl<'cfg> PackageSet<'cfg> {
timeout,
next_speed_check: Cell::new(Instant::now()),
next_speed_check_bytes_threshold: Cell::new(0),
_lock: self.config.acquire_package_cache_lock()?,
_lock: self
.config
.acquire_package_cache_lock(CacheLockMode::DownloadExclusive)?,
})
}
@ -478,6 +480,9 @@ impl<'cfg> PackageSet<'cfg> {
pub fn get_many(&self, ids: impl IntoIterator<Item = PackageId>) -> CargoResult<Vec<&Package>> {
let mut pkgs = Vec::new();
let _lock = self
.config
.acquire_package_cache_lock(CacheLockMode::DownloadExclusive)?;
let mut downloads = self.enable_download()?;
for id in ids {
pkgs.extend(downloads.start(id)?);

View File

@ -23,6 +23,7 @@ use crate::core::Shell;
use crate::core::Summary;
use crate::core::Workspace;
use crate::sources::source::QueryKind;
use crate::util::cache_lock::CacheLockMode;
use crate::util::style;
use crate::util::toml_mut::dependency::Dependency;
use crate::util::toml_mut::dependency::GitSource;
@ -77,7 +78,9 @@ pub fn add(workspace: &Workspace<'_>, options: &AddOptions<'_>) -> CargoResult<(
let mut registry = PackageRegistry::new(options.config)?;
let deps = {
let _lock = options.config.acquire_package_cache_lock()?;
let _lock = options
.config
.acquire_package_cache_lock(CacheLockMode::DownloadExclusive)?;
registry.lock_patches();
options
.dependencies

View File

@ -3,6 +3,7 @@ use crate::core::resolver::features::{CliFeatures, HasDevUnits};
use crate::core::{PackageId, PackageIdSpec};
use crate::core::{Resolve, SourceId, Workspace};
use crate::ops;
use crate::util::cache_lock::CacheLockMode;
use crate::util::config::Config;
use crate::util::style;
use crate::util::CargoResult;
@ -48,7 +49,9 @@ pub fn update_lockfile(ws: &Workspace<'_>, opts: &UpdateOptions<'_>) -> CargoRes
// Updates often require a lot of modifications to the registry, so ensure
// that we're synchronized against other Cargos.
let _lock = ws.config().acquire_package_cache_lock()?;
let _lock = ws
.config()
.acquire_package_cache_lock(CacheLockMode::DownloadExclusive)?;
let max_rust_version = ws.rust_version();

View File

@ -13,6 +13,7 @@ use crate::core::{registry::PackageRegistry, resolver::HasDevUnits};
use crate::core::{Feature, Shell, Verbosity, Workspace};
use crate::core::{Package, PackageId, PackageSet, Resolve, SourceId};
use crate::sources::PathSource;
use crate::util::cache_lock::CacheLockMode;
use crate::util::config::JobsConfig;
use crate::util::errors::CargoResult;
use crate::util::toml::TomlManifest;
@ -132,7 +133,7 @@ pub fn package_one(
let dir = ws.target_dir().join("package");
let mut dst = {
let tmp = format!(".{}", filename);
dir.open_rw(&tmp, config, "package scratch space")?
dir.open_rw_exclusive_create(&tmp, config, "package scratch space")?
};
// Package up and test a temporary tarball and only move it to the final
@ -806,7 +807,7 @@ pub fn check_yanked(
) -> CargoResult<()> {
// Checking the yanked status involves taking a look at the registry and
// maybe updating files, so be sure to lock it here.
let _lock = config.acquire_package_cache_lock()?;
let _lock = config.acquire_package_cache_lock(CacheLockMode::DownloadExclusive)?;
let mut sources = pkg_set.sources_mut();
let mut pending: Vec<PackageId> = resolve.iter().collect();

View File

@ -17,6 +17,7 @@ use crate::ops::{self, CompileFilter, CompileOptions};
use crate::sources::source::QueryKind;
use crate::sources::source::Source;
use crate::sources::PathSource;
use crate::util::cache_lock::CacheLockMode;
use crate::util::errors::CargoResult;
use crate::util::Config;
use crate::util::{FileLock, Filesystem};
@ -97,8 +98,10 @@ pub struct CrateListingV1 {
impl InstallTracker {
/// Create an InstallTracker from information on disk.
pub fn load(config: &Config, root: &Filesystem) -> CargoResult<InstallTracker> {
let v1_lock = root.open_rw(Path::new(".crates.toml"), config, "crate metadata")?;
let v2_lock = root.open_rw(Path::new(".crates2.json"), config, "crate metadata")?;
let v1_lock =
root.open_rw_exclusive_create(Path::new(".crates.toml"), config, "crate metadata")?;
let v2_lock =
root.open_rw_exclusive_create(Path::new(".crates2.json"), config, "crate metadata")?;
let v1 = (|| -> CargoResult<_> {
let mut contents = String::new();
@ -536,7 +539,7 @@ where
// This operation may involve updating some sources or making a few queries
// which may involve frobbing caches, as a result make sure we synchronize
// with other global Cargos
let _lock = config.acquire_package_cache_lock()?;
let _lock = config.acquire_package_cache_lock(CacheLockMode::DownloadExclusive)?;
if needs_update {
source.invalidate_cache();
@ -604,7 +607,7 @@ where
// This operation may involve updating some sources or making a few queries
// which may involve frobbing caches, as a result make sure we synchronize
// with other global Cargos
let _lock = config.acquire_package_cache_lock()?;
let _lock = config.acquire_package_cache_lock(CacheLockMode::DownloadExclusive)?;
source.invalidate_cache();

View File

@ -12,7 +12,7 @@ pub fn load_pkg_lockfile(ws: &Workspace<'_>) -> CargoResult<Option<Resolve>> {
return Ok(None);
}
let mut f = lock_root.open_ro("Cargo.lock", ws.config(), "Cargo.lock file")?;
let mut f = lock_root.open_ro_shared("Cargo.lock", ws.config(), "Cargo.lock file")?;
let mut s = String::new();
f.read_to_string(&mut s)
@ -79,7 +79,7 @@ pub fn write_pkg_lockfile(ws: &Workspace<'_>, resolve: &mut Resolve) -> CargoRes
// Ok, if that didn't work just write it out
lock_root
.open_rw("Cargo.lock", ws.config(), "Cargo.lock file")
.open_rw_exclusive_create("Cargo.lock", ws.config(), "Cargo.lock file")
.and_then(|mut f| {
f.file().set_len(0)?;
f.write_all(out.as_bytes())?;
@ -100,7 +100,7 @@ fn resolve_to_string_orig(
) -> (Option<String>, String, Filesystem) {
// Load the original lock file if it exists.
let lock_root = lock_root(ws);
let orig = lock_root.open_ro("Cargo.lock", ws.config(), "Cargo.lock file");
let orig = lock_root.open_ro_shared("Cargo.lock", ws.config(), "Cargo.lock file");
let orig = orig.and_then(|mut f| {
let mut s = String::new();
f.read_to_string(&mut s)?;

View File

@ -22,6 +22,7 @@ use crate::core::SourceId;
use crate::sources::source::Source;
use crate::sources::{RegistrySource, SourceConfigMap};
use crate::util::auth;
use crate::util::cache_lock::CacheLockMode;
use crate::util::config::{Config, PathAndArgs};
use crate::util::errors::CargoResult;
use crate::util::network::http::http_handle;
@ -131,7 +132,7 @@ fn registry(
}
let cfg = {
let _lock = config.acquire_package_cache_lock()?;
let _lock = config.acquire_package_cache_lock(CacheLockMode::DownloadExclusive)?;
let mut src = RegistrySource::remote(source_ids.replacement, &HashSet::new(), config)?;
// Only update the index if `force_update` is set.
if force_update {

View File

@ -30,6 +30,7 @@ use crate::sources::source::QueryKind;
use crate::sources::SourceConfigMap;
use crate::sources::CRATES_IO_REGISTRY;
use crate::util::auth;
use crate::util::cache_lock::CacheLockMode;
use crate::util::config::JobsConfig;
use crate::util::Progress;
use crate::util::ProgressStyle;
@ -233,7 +234,7 @@ fn wait_for_publish(
progress.tick_now(0, max, "")?;
let is_available = loop {
{
let _lock = config.acquire_package_cache_lock()?;
let _lock = config.acquire_package_cache_lock(CacheLockMode::DownloadExclusive)?;
// Force re-fetching the source
//
// As pulling from a git source is expensive, we track when we've done it within the

View File

@ -68,6 +68,7 @@ use crate::core::Feature;
use crate::core::{GitReference, PackageId, PackageIdSpec, PackageSet, SourceId, Workspace};
use crate::ops;
use crate::sources::PathSource;
use crate::util::cache_lock::CacheLockMode;
use crate::util::errors::CargoResult;
use crate::util::RustVersion;
use crate::util::{profile, CanonicalUrl};
@ -289,7 +290,9 @@ pub fn resolve_with_previous<'cfg>(
) -> CargoResult<Resolve> {
// We only want one Cargo at a time resolving a crate graph since this can
// involve a lot of frobbing of the global caches.
let _lock = ws.config().acquire_package_cache_lock()?;
let _lock = ws
.config()
.acquire_package_cache_lock(CacheLockMode::DownloadExclusive)?;
// Here we place an artificial limitation that all non-registry sources
// cannot be locked at more than one revision. This means that if a Git

View File

@ -8,6 +8,7 @@ use crate::sources::source::MaybePackage;
use crate::sources::source::QueryKind;
use crate::sources::source::Source;
use crate::sources::PathSource;
use crate::util::cache_lock::CacheLockMode;
use crate::util::errors::CargoResult;
use crate::util::hex::short_hash;
use crate::util::Config;
@ -212,7 +213,9 @@ impl<'cfg> Source for GitSource<'cfg> {
// Ignore errors creating it, in case this is a read-only filesystem:
// perhaps the later operations can succeed anyhow.
let _ = git_fs.create_dir();
let git_path = self.config.assert_package_cache_locked(&git_fs);
let git_path = self
.config
.assert_package_cache_locked(CacheLockMode::DownloadExclusive, &git_fs);
// Before getting a checkout, make sure that `<cargo_home>/git` is
// marked as excluded from indexing and backups. Older versions of Cargo

View File

@ -12,6 +12,7 @@ use crate::core::PackageId;
use crate::sources::registry::MaybeLock;
use crate::sources::registry::RegistryConfig;
use crate::util::auth;
use crate::util::cache_lock::CacheLockMode;
use crate::util::errors::CargoResult;
use crate::util::{Config, Filesystem};
use std::fmt::Write as FmtWrite;
@ -38,7 +39,7 @@ pub(super) fn download(
registry_config: RegistryConfig,
) -> CargoResult<MaybeLock> {
let path = cache_path.join(&pkg.tarball_name());
let path = config.assert_package_cache_locked(&path);
let path = config.assert_package_cache_locked(CacheLockMode::DownloadExclusive, &path);
// Attempt to open a read-only copy first to avoid an exclusive write
// lock and also work with read-only filesystems. Note that we check the
@ -117,7 +118,7 @@ pub(super) fn finish_download(
cache_path.create_dir()?;
let path = cache_path.join(&pkg.tarball_name());
let path = config.assert_package_cache_locked(&path);
let path = config.assert_package_cache_locked(CacheLockMode::DownloadExclusive, &path);
let mut dst = OpenOptions::new()
.create(true)
.read(true)
@ -144,7 +145,7 @@ pub(super) fn is_crate_downloaded(
pkg: PackageId,
) -> bool {
let path = cache_path.join(pkg.tarball_name());
let path = config.assert_package_cache_locked(&path);
let path = config.assert_package_cache_locked(CacheLockMode::DownloadExclusive, &path);
if let Ok(meta) = fs::metadata(path) {
return meta.len() > 0;
}

View File

@ -4,6 +4,7 @@ use crate::core::{PackageId, SourceId};
use crate::sources::registry::download;
use crate::sources::registry::MaybeLock;
use crate::sources::registry::{LoadResponse, RegistryConfig, RegistryData};
use crate::util::cache_lock::CacheLockMode;
use crate::util::errors::{CargoResult, HttpNotSuccessful};
use crate::util::network::http::http_handle;
use crate::util::network::retry::{Retry, RetryResult};
@ -461,7 +462,8 @@ impl<'cfg> RegistryData for HttpRegistry<'cfg> {
}
fn assert_index_locked<'a>(&self, path: &'a Filesystem) -> &'a Path {
self.config.assert_package_cache_locked(path)
self.config
.assert_package_cache_locked(CacheLockMode::DownloadExclusive, path)
}
fn is_updated(&self) -> bool {
@ -744,6 +746,7 @@ impl<'cfg> RegistryData for HttpRegistry<'cfg> {
Poll::Ready(cfg) => break cfg.to_owned(),
}
};
download::download(
&self.cache_path,
&self.config,

View File

@ -89,6 +89,7 @@ use crate::core::dependency::{Artifact, DepKind};
use crate::core::Dependency;
use crate::core::{PackageId, SourceId, Summary};
use crate::sources::registry::{LoadResponse, RegistryData};
use crate::util::cache_lock::CacheLockMode;
use crate::util::interning::InternedString;
use crate::util::IntoUrl;
use crate::util::{internal, CargoResult, Config, Filesystem, OptVersionReq, RustVersion};
@ -823,7 +824,7 @@ impl Summaries {
// something in case of error.
if paths::create_dir_all(cache_path.parent().unwrap()).is_ok() {
let path = Filesystem::new(cache_path.clone());
config.assert_package_cache_locked(&path);
config.assert_package_cache_locked(CacheLockMode::DownloadExclusive, &path);
if let Err(e) = fs::write(cache_path, &cache_bytes) {
tracing::info!("failed to write cache: {}", e);
}

View File

@ -206,6 +206,7 @@ use crate::sources::source::MaybePackage;
use crate::sources::source::QueryKind;
use crate::sources::source::Source;
use crate::sources::PathSource;
use crate::util::cache_lock::CacheLockMode;
use crate::util::hex;
use crate::util::interning::InternedString;
use crate::util::network::PollExt;
@ -581,7 +582,9 @@ impl<'cfg> RegistrySource<'cfg> {
let package_dir = format!("{}-{}", pkg.name(), pkg.version());
let dst = self.src_path.join(&package_dir);
let path = dst.join(PACKAGE_SOURCE_LOCK);
let path = self.config.assert_package_cache_locked(&path);
let path = self
.config
.assert_package_cache_locked(CacheLockMode::DownloadExclusive, &path);
let unpack_dir = path.parent().unwrap();
match fs::read_to_string(path) {
Ok(ok) => match serde_json::from_str::<LockMetadata>(&ok) {

View File

@ -6,6 +6,7 @@ use crate::sources::git::fetch::RemoteKind;
use crate::sources::registry::download;
use crate::sources::registry::MaybeLock;
use crate::sources::registry::{LoadResponse, RegistryConfig, RegistryData};
use crate::util::cache_lock::CacheLockMode;
use crate::util::errors::CargoResult;
use crate::util::interning::InternedString;
use crate::util::{Config, Filesystem};
@ -104,7 +105,9 @@ impl<'cfg> RemoteRegistry<'cfg> {
fn repo(&self) -> CargoResult<&git2::Repository> {
self.repo.try_borrow_with(|| {
trace!("acquiring registry index lock");
let path = self.config.assert_package_cache_locked(&self.index_path);
let path = self
.config
.assert_package_cache_locked(CacheLockMode::DownloadExclusive, &self.index_path);
match git2::Repository::open(&path) {
Ok(repo) => Ok(repo),
@ -216,7 +219,8 @@ impl<'cfg> RegistryData for RemoteRegistry<'cfg> {
}
fn assert_index_locked<'a>(&self, path: &'a Filesystem) -> &'a Path {
self.config.assert_package_cache_locked(path)
self.config
.assert_package_cache_locked(CacheLockMode::DownloadExclusive, path)
}
/// Read the general concept for `load()` on [`RegistryData::load`].
@ -302,7 +306,8 @@ impl<'cfg> RegistryData for RemoteRegistry<'cfg> {
fn config(&mut self) -> Poll<CargoResult<Option<RegistryConfig>>> {
debug!("loading config");
self.prepare()?;
self.config.assert_package_cache_locked(&self.index_path);
self.config
.assert_package_cache_locked(CacheLockMode::DownloadExclusive, &self.index_path);
match ready!(self.load(Path::new(""), Path::new(RegistryConfig::NAME), None)?) {
LoadResponse::Data { raw_data, .. } => {
trace!("config loaded");
@ -346,7 +351,9 @@ impl<'cfg> RegistryData for RemoteRegistry<'cfg> {
self.head.set(None);
*self.tree.borrow_mut() = None;
self.current_sha.set(None);
let _path = self.config.assert_package_cache_locked(&self.index_path);
let _path = self
.config
.assert_package_cache_locked(CacheLockMode::DownloadExclusive, &self.index_path);
if !self.quiet {
self.config
.shell()

View File

@ -0,0 +1,549 @@
//! Support for locking the package and index caches.
//!
//! This implements locking on the package and index caches (source files,
//! `.crate` files, and index caches) to coordinate when multiple cargos are
//! running at the same time.
//!
//! ## Usage
//!
//! There is a global [`CacheLocker`] held inside cargo's venerable
//! [`Config`]. The `CacheLocker` manages creating and tracking the locks
//! being held. There are methods on `Config` for managing the locks:
//!
//! - [`Config::acquire_package_cache_lock`] --- Acquires a lock. May block if
//! another process holds a lock.
//! - [`Config::try_acquire_package_cache_lock`] --- Acquires a lock, returning
//! immediately if it would block.
//! - [`Config::assert_package_cache_locked`] --- This is used to ensure the
//! proper lock is being held.
//!
//! Lower-level code that accesses the package cache typically just use
//! `assert_package_cache_locked` to ensure that the correct lock is being
//! held. Higher-level code is responsible for acquiring the appropriate lock,
//! and holding it during the duration that it is performing its operation.
//!
//! ## Types of locking
//!
//! There are three styles of locks:
//!
//! * [`CacheLockMode::DownloadExclusive`] -- This is an exclusive lock
//! acquired while downloading packages and doing resolution.
//! * [`CacheLockMode::Shared`] -- This is a shared lock acquired while a
//! build is running. In other words, whenever cargo just needs to read from
//! the cache, it should hold this lock. This is here to ensure that no
//! cargos are trying to read the source caches when cache garbage
//! collection runs.
//! * [`CacheLockMode::MutateExclusive`] -- This is an exclusive lock acquired
//! whenever needing to modify existing source files (for example, with
//! cache garbage collection). This is acquired to make sure that no other
//! cargo is reading from the cache.
//!
//! Importantly, a `DownloadExclusive` lock does *not* interfere with a
//! `Shared` lock. The download process generally does not modify source files
//! (it only adds new ones), so other cargos should be able to safely proceed
//! in reading source files[^1].
//!
//! See the [`CacheLockMode`] enum docs for more details on when the different
//! modes should be used.
//!
//! ## Locking implementation details
//!
//! This is implemented by two separate lock files, the "download" one and the
//! "mutate" one. The `MutateExclusive` lock acquired both the "mutate" and
//! "download" locks. The `Shared` lock acquires the "mutate" lock in share
//! mode.
//!
//! An important rule is that `MutateExclusive` acquires the locks in the
//! order "mutate" first and then the "download". That helps prevent
//! deadlocks. It is not allowed for a cargo to first acquire a
//! `DownloadExclusive` lock and then a `Shared` lock because that would open
//! it up for deadlock.
//!
//! Another rule is that there should be only one [`CacheLocker`] per process
//! to uphold the ordering rules. You could in theory have multiple if you
//! could ensure that other threads would make progress and drop a lock, but
//! cargo is not architected that way.
//!
//! It is safe to recursively acquire a lock as many times as you want.
//!
//! ## Interaction with older cargos
//!
//! Before version 1.74, cargo only acquired the `DownloadExclusive` lock when
//! downloading and doing resolution. Newer cargos that acquire
//! `MutateExclusive` should still correctly block when an old cargo is
//! downloading (because it also acquires `DownloadExclusive`), but they do
//! not properly coordinate when an old cargo is in the build phase (because
//! it holds no locks). This isn't expected to be much of a problem because
//! the intended use of mutating the cache is only to delete old contents
//! which aren't currently being used. It is possible for there to be a
//! conflict, particularly if the user manually deletes the entire cache, but
//! it is not expected for this scenario to happen too often, and the only
//! consequence is that one side or the other encounters an error and needs to
//! retry.
//!
//! [^1]: A minor caveat is that downloads will delete an existing `src`
//! directory if it was extracted via an old cargo. See
//! [`crate::sources::registry::RegistrySource::unpack_package`]. This
//! should probably be fixed, but is unlikely to be a problem if the user is
//! only using versions of cargo with the same deletion logic.
use super::FileLock;
use crate::CargoResult;
use crate::Config;
use anyhow::Context;
use std::cell::RefCell;
use std::io;
/// The style of lock to acquire.
#[derive(Copy, Clone, Debug, PartialEq)]
pub enum CacheLockMode {
/// A `DownloadExclusive` lock ensures that only one cargo is doing
/// resolution and downloading new packages.
///
/// You should use this when downloading new packages or doing resolution.
///
/// If another cargo has a `MutateExclusive` lock, then an attempt to get
/// a `DownloadExclusive` lock will block.
///
/// If another cargo has a `Shared` lock, then both can operate
/// concurrently.
DownloadExclusive,
/// A `Shared` lock allows multiple cargos to read from the source files.
///
/// You should use this when cargo is reading source files from the
/// package cache. This is typically done during the build phase, since
/// cargo only needs to read files during that time. This allows multiple
/// cargo processes to build concurrently without interfering with one
/// another, while guarding against other cargos using `MutateExclusive`.
///
/// If another cargo has a `MutateExclusive` lock, then an attempt to get
/// a `Shared` will block.
///
/// If another cargo has a `DownloadExclusive` lock, then they both can
/// operate concurrently under the assumption that downloading does not
/// modify existing source files.
Shared,
/// A `MutateExclusive` lock ensures no other cargo is reading or writing
/// from the package caches.
///
/// You should use this when modifying existing files in the package
/// cache. For example, things like garbage collection want to avoid
/// deleting files while other cargos are trying to read (`Shared`) or
/// resolve or download (`DownloadExclusive`).
///
/// If another cargo has a `DownloadExclusive` or `Shared` lock, then this
/// will block until they all release their locks.
MutateExclusive,
}
/// Whether or not a lock attempt should block.
#[derive(Copy, Clone)]
enum BlockingMode {
Blocking,
NonBlocking,
}
use BlockingMode::*;
/// Whether or not a lock attempt blocked or succeeded.
#[derive(PartialEq, Copy, Clone)]
#[must_use]
enum LockingResult {
LockAcquired,
WouldBlock,
}
use LockingResult::*;
/// A file lock, with a counter to assist with recursive locking.
#[derive(Debug)]
struct RecursiveLock {
/// The file lock.
///
/// An important note is that locks can be `None` even when they are held.
/// This can happen on things like old NFS mounts where locking isn't
/// supported. We otherwise pretend we have a lock via the lock count. See
/// [`FileLock`] for more detail on that.
lock: Option<FileLock>,
/// Number locks held, to support recursive locking.
count: u32,
/// If this is `true`, it is an exclusive lock, otherwise it is shared.
is_exclusive: bool,
/// The filename of the lock.
filename: &'static str,
}
impl RecursiveLock {
fn new(filename: &'static str) -> RecursiveLock {
RecursiveLock {
lock: None,
count: 0,
is_exclusive: false,
filename,
}
}
/// Low-level lock count increment routine.
fn increment(&mut self) {
self.count = self.count.checked_add(1).unwrap();
}
/// Unlocks a previously acquired lock.
fn decrement(&mut self) {
let new_cnt = self.count.checked_sub(1).unwrap();
self.count = new_cnt;
if new_cnt == 0 {
// This will drop, releasing the lock.
self.lock = None;
}
}
/// Acquires a shared lock.
fn lock_shared(
&mut self,
config: &Config,
description: &'static str,
blocking: BlockingMode,
) -> LockingResult {
match blocking {
Blocking => {
self.lock_shared_blocking(config, description);
LockAcquired
}
NonBlocking => self.lock_shared_nonblocking(config),
}
}
/// Acquires a shared lock, blocking if held by another locker.
fn lock_shared_blocking(&mut self, config: &Config, description: &'static str) {
if self.count == 0 {
self.is_exclusive = false;
self.lock =
match config
.home()
.open_ro_shared_create(self.filename, config, description)
{
Ok(lock) => Some(lock),
Err(e) => {
// There is no error here because locking is mostly a
// best-effort attempt. If cargo home is read-only, we don't
// want to fail just because we couldn't create the lock file.
tracing::warn!("failed to acquire cache lock {}: {e:?}", self.filename);
None
}
};
}
self.increment();
}
/// Acquires a shared lock, returns [`WouldBlock`] if held by another locker.
fn lock_shared_nonblocking(&mut self, config: &Config) -> LockingResult {
if self.count == 0 {
self.is_exclusive = false;
self.lock = match config.home().try_open_ro_shared_create(self.filename) {
Ok(Some(lock)) => Some(lock),
Ok(None) => {
return WouldBlock;
}
Err(e) => {
// Pretend that the lock was acquired (see lock_shared_blocking).
tracing::warn!("failed to acquire cache lock {}: {e:?}", self.filename);
None
}
};
}
self.increment();
LockAcquired
}
/// Acquires an exclusive lock.
fn lock_exclusive(
&mut self,
config: &Config,
description: &'static str,
blocking: BlockingMode,
) -> CargoResult<LockingResult> {
if self.count > 0 && !self.is_exclusive {
// Lock upgrades are dicey. It might be possible to support
// this but would take a bit of work, and so far it isn't
// needed.
panic!("lock upgrade from shared to exclusive not supported");
}
match blocking {
Blocking => {
self.lock_exclusive_blocking(config, description)?;
Ok(LockAcquired)
}
NonBlocking => self.lock_exclusive_nonblocking(config),
}
}
/// Acquires an exclusive lock, blocking if held by another locker.
fn lock_exclusive_blocking(
&mut self,
config: &Config,
description: &'static str,
) -> CargoResult<()> {
if self.count == 0 {
self.is_exclusive = true;
match config
.home()
.open_rw_exclusive_create(self.filename, config, description)
{
Ok(lock) => self.lock = Some(lock),
Err(e) => {
if maybe_readonly(&e) {
// This is a best-effort attempt to at least try to
// acquire some sort of lock. This can help in the
// situation where this cargo only has read-only access,
// but maybe some other cargo has read-write. This will at
// least attempt to coordinate with it.
//
// We don't want to fail on a read-only mount because
// cargo grabs an exclusive lock in situations where it
// may only be reading from the package cache. In that
// case, cargo isn't writing anything, and we don't want
// to fail on that.
self.lock_shared_blocking(config, description);
// This has to pretend it is exclusive for recursive locks to work.
self.is_exclusive = true;
return Ok(());
} else {
return Err(e).with_context(|| "failed to acquire package cache lock");
}
}
}
}
self.increment();
Ok(())
}
/// Acquires an exclusive lock, returns [`WouldBlock`] if held by another locker.
fn lock_exclusive_nonblocking(&mut self, config: &Config) -> CargoResult<LockingResult> {
if self.count == 0 {
self.is_exclusive = true;
match config.home().try_open_rw_exclusive_create(self.filename) {
Ok(Some(lock)) => self.lock = Some(lock),
Ok(None) => return Ok(WouldBlock),
Err(e) => {
if maybe_readonly(&e) {
let result = self.lock_shared_nonblocking(config);
// This has to pretend it is exclusive for recursive locks to work.
self.is_exclusive = true;
return Ok(result);
} else {
return Err(e).with_context(|| "failed to acquire package cache lock");
}
}
}
}
self.increment();
Ok(LockAcquired)
}
}
/// The state of the [`CacheLocker`].
#[derive(Debug)]
struct CacheState {
/// The cache lock guards the package cache used for download and
/// resolution (append operations that should not interfere with reading
/// from existing src files).
cache_lock: RecursiveLock,
/// The mutate lock is used to either guard the entire package cache for
/// destructive modifications (in exclusive mode), or for reading the
/// package cache src files (in shared mode).
///
/// Note that [`CacheLockMode::MutateExclusive`] holds both
/// [`CacheState::mutate_lock`] and [`CacheState::cache_lock`].
mutate_lock: RecursiveLock,
}
impl CacheState {
fn lock(
&mut self,
config: &Config,
mode: CacheLockMode,
blocking: BlockingMode,
) -> CargoResult<LockingResult> {
use CacheLockMode::*;
if mode == Shared && self.cache_lock.count > 0 && self.mutate_lock.count == 0 {
// Shared lock, when a DownloadExclusive is held.
//
// This isn't supported because it could cause a deadlock. If
// one cargo is attempting to acquire a MutateExclusive lock,
// and acquires the mutate lock, but is blocked on the
// download lock, and the cargo that holds the download lock
// attempts to get a shared lock, they would end up blocking
// each other.
panic!("shared lock while holding download lock is not allowed");
}
match mode {
Shared => {
if self.mutate_lock.lock_shared(config, SHARED_DESCR, blocking) == WouldBlock {
return Ok(WouldBlock);
}
}
DownloadExclusive => {
if self
.cache_lock
.lock_exclusive(config, DOWNLOAD_EXCLUSIVE_DESCR, blocking)?
== WouldBlock
{
return Ok(WouldBlock);
}
}
MutateExclusive => {
if self
.mutate_lock
.lock_exclusive(config, MUTATE_EXCLUSIVE_DESCR, blocking)?
== WouldBlock
{
return Ok(WouldBlock);
}
// Part of the contract of MutateExclusive is that it doesn't
// allow any processes to have a lock on the package cache, so
// this acquires both locks.
match self
.cache_lock
.lock_exclusive(config, DOWNLOAD_EXCLUSIVE_DESCR, blocking)
{
Ok(LockAcquired) => {}
Ok(WouldBlock) => return Ok(WouldBlock),
Err(e) => {
self.mutate_lock.decrement();
return Err(e);
}
}
}
}
Ok(LockAcquired)
}
}
/// A held lock guard.
///
/// When this is dropped, the lock will be released.
#[must_use]
pub struct CacheLock<'lock> {
mode: CacheLockMode,
locker: &'lock CacheLocker,
}
impl Drop for CacheLock<'_> {
fn drop(&mut self) {
use CacheLockMode::*;
let mut state = self.locker.state.borrow_mut();
match self.mode {
Shared => {
state.mutate_lock.decrement();
}
DownloadExclusive => {
state.cache_lock.decrement();
}
MutateExclusive => {
state.cache_lock.decrement();
state.mutate_lock.decrement();
}
}
}
}
/// The filename for the [`CacheLockMode::DownloadExclusive`] lock.
const CACHE_LOCK_NAME: &str = ".package-cache";
/// The filename for the [`CacheLockMode::MutateExclusive`] and
/// [`CacheLockMode::Shared`] lock.
const MUTATE_NAME: &str = ".package-cache-mutate";
// Descriptions that are displayed in the "Blocking" message shown to the user.
const SHARED_DESCR: &str = "shared package cache";
const DOWNLOAD_EXCLUSIVE_DESCR: &str = "package cache";
const MUTATE_EXCLUSIVE_DESCR: &str = "package cache mutation";
/// A locker that can be used to acquire locks.
///
/// See the [`crate::util::cache_lock`] module documentation for an overview
/// of how cache locking works.
#[derive(Debug)]
pub struct CacheLocker {
/// The state of the locker.
///
/// [`CacheLocker`] uses interior mutability because it is stuffed inside
/// the global `Config`, which does not allow mutation.
state: RefCell<CacheState>,
}
impl CacheLocker {
/// Creates a new `CacheLocker`.
pub fn new() -> CacheLocker {
CacheLocker {
state: RefCell::new(CacheState {
cache_lock: RecursiveLock::new(CACHE_LOCK_NAME),
mutate_lock: RecursiveLock::new(MUTATE_NAME),
}),
}
}
/// Acquires a lock with the given mode, possibly blocking if another
/// cargo is holding the lock.
pub fn lock(&self, config: &Config, mode: CacheLockMode) -> CargoResult<CacheLock<'_>> {
let mut state = self.state.borrow_mut();
let _ = state.lock(config, mode, Blocking)?;
Ok(CacheLock { mode, locker: self })
}
/// Acquires a lock with the given mode, returning `None` if another cargo
/// is holding the lock.
pub fn try_lock(
&self,
config: &Config,
mode: CacheLockMode,
) -> CargoResult<Option<CacheLock<'_>>> {
let mut state = self.state.borrow_mut();
if state.lock(config, mode, NonBlocking)? == LockAcquired {
Ok(Some(CacheLock { mode, locker: self }))
} else {
Ok(None)
}
}
/// Returns whether or not a lock is held for the given mode in this locker.
///
/// This does not tell you whether or not it is locked in some other
/// locker (such as in another process).
///
/// Note that `Shared` will return true if a `MutateExclusive` lock is
/// held, since `MutateExclusive` is just an upgraded `Shared`. Likewise,
/// `DownlaodExclusive` will return true if a `MutateExclusive` lock is
/// held since they overlap.
pub fn is_locked(&self, mode: CacheLockMode) -> bool {
let state = self.state.borrow();
match (
mode,
state.cache_lock.count,
state.mutate_lock.count,
state.mutate_lock.is_exclusive,
) {
(CacheLockMode::Shared, _, 1.., _) => true,
(CacheLockMode::MutateExclusive, _, 1.., true) => true,
(CacheLockMode::DownloadExclusive, 1.., _, _) => true,
_ => false,
}
}
}
/// Returns whether or not the error appears to be from a read-only filesystem.
fn maybe_readonly(err: &anyhow::Error) -> bool {
err.chain().any(|err| {
if let Some(io) = err.downcast_ref::<io::Error>() {
if io.kind() == io::ErrorKind::PermissionDenied {
return true;
}
#[cfg(unix)]
return io.raw_os_error() == Some(libc::EROFS);
}
false
})
}

View File

@ -49,6 +49,7 @@
//! translate from `ConfigValue` and environment variables to the caller's
//! desired type.
use crate::util::cache_lock::{CacheLock, CacheLockMode, CacheLocker};
use std::borrow::Cow;
use std::cell::{RefCell, RefMut};
use std::collections::hash_map::Entry::{Occupied, Vacant};
@ -58,7 +59,7 @@ use std::ffi::{OsStr, OsString};
use std::fmt;
use std::fs::{self, File};
use std::io::prelude::*;
use std::io::{self, SeekFrom};
use std::io::SeekFrom;
use std::mem;
use std::path::{Path, PathBuf};
use std::str::FromStr;
@ -78,7 +79,7 @@ use crate::util::network::http::http_handle;
use crate::util::toml as cargo_toml;
use crate::util::{internal, CanonicalUrl};
use crate::util::{try_canonicalize, validate_package_name};
use crate::util::{FileLock, Filesystem, IntoUrl, IntoUrlWithBase, Rustc};
use crate::util::{Filesystem, IntoUrl, IntoUrlWithBase, Rustc};
use anyhow::{anyhow, bail, format_err, Context as _};
use cargo_credential::Secret;
use cargo_util::paths;
@ -215,9 +216,8 @@ pub struct Config {
credential_cache: LazyCell<RefCell<HashMap<CanonicalUrl, CredentialCacheValue>>>,
/// Cache of registry config from from the `[registries]` table.
registry_config: LazyCell<RefCell<HashMap<SourceId, Option<RegistryConfig>>>>,
/// Lock, if held, of the global package cache along with the number of
/// acquisitions so far.
package_cache_lock: RefCell<Option<(Option<FileLock>, usize)>>,
/// Locks on the package and index caches.
package_cache_lock: CacheLocker,
/// Cached configuration parsed by Cargo
http_config: LazyCell<CargoHttpConfig>,
future_incompat_config: LazyCell<CargoFutureIncompatConfig>,
@ -307,7 +307,7 @@ impl Config {
updated_sources: LazyCell::new(),
credential_cache: LazyCell::new(),
registry_config: LazyCell::new(),
package_cache_lock: RefCell::new(None),
package_cache_lock: CacheLocker::new(),
http_config: LazyCell::new(),
future_incompat_config: LazyCell::new(),
net_config: LazyCell::new(),
@ -1876,10 +1876,20 @@ impl Config {
T::deserialize(d).map_err(|e| e.into())
}
pub fn assert_package_cache_locked<'a>(&self, f: &'a Filesystem) -> &'a Path {
/// Obtain a [`Path`] from a [`Filesystem`], verifying that the
/// appropriate lock is already currently held.
///
/// Locks are usually acquired via [`Config::acquire_package_cache_lock`]
/// or [`Config::try_acquire_package_cache_lock`].
#[track_caller]
pub fn assert_package_cache_locked<'a>(
&self,
mode: CacheLockMode,
f: &'a Filesystem,
) -> &'a Path {
let ret = f.as_path_unlocked();
assert!(
self.package_cache_lock.borrow().is_some(),
self.package_cache_lock.is_locked(mode),
"package cache lock is not currently held, Cargo forgot to call \
`acquire_package_cache_lock` before we got to this stack frame",
);
@ -1887,72 +1897,26 @@ impl Config {
ret
}
/// Acquires an exclusive lock on the global "package cache"
/// Acquires a lock on the global "package cache", blocking if another
/// cargo holds the lock.
///
/// This lock is global per-process and can be acquired recursively. An RAII
/// structure is returned to release the lock, and if this process
/// abnormally terminates the lock is also released.
pub fn acquire_package_cache_lock(&self) -> CargoResult<PackageCacheLock<'_>> {
let mut slot = self.package_cache_lock.borrow_mut();
match *slot {
// We've already acquired the lock in this process, so simply bump
// the count and continue.
Some((_, ref mut cnt)) => {
*cnt += 1;
}
None => {
let path = ".package-cache";
let desc = "package cache";
// First, attempt to open an exclusive lock which is in general
// the purpose of this lock!
//
// If that fails because of a readonly filesystem or a
// permission error, though, then we don't really want to fail
// just because of this. All files that this lock protects are
// in subfolders, so they're assumed by Cargo to also be
// readonly or have invalid permissions for us to write to. If
// that's the case, then we don't really need to grab a lock in
// the first place here.
//
// Despite this we attempt to grab a readonly lock. This means
// that if our read-only folder is shared read-write with
// someone else on the system we should synchronize with them,
// but if we can't even do that then we did our best and we just
// keep on chugging elsewhere.
match self.home_path.open_rw(path, self, desc) {
Ok(lock) => *slot = Some((Some(lock), 1)),
Err(e) => {
if maybe_readonly(&e) {
let lock = self.home_path.open_ro(path, self, desc).ok();
*slot = Some((lock, 1));
return Ok(PackageCacheLock(self));
}
Err(e).with_context(|| "failed to acquire package cache lock")?;
}
}
}
}
return Ok(PackageCacheLock(self));
fn maybe_readonly(err: &anyhow::Error) -> bool {
err.chain().any(|err| {
if let Some(io) = err.downcast_ref::<io::Error>() {
if io.kind() == io::ErrorKind::PermissionDenied {
return true;
}
#[cfg(unix)]
return io.raw_os_error() == Some(libc::EROFS);
}
false
})
}
/// See [`crate::util::cache_lock`] for an in-depth discussion of locking
/// and lock modes.
pub fn acquire_package_cache_lock(&self, mode: CacheLockMode) -> CargoResult<CacheLock<'_>> {
self.package_cache_lock.lock(self, mode)
}
pub fn release_package_cache_lock(&self) {}
/// Acquires a lock on the global "package cache", returning `None` if
/// another cargo holds the lock.
///
/// See [`crate::util::cache_lock`] for an in-depth discussion of locking
/// and lock modes.
pub fn try_acquire_package_cache_lock(
&self,
mode: CacheLockMode,
) -> CargoResult<Option<CacheLock<'_>>> {
self.package_cache_lock.try_lock(self, mode)
}
}
/// Internal error for serde errors.
@ -2271,7 +2235,7 @@ pub fn save_credentials(
let mut file = {
cfg.home_path.create_dir()?;
cfg.home_path
.open_rw(filename, cfg, "credentials' config file")?
.open_rw_exclusive_create(filename, cfg, "credentials' config file")?
};
let mut contents = String::new();
@ -2390,19 +2354,6 @@ pub fn save_credentials(
}
}
pub struct PackageCacheLock<'a>(&'a Config);
impl Drop for PackageCacheLock<'_> {
fn drop(&mut self) {
let mut slot = self.0.package_cache_lock.borrow_mut();
let (_, cnt) = slot.as_mut().unwrap();
*cnt -= 1;
if *cnt == 0 {
*slot = None;
}
}
}
#[derive(Debug, Default, Deserialize, PartialEq)]
#[serde(rename_all = "kebab-case")]
pub struct CargoHttpConfig {

View File

@ -1,3 +1,12 @@
//! File-locking support.
//!
//! This module defines the [`Filesystem`] type which is an abstraction over a
//! filesystem, ensuring that access to the filesystem is only done through
//! coordinated locks.
//!
//! The [`FileLock`] type represents a locked file, and provides access to the
//! file.
use std::fs::{File, OpenOptions};
use std::io;
use std::io::{Read, Seek, SeekFrom, Write};
@ -10,18 +19,22 @@ use anyhow::Context as _;
use cargo_util::paths;
use sys::*;
/// A locked file.
///
/// This provides access to file while holding a lock on the file. This type
/// implements the [`Read`], [`Write`], and [`Seek`] traits to provide access
/// to the underlying file.
///
/// Locks are either shared (multiple processes can access the file) or
/// exclusive (only one process can access the file).
///
/// This type is created via methods on the [`Filesystem`] type.
///
/// When this value is dropped, the lock will be released.
#[derive(Debug)]
pub struct FileLock {
f: Option<File>,
path: PathBuf,
state: State,
}
#[derive(PartialEq, Debug)]
enum State {
Unlocked,
Shared,
Exclusive,
}
impl FileLock {
@ -35,13 +48,11 @@ impl FileLock {
/// Note that special care must be taken to ensure that the path is not
/// referenced outside the lifetime of this lock.
pub fn path(&self) -> &Path {
assert_ne!(self.state, State::Unlocked);
&self.path
}
/// Returns the parent path containing this file
pub fn parent(&self) -> &Path {
assert_ne!(self.state, State::Unlocked);
self.path.parent().unwrap()
}
@ -91,9 +102,9 @@ impl Write for FileLock {
impl Drop for FileLock {
fn drop(&mut self) {
if self.state != State::Unlocked {
if let Some(f) = self.f.take() {
let _ = unlock(&f);
if let Some(f) = self.f.take() {
if let Err(e) = unlock(&f) {
tracing::warn!("failed to release lock: {e:?}");
}
}
}
@ -105,6 +116,32 @@ impl Drop for FileLock {
/// The `Path` of a filesystem cannot be learned unless it's done in a locked
/// fashion, and otherwise functions on this structure are prepared to handle
/// concurrent invocations across multiple instances of Cargo.
///
/// The methods on `Filesystem` that open files return a [`FileLock`] which
/// holds the lock, and that type provides methods for accessing the
/// underlying file.
///
/// If the blocking methods (like [`Filesystem::open_ro_shared`]) detect that
/// they will block, then they will display a message to the user letting them
/// know it is blocked. There are non-blocking variants starting with the
/// `try_` prefix like [`Filesystem::try_open_ro_shared_create`].
///
/// The behavior of locks acquired by the `Filesystem` depend on the operating
/// system. On unix-like system, they are advisory using [`flock`], and thus
/// not enforced against processes which do not try to acquire the lock. On
/// Windows, they are mandatory using [`LockFileEx`], enforced against all
/// processes.
///
/// This **does not** guarantee that a lock is acquired. In some cases, for
/// example on filesystems that don't support locking, it will return a
/// [`FileLock`] even though the filesystem lock was not acquired. This is
/// intended to provide a graceful fallback instead of refusing to work.
/// Usually there aren't multiple processes accessing the same resource. In
/// that case, it is the user's responsibility to not run concurrent
/// processes.
///
/// [`flock`]: https://linux.die.net/man/2/flock
/// [`LockFileEx`]: https://learn.microsoft.com/en-us/windows/win32/api/fileapi/nf-fileapi-lockfileex
#[derive(Clone, Debug)]
pub struct Filesystem {
root: PathBuf,
@ -157,68 +194,119 @@ impl Filesystem {
self.root.display()
}
/// Opens exclusive access to a file, returning the locked version of a
/// file.
/// Opens read-write exclusive access to a file, returning the locked
/// version of a file.
///
/// This function will create a file at `path` if it doesn't already exist
/// (including intermediate directories), and then it will acquire an
/// exclusive lock on `path`. If the process must block waiting for the
/// lock, the `msg` is printed to `config`.
/// lock, the `msg` is printed to [`Config`].
///
/// The returned file can be accessed to look at the path and also has
/// read/write access to the underlying file.
pub fn open_rw<P>(&self, path: P, config: &Config, msg: &str) -> CargoResult<FileLock>
pub fn open_rw_exclusive_create<P>(
&self,
path: P,
config: &Config,
msg: &str,
) -> CargoResult<FileLock>
where
P: AsRef<Path>,
{
self.open(
path.as_ref(),
OpenOptions::new().read(true).write(true).create(true),
State::Exclusive,
config,
msg,
)
let mut opts = OpenOptions::new();
opts.read(true).write(true).create(true);
let (path, f) = self.open(path.as_ref(), &opts, true)?;
acquire(config, msg, &path, &|| try_lock_exclusive(&f), &|| {
lock_exclusive(&f)
})?;
Ok(FileLock { f: Some(f), path })
}
/// Opens shared access to a file, returning the locked version of a file.
/// A non-blocking version of [`Filesystem::open_rw_exclusive_create`].
///
/// Returns `None` if the operation would block due to another process
/// holding the lock.
pub fn try_open_rw_exclusive_create<P: AsRef<Path>>(
&self,
path: P,
) -> CargoResult<Option<FileLock>> {
let mut opts = OpenOptions::new();
opts.read(true).write(true).create(true);
let (path, f) = self.open(path.as_ref(), &opts, true)?;
if try_acquire(&path, &|| try_lock_exclusive(&f))? {
Ok(Some(FileLock { f: Some(f), path }))
} else {
Ok(None)
}
}
/// Opens read-only shared access to a file, returning the locked version of a file.
///
/// This function will fail if `path` doesn't already exist, but if it does
/// then it will acquire a shared lock on `path`. If the process must block
/// waiting for the lock, the `msg` is printed to `config`.
/// waiting for the lock, the `msg` is printed to [`Config`].
///
/// The returned file can be accessed to look at the path and also has read
/// access to the underlying file. Any writes to the file will return an
/// error.
pub fn open_ro<P>(&self, path: P, config: &Config, msg: &str) -> CargoResult<FileLock>
pub fn open_ro_shared<P>(&self, path: P, config: &Config, msg: &str) -> CargoResult<FileLock>
where
P: AsRef<Path>,
{
self.open(
path.as_ref(),
OpenOptions::new().read(true),
State::Shared,
config,
msg,
)
let (path, f) = self.open(path.as_ref(), &OpenOptions::new().read(true), false)?;
acquire(config, msg, &path, &|| try_lock_shared(&f), &|| {
lock_shared(&f)
})?;
Ok(FileLock { f: Some(f), path })
}
fn open(
/// Opens read-only shared access to a file, returning the locked version of a file.
///
/// Compared to [`Filesystem::open_ro_shared`], this will create the file
/// (and any directories in the parent) if the file does not already
/// exist.
pub fn open_ro_shared_create<P: AsRef<Path>>(
&self,
path: &Path,
opts: &OpenOptions,
state: State,
path: P,
config: &Config,
msg: &str,
) -> CargoResult<FileLock> {
let path = self.root.join(path);
let mut opts = OpenOptions::new();
opts.read(true).write(true).create(true);
let (path, f) = self.open(path.as_ref(), &opts, true)?;
acquire(config, msg, &path, &|| try_lock_shared(&f), &|| {
lock_shared(&f)
})?;
Ok(FileLock { f: Some(f), path })
}
// If we want an exclusive lock then if we fail because of NotFound it's
// likely because an intermediate directory didn't exist, so try to
// create the directory and then continue.
/// A non-blocking version of [`Filesystem::open_ro_shared_create`].
///
/// Returns `None` if the operation would block due to another process
/// holding the lock.
pub fn try_open_ro_shared_create<P: AsRef<Path>>(
&self,
path: P,
) -> CargoResult<Option<FileLock>> {
let mut opts = OpenOptions::new();
opts.read(true).write(true).create(true);
let (path, f) = self.open(path.as_ref(), &opts, true)?;
if try_acquire(&path, &|| try_lock_shared(&f))? {
Ok(Some(FileLock { f: Some(f), path }))
} else {
Ok(None)
}
}
fn open(&self, path: &Path, opts: &OpenOptions, create: bool) -> CargoResult<(PathBuf, File)> {
let path = self.root.join(path);
let f = opts
.open(&path)
.or_else(|e| {
if e.kind() == io::ErrorKind::NotFound && state == State::Exclusive {
// If we were requested to create this file, and there was a
// NotFound error, then that was likely due to missing
// intermediate directories. Try creating them and try again.
if e.kind() == io::ErrorKind::NotFound && create {
paths::create_dir_all(path.parent().unwrap())?;
Ok(opts.open(&path)?)
} else {
@ -226,24 +314,7 @@ impl Filesystem {
}
})
.with_context(|| format!("failed to open: {}", path.display()))?;
match state {
State::Exclusive => {
acquire(config, msg, &path, &|| try_lock_exclusive(&f), &|| {
lock_exclusive(&f)
})?;
}
State::Shared => {
acquire(config, msg, &path, &|| try_lock_shared(&f), &|| {
lock_shared(&f)
})?;
}
State::Unlocked => {}
}
Ok(FileLock {
f: Some(f),
path,
state,
})
Ok((path, f))
}
}
@ -259,28 +330,7 @@ impl PartialEq<Filesystem> for Path {
}
}
/// Acquires a lock on a file in a "nice" manner.
///
/// Almost all long-running blocking actions in Cargo have a status message
/// associated with them as we're not sure how long they'll take. Whenever a
/// conflicted file lock happens, this is the case (we're not sure when the lock
/// will be released).
///
/// This function will acquire the lock on a `path`, printing out a nice message
/// to the console if we have to wait for it. It will first attempt to use `try`
/// to acquire a lock on the crate, and in the case of contention it will emit a
/// status message based on `msg` to `config`'s shell, and then use `block` to
/// block waiting to acquire a lock.
///
/// Returns an error if the lock could not be acquired or if any error other
/// than a contention error happens.
fn acquire(
config: &Config,
msg: &str,
path: &Path,
lock_try: &dyn Fn() -> io::Result<()>,
lock_block: &dyn Fn() -> io::Result<()>,
) -> CargoResult<()> {
fn try_acquire(path: &Path, lock_try: &dyn Fn() -> io::Result<()>) -> CargoResult<bool> {
// File locking on Unix is currently implemented via `flock`, which is known
// to be broken on NFS. We could in theory just ignore errors that happen on
// NFS, but apparently the failure mode [1] for `flock` on NFS is **blocking
@ -292,16 +342,17 @@ fn acquire(
//
// [1]: https://github.com/rust-lang/cargo/issues/2615
if is_on_nfs_mount(path) {
return Ok(());
tracing::debug!("{path:?} appears to be an NFS mount, not trying to lock");
return Ok(true);
}
match lock_try() {
Ok(()) => return Ok(()),
Ok(()) => return Ok(true),
// In addition to ignoring NFS which is commonly not working we also
// just ignore locking on filesystems that look like they don't
// implement file locking.
Err(e) if error_unsupported(&e) => return Ok(()),
Err(e) if error_unsupported(&e) => return Ok(true),
Err(e) => {
if !error_contended(&e) {
@ -311,36 +362,64 @@ fn acquire(
}
}
}
Ok(false)
}
/// Acquires a lock on a file in a "nice" manner.
///
/// Almost all long-running blocking actions in Cargo have a status message
/// associated with them as we're not sure how long they'll take. Whenever a
/// conflicted file lock happens, this is the case (we're not sure when the lock
/// will be released).
///
/// This function will acquire the lock on a `path`, printing out a nice message
/// to the console if we have to wait for it. It will first attempt to use `try`
/// to acquire a lock on the crate, and in the case of contention it will emit a
/// status message based on `msg` to [`Config`]'s shell, and then use `block` to
/// block waiting to acquire a lock.
///
/// Returns an error if the lock could not be acquired or if any error other
/// than a contention error happens.
fn acquire(
config: &Config,
msg: &str,
path: &Path,
lock_try: &dyn Fn() -> io::Result<()>,
lock_block: &dyn Fn() -> io::Result<()>,
) -> CargoResult<()> {
if try_acquire(path, lock_try)? {
return Ok(());
}
let msg = format!("waiting for file lock on {}", msg);
config
.shell()
.status_with_color("Blocking", &msg, &style::NOTE)?;
lock_block().with_context(|| format!("failed to lock file: {}", path.display()))?;
return Ok(());
Ok(())
}
#[cfg(all(target_os = "linux", not(target_env = "musl")))]
fn is_on_nfs_mount(path: &Path) -> bool {
use std::ffi::CString;
use std::mem;
use std::os::unix::prelude::*;
#[cfg(all(target_os = "linux", not(target_env = "musl")))]
fn is_on_nfs_mount(path: &Path) -> bool {
use std::ffi::CString;
use std::mem;
use std::os::unix::prelude::*;
let Ok(path) = CString::new(path.as_os_str().as_bytes()) else {
return false;
};
let Ok(path) = CString::new(path.as_os_str().as_bytes()) else {
return false;
};
unsafe {
let mut buf: libc::statfs = mem::zeroed();
let r = libc::statfs(path.as_ptr(), &mut buf);
unsafe {
let mut buf: libc::statfs = mem::zeroed();
let r = libc::statfs(path.as_ptr(), &mut buf);
r == 0 && buf.f_type as u32 == libc::NFS_SUPER_MAGIC as u32
}
r == 0 && buf.f_type as u32 == libc::NFS_SUPER_MAGIC as u32
}
}
#[cfg(any(not(target_os = "linux"), target_env = "musl"))]
fn is_on_nfs_mount(_path: &Path) -> bool {
false
}
#[cfg(any(not(target_os = "linux"), target_env = "musl"))]
fn is_on_nfs_mount(_path: &Path) -> bool {
false
}
#[cfg(unix)]

View File

@ -31,6 +31,7 @@ pub use self::workspace::{
};
pub mod auth;
pub mod cache_lock;
mod canonical_url;
pub mod command_prelude;
pub mod config;

View File

@ -0,0 +1,304 @@
//! Tests for `CacheLock`.
use crate::config::ConfigBuilder;
use cargo::util::cache_lock::{CacheLockMode, CacheLocker};
use cargo_test_support::paths::{self, CargoPathExt};
use cargo_test_support::{retry, thread_wait_timeout, threaded_timeout};
use std::thread::JoinHandle;
/// Helper to verify that it is OK to acquire the given lock (it shouldn't block).
fn verify_lock_is_ok(mode: CacheLockMode) {
let root = paths::root();
threaded_timeout(10, move || {
let config = ConfigBuilder::new().root(root).build();
let locker = CacheLocker::new();
// This would block if it is held.
let _lock = locker.lock(&config, mode).unwrap();
assert!(locker.is_locked(mode));
});
}
/// Helper to acquire two locks from the same locker.
fn a_b_nested(a: CacheLockMode, b: CacheLockMode) {
let config = ConfigBuilder::new().build();
let locker = CacheLocker::new();
let lock1 = locker.lock(&config, a).unwrap();
assert!(locker.is_locked(a));
let lock2 = locker.lock(&config, b).unwrap();
assert!(locker.is_locked(b));
drop(lock2);
drop(lock1);
// Verify locks were unlocked.
verify_lock_is_ok(CacheLockMode::Shared);
verify_lock_is_ok(CacheLockMode::DownloadExclusive);
verify_lock_is_ok(CacheLockMode::MutateExclusive);
}
/// Helper to acquire two locks from separate lockers, verifying that they
/// don't block each other.
fn a_then_b_separate_not_blocked(a: CacheLockMode, b: CacheLockMode, verify: CacheLockMode) {
let config = ConfigBuilder::new().build();
let locker1 = CacheLocker::new();
let lock1 = locker1.lock(&config, a).unwrap();
assert!(locker1.is_locked(a));
let locker2 = CacheLocker::new();
let lock2 = locker2.lock(&config, b).unwrap();
assert!(locker2.is_locked(b));
let thread = verify_lock_would_block(verify);
// Unblock the thread.
drop(lock1);
drop(lock2);
// Verify the thread is unblocked.
thread_wait_timeout::<()>(100, thread);
}
/// Helper to acquire two locks from separate lockers, verifying that the
/// second one blocks.
fn a_then_b_separate_blocked(a: CacheLockMode, b: CacheLockMode) {
let config = ConfigBuilder::new().build();
let locker = CacheLocker::new();
let lock = locker.lock(&config, a).unwrap();
assert!(locker.is_locked(a));
let thread = verify_lock_would_block(b);
// Unblock the thread.
drop(lock);
// Verify the thread is unblocked.
thread_wait_timeout::<()>(100, thread);
}
/// Helper to verify that acquiring the given mode would block.
///
/// Always call `thread_wait_timeout` on the result.
#[must_use]
fn verify_lock_would_block(mode: CacheLockMode) -> JoinHandle<()> {
let root = paths::root();
// Spawn a thread that will block on the lock.
let thread = std::thread::spawn(move || {
let config = ConfigBuilder::new().root(root).build();
let locker2 = CacheLocker::new();
let lock2 = locker2.lock(&config, mode).unwrap();
assert!(locker2.is_locked(mode));
drop(lock2);
});
// Verify that it blocked.
retry(100, || {
if let Ok(s) = std::fs::read_to_string(paths::root().join("shell.out")) {
if s.trim().starts_with("Blocking waiting for file lock on") {
return Some(());
} else {
eprintln!("unexpected output: {s}");
// Try again, it might have been partially written.
}
}
None
});
thread
}
#[test]
fn new_is_unlocked() {
let locker = CacheLocker::new();
assert!(!locker.is_locked(CacheLockMode::Shared));
assert!(!locker.is_locked(CacheLockMode::DownloadExclusive));
assert!(!locker.is_locked(CacheLockMode::MutateExclusive));
}
#[cargo_test]
fn multiple_shared() {
// Test that two nested shared locks from the same locker are safe to acquire.
a_b_nested(CacheLockMode::Shared, CacheLockMode::Shared);
}
#[cargo_test]
fn multiple_shared_separate() {
// Test that two independent shared locks are safe to acquire at the same time.
a_then_b_separate_not_blocked(
CacheLockMode::Shared,
CacheLockMode::Shared,
CacheLockMode::MutateExclusive,
);
}
#[cargo_test]
fn multiple_download() {
// That that two nested download locks from the same locker are safe to acquire.
a_b_nested(
CacheLockMode::DownloadExclusive,
CacheLockMode::DownloadExclusive,
);
}
#[cargo_test]
fn multiple_mutate() {
// That that two nested mutate locks from the same locker are safe to acquire.
a_b_nested(
CacheLockMode::MutateExclusive,
CacheLockMode::MutateExclusive,
);
}
#[cargo_test]
#[should_panic(expected = "lock is not allowed")]
fn download_then_shared() {
// This sequence is not supported.
a_b_nested(CacheLockMode::DownloadExclusive, CacheLockMode::Shared);
}
#[cargo_test]
#[should_panic(expected = "lock upgrade from shared to exclusive not supported")]
fn shared_then_mutate() {
// This sequence is not supported.
a_b_nested(CacheLockMode::Shared, CacheLockMode::MutateExclusive);
}
#[cargo_test]
fn shared_then_download() {
a_b_nested(CacheLockMode::Shared, CacheLockMode::DownloadExclusive);
// Verify drop actually unlocked.
verify_lock_is_ok(CacheLockMode::DownloadExclusive);
verify_lock_is_ok(CacheLockMode::MutateExclusive);
}
#[cargo_test]
fn mutate_then_shared() {
a_b_nested(CacheLockMode::MutateExclusive, CacheLockMode::Shared);
// Verify drop actually unlocked.
verify_lock_is_ok(CacheLockMode::MutateExclusive);
}
#[cargo_test]
fn download_then_mutate() {
a_b_nested(
CacheLockMode::DownloadExclusive,
CacheLockMode::MutateExclusive,
);
// Verify drop actually unlocked.
verify_lock_is_ok(CacheLockMode::DownloadExclusive);
verify_lock_is_ok(CacheLockMode::MutateExclusive);
}
#[cargo_test]
fn mutate_then_download() {
a_b_nested(
CacheLockMode::MutateExclusive,
CacheLockMode::DownloadExclusive,
);
// Verify drop actually unlocked.
verify_lock_is_ok(CacheLockMode::MutateExclusive);
verify_lock_is_ok(CacheLockMode::DownloadExclusive);
}
#[cargo_test]
fn readonly() {
// In a permission denied situation, it should still allow a lock. It just
// silently behaves as-if it was locked.
let cargo_home = paths::home().join(".cargo");
std::fs::create_dir_all(&cargo_home).unwrap();
let mut perms = std::fs::metadata(&cargo_home).unwrap().permissions();
perms.set_readonly(true);
std::fs::set_permissions(&cargo_home, perms).unwrap();
let config = ConfigBuilder::new().build();
let locker = CacheLocker::new();
for mode in [
CacheLockMode::Shared,
CacheLockMode::DownloadExclusive,
CacheLockMode::MutateExclusive,
] {
let _lock1 = locker.lock(&config, mode).unwrap();
// Make sure it can recursively acquire the lock, too.
let _lock2 = locker.lock(&config, mode).unwrap();
}
}
#[cargo_test]
fn download_then_shared_separate() {
a_then_b_separate_not_blocked(
CacheLockMode::DownloadExclusive,
CacheLockMode::Shared,
CacheLockMode::MutateExclusive,
);
}
#[cargo_test]
fn shared_then_download_separate() {
a_then_b_separate_not_blocked(
CacheLockMode::Shared,
CacheLockMode::DownloadExclusive,
CacheLockMode::MutateExclusive,
);
}
#[cargo_test]
fn multiple_download_separate() {
// Test that with two independent download locks, the second blocks until
// the first is released.
a_then_b_separate_blocked(
CacheLockMode::DownloadExclusive,
CacheLockMode::DownloadExclusive,
);
}
#[cargo_test]
fn multiple_mutate_separate() {
// Test that with two independent mutate locks, the second blocks until
// the first is released.
a_then_b_separate_blocked(
CacheLockMode::MutateExclusive,
CacheLockMode::MutateExclusive,
);
}
#[cargo_test]
fn shared_then_mutate_separate() {
a_then_b_separate_blocked(CacheLockMode::Shared, CacheLockMode::MutateExclusive);
}
#[cargo_test]
fn download_then_mutate_separate() {
a_then_b_separate_blocked(
CacheLockMode::DownloadExclusive,
CacheLockMode::MutateExclusive,
);
}
#[cargo_test]
fn mutate_then_download_separate() {
a_then_b_separate_blocked(
CacheLockMode::MutateExclusive,
CacheLockMode::DownloadExclusive,
);
}
#[cargo_test]
fn mutate_then_shared_separate() {
a_then_b_separate_blocked(CacheLockMode::MutateExclusive, CacheLockMode::Shared);
}
#[cargo_test(ignore_windows = "no method to prevent creating or locking a file")]
fn mutate_err_is_atomic() {
// Verifies that when getting a mutate lock, that if the first lock
// succeeds, but the second one fails, that the first lock is released.
let config = ConfigBuilder::new().build();
let locker = CacheLocker::new();
let cargo_home = config.home().as_path_unlocked();
let cache_path = cargo_home.join(".package-cache");
// This is a hacky way to force an error acquiring the download lock. By
// making it a directory, it is unable to open it.
// TODO: Unfortunately this doesn't work on Windows. I don't have any
// ideas on how to simulate an error on Windows.
cache_path.mkdir_p();
match locker.lock(&config, CacheLockMode::MutateExclusive) {
Ok(_) => panic!("did not expect lock to succeed"),
Err(e) => {
let msg = format!("{e:?}");
assert!(msg.contains("failed to open:"), "{msg}");
}
}
assert!(!locker.is_locked(CacheLockMode::MutateExclusive));
assert!(!locker.is_locked(CacheLockMode::DownloadExclusive));
assert!(!locker.is_locked(CacheLockMode::Shared));
cache_path.rm_rf();
verify_lock_is_ok(CacheLockMode::DownloadExclusive);
verify_lock_is_ok(CacheLockMode::Shared);
verify_lock_is_ok(CacheLockMode::MutateExclusive);
}

View File

@ -21,6 +21,7 @@ pub struct ConfigBuilder {
unstable: Vec<String>,
config_args: Vec<String>,
cwd: Option<PathBuf>,
root: Option<PathBuf>,
enable_nightly_features: bool,
}
@ -30,6 +31,7 @@ impl ConfigBuilder {
env: HashMap::new(),
unstable: Vec::new(),
config_args: Vec::new(),
root: None,
cwd: None,
enable_nightly_features: false,
}
@ -60,8 +62,28 @@ impl ConfigBuilder {
}
/// Sets the current working directory where config files will be loaded.
///
/// Default is the root from [`ConfigBuilder::root`] or [`paths::root`].
pub fn cwd(&mut self, path: impl AsRef<Path>) -> &mut Self {
self.cwd = Some(paths::root().join(path.as_ref()));
let path = path.as_ref();
let cwd = self
.root
.as_ref()
.map_or_else(|| paths::root().join(path), |r| r.join(path));
self.cwd = Some(cwd);
self
}
/// Sets the test root directory.
///
/// This generally should not be necessary. It is only useful if you want
/// to create a `Config` from within a thread. Since Cargo's testsuite
/// uses thread-local storage, this can be used to avoid accessing that
/// thread-local storage.
///
/// Default is [`paths::root`].
pub fn root(&mut self, path: impl Into<PathBuf>) -> &mut Self {
self.root = Some(path.into());
self
}
@ -72,14 +94,15 @@ impl ConfigBuilder {
/// Creates the `Config`, returning a Result.
pub fn build_err(&self) -> CargoResult<Config> {
let output = Box::new(fs::File::create(paths::root().join("shell.out")).unwrap());
let root = self.root.clone().unwrap_or_else(|| paths::root());
let output = Box::new(fs::File::create(root.join("shell.out")).unwrap());
let shell = Shell::from_write(output);
let cwd = self.cwd.clone().unwrap_or_else(|| paths::root());
let homedir = paths::home();
let cwd = self.cwd.clone().unwrap_or_else(|| root.clone());
let homedir = root.join("home").join(".cargo");
let mut config = Config::new(shell, cwd, homedir);
config.nightly_features_allowed = self.enable_nightly_features || !self.unstable.is_empty();
config.set_env(self.env.clone());
config.set_search_stop_path(paths::root());
config.set_search_stop_path(&root);
config.configure(
0,
false,

View File

@ -17,6 +17,7 @@ mod build_plan;
mod build_script;
mod build_script_env;
mod build_script_extra_link_arg;
mod cache_lock;
mod cache_messages;
mod cargo;
mod cargo_add;

View File

@ -1,5 +1,6 @@
//! Tests for the `cargo search` command.
use cargo::util::cache_lock::CacheLockMode;
use cargo_test_support::cargo_process;
use cargo_test_support::paths;
use cargo_test_support::registry::{RegistryBuilder, Response};
@ -100,7 +101,9 @@ fn not_update() {
paths::root(),
paths::home().join(".cargo"),
);
let lock = cfg.acquire_package_cache_lock().unwrap();
let lock = cfg
.acquire_package_cache_lock(CacheLockMode::DownloadExclusive)
.unwrap();
let mut regsrc = RegistrySource::remote(sid, &HashSet::new(), &cfg).unwrap();
regsrc.invalidate_cache();
regsrc.block_until_ready().unwrap();