Only use a single cache file per Python package (#5117)

## Summary

This changes the caching design from one cache file per source file, to
one cache file per package. This greatly reduces the amount of cache
files that are opened and written, while maintaining roughly the same
(combined) size as bincode is very compact.

Below are some very much not scientific performance tests. It uses
projects/sources to check:

* small.py: single, 31 bytes Python file with 2 errors.
* test.py: single, 43k Python file with 8 errors.
* fastapi: FastAPI repo, 1134 files checked, 0 errors.

Source   | Before # files | After # files | Before size | After size
-------|-------|-------|-------|-------
small.py | 1              | 1             | 20 K        | 20 K
test.py  | 1              | 1             | 60 K        | 60 K
fastapi  | 1134           | 518           | 4.5 M       | 2.3 M

One question that might come up is why fastapi still has 518 cache files
and not 1? That is because this is using the existing package
resolution, which sees examples, docs, etc. as separate from the "main"
source code (in the fastapi directory in the repo). In this future it
might be worth consider switching to a one cache file per repo strategy.

This new design is not perfect and does have a number of known issues.
First, like the old design it doesn't remove the cache for a source file
that has been (re)moved until `ruff clean` is called.

Second, this currently uses a large mutex around the mutation of the
package cache (e.g. inserting result). This could be (or become) a
bottleneck. It's future work to test and improve this (if needed).

Third, currently the packages and opened and stored in a sequential
loop, this could be done parallel. This is also future work.


## Test Plan

Run `ruff check` (with caching enabled) twice on any Python source code
and it should produce the same results.
This commit is contained in:
Thomas de Zeeuw 2023-06-19 17:46:13 +02:00 committed by GitHub
parent b8d378b0a3
commit e3c12764f8
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 285 additions and 249 deletions

1
Cargo.lock generated
View File

@ -1908,6 +1908,7 @@ dependencies = [
"glob",
"ignore",
"itertools",
"itoa",
"log",
"mimalloc",
"notify",

View File

@ -8,8 +8,7 @@ pub mod globset;
pub const CACHE_DIR_NAME: &str = ".ruff_cache";
/// Return the cache directory for a given project root. Defers to the
/// `RUFF_CACHE_DIR` environment variable, if set.
/// Return the cache directory for a given project root.
pub fn cache_dir(project_root: &Path) -> PathBuf {
project_root.join(CACHE_DIR_NAME)
}

View File

@ -47,6 +47,7 @@ filetime = { workspace = true }
glob = { workspace = true }
ignore = { workspace = true }
itertools = { workspace = true }
itoa = { version = "1.0.6" }
log = { workspace = true }
notify = { version = "5.1.0" }
path-absolutize = { workspace = true, features = ["once_cell_cache"] }

View File

@ -1,171 +1,247 @@
use std::cell::RefCell;
use std::fs;
use std::collections::HashMap;
use std::fs::{self, File};
use std::hash::Hasher;
use std::io::Write;
#[cfg(unix)]
use std::os::unix::fs::PermissionsExt;
use std::path::Path;
use std::io::{self, BufReader, BufWriter, Write};
use std::path::{Path, PathBuf};
use std::sync::Mutex;
use std::time::SystemTime;
use anyhow::Result;
use filetime::FileTime;
use log::error;
use path_absolutize::Absolutize;
use ruff_text_size::{TextRange, TextSize};
use serde::ser::{SerializeSeq, SerializeStruct};
use serde::{Deserialize, Serialize, Serializer};
use anyhow::{anyhow, Context, Result};
use serde::{Deserialize, Serialize};
use ruff::message::Message;
use ruff::settings::{AllSettings, Settings};
use ruff::settings::Settings;
use ruff_cache::{CacheKey, CacheKeyHasher};
use ruff_diagnostics::{DiagnosticKind, Fix};
use ruff_python_ast::imports::ImportMap;
use ruff_python_ast::source_code::SourceFileBuilder;
use ruff_text_size::{TextRange, TextSize};
const CARGO_PKG_VERSION: &str = env!("CARGO_PKG_VERSION");
use crate::diagnostics::Diagnostics;
/// Vec storing all source files. The tuple is (filename, source code).
type Files<'a> = Vec<(&'a str, &'a str)>;
type FilesBuf = Vec<(String, String)>;
struct CheckResultRef<'a> {
imports: &'a ImportMap,
messages: &'a [Message],
/// On disk representation of a cache of a package.
#[derive(Deserialize, Debug, Serialize)]
pub(crate) struct PackageCache {
/// Location of the cache.
///
/// Not stored on disk, just used as a storage location.
#[serde(skip)]
path: PathBuf,
/// Path to the root of the package.
///
/// Usually this is a directory, but it can also be a single file in case of
/// single file "packages", e.g. scripts.
package_root: PathBuf,
/// Mapping of source file path to it's cached data.
// TODO: look into concurrent hashmap or similar instead of a mutex.
files: Mutex<HashMap<RelativePathBuf, FileCache>>,
}
impl Serialize for CheckResultRef<'_> {
fn serialize<S>(&self, serializer: S) -> std::result::Result<S::Ok, S::Error>
where
S: Serializer,
{
let mut s = serializer.serialize_struct("CheckResultRef", 3)?;
impl PackageCache {
/// Open or create a new package cache.
///
/// `package_root` must be canonicalized.
pub(crate) fn open(
cache_dir: &Path,
package_root: PathBuf,
settings: &Settings,
) -> Result<PackageCache> {
debug_assert!(package_root.is_absolute(), "package root not canonicalized");
s.serialize_field("imports", &self.imports)?;
let mut buf = itoa::Buffer::new();
let key = Path::new(buf.format(cache_key(&package_root, settings)));
let path = PathBuf::from_iter([cache_dir, Path::new("content"), key]);
let serialize_messages = SerializeMessages {
messages: self.messages,
files: RefCell::default(),
let file = match File::open(&path) {
Ok(file) => file,
Err(err) if err.kind() == io::ErrorKind::NotFound => {
// No cache exist yet, return an empty cache.
return Ok(PackageCache {
path,
package_root,
files: Mutex::new(HashMap::new()),
});
}
Err(err) => {
return Err(err)
.with_context(|| format!("Failed to open cache file '{}'", path.display()))?
}
};
s.serialize_field("messages", &serialize_messages)?;
let mut cache: PackageCache = bincode::deserialize_from(BufReader::new(file))
.with_context(|| format!("Failed parse cache file '{}'", path.display()))?;
let files = serialize_messages.files.take();
s.serialize_field("files", &files)?;
s.end()
}
}
struct SerializeMessages<'a> {
messages: &'a [Message],
files: RefCell<Files<'a>>,
}
impl Serialize for SerializeMessages<'_> {
fn serialize<S>(&self, serializer: S) -> std::result::Result<S::Ok, S::Error>
where
S: Serializer,
{
let mut s = serializer.serialize_seq(Some(self.messages.len()))?;
let mut files = self.files.borrow_mut();
for message in self.messages {
// Using a Vec instead of a HashMap because the cache is per file and the large majority of
// files have exactly one source file.
let file_id = if let Some(position) = files
.iter()
.position(|(filename, _)| *filename == message.filename())
{
position
} else {
let index = files.len();
files.push((message.filename(), message.file.source_text()));
index
};
s.serialize_element(&SerializeMessage { message, file_id })?;
// Sanity check.
if cache.package_root != package_root {
return Err(anyhow!(
"Different package root in cache: expected '{}', got '{}'",
package_root.display(),
cache.package_root.display(),
));
}
s.end()
cache.path = path;
Ok(cache)
}
/// Store the cache to disk.
pub(crate) fn store(&self) -> Result<()> {
let file = File::create(&self.path)
.with_context(|| format!("Failed to create cache file '{}'", self.path.display()))?;
let writer = BufWriter::new(file);
bincode::serialize_into(writer, &self).with_context(|| {
format!(
"Failed to serialise cache to file '{}'",
self.path.display()
)
})
}
/// Returns the relative path based on `path` and the package root.
///
/// Returns `None` if `path` is not within the package.
pub(crate) fn relative_path<'a>(&self, path: &'a Path) -> Option<&'a RelativePath> {
path.strip_prefix(&self.package_root).ok()
}
/// Get the cached results for a single file at relative `path`. This uses
/// `file_last_modified` to determine if the results are still accurate
/// (i.e. if the file hasn't been modified since the cached run).
///
/// This returns `None` if `file_last_modified` differs from the cached
/// timestamp or if the cache doesn't contain results for the file.
pub(crate) fn get(
&self,
path: &RelativePath,
file_last_modified: SystemTime,
) -> Option<FileCache> {
let files = self.files.lock().unwrap();
let file = files.get(path)?;
// Make sure the file hasn't changed since the cached run.
if file.last_modified != file_last_modified {
return None;
}
Some(file.clone())
}
/// Add or update a file cache at `path` relative to the package root.
pub(crate) fn update(&self, path: RelativePathBuf, file: FileCache) {
self.files.lock().unwrap().insert(path, file);
}
/// Remove a file cache at `path` relative to the package root.
pub(crate) fn remove(&self, path: &RelativePath) {
self.files.lock().unwrap().remove(path);
}
}
struct SerializeMessage<'a> {
message: &'a Message,
file_id: usize,
/// [`Path`] that is relative to the package root in [`PackageCache`].
pub(crate) type RelativePath = Path;
/// [`PathBuf`] that is relative to the package root in [`PackageCache`].
pub(crate) type RelativePathBuf = PathBuf;
/// On disk representation of the cache per source file.
#[derive(Clone, Deserialize, Debug, Serialize)]
pub(crate) struct FileCache {
/// Timestamp when the file was last modified before the (cached) check.
last_modified: SystemTime,
/// Imports made.
imports: ImportMap,
/// Diagnostic messages.
messages: Vec<CacheMessage>,
/// Source code of the file.
///
/// # Notes
///
/// This will be empty if `messages` is empty.
source: String,
}
impl Serialize for SerializeMessage<'_> {
fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
where
S: Serializer,
{
let Message {
kind,
range,
fix,
// Serialized manually for all files
file: _,
noqa_offset: noqa_row,
} = self.message;
impl FileCache {
/// Create a new source file cache.
pub(crate) fn new(
last_modified: SystemTime,
messages: &[Message],
imports: &ImportMap,
) -> FileCache {
let source = if let Some(msg) = messages.first() {
msg.file.source_text().to_owned()
} else {
String::new() // No messages, no need to keep the source!
};
let mut s = serializer.serialize_struct("Message", 5)?;
let messages = messages
.iter()
.map(|msg| {
// Make sure that all message use the same source file.
assert!(
msg.file == messages.first().unwrap().file,
"message uses a different source file"
);
CacheMessage {
kind: msg.kind.clone(),
range: msg.range,
fix: msg.fix.clone(),
noqa_offset: msg.noqa_offset,
}
})
.collect();
s.serialize_field("kind", &kind)?;
s.serialize_field("range", &range)?;
s.serialize_field("fix", &fix)?;
s.serialize_field("file_id", &self.file_id)?;
s.serialize_field("noqa_row", &noqa_row)?;
FileCache {
last_modified,
imports: imports.clone(),
messages,
source,
}
}
s.end()
/// Convert the file cache into `Diagnostics`, using `path` as file name.
pub(crate) fn into_diagnostics(self, path: &Path) -> Diagnostics {
let messages = if self.messages.is_empty() {
Vec::new()
} else {
let file = SourceFileBuilder::new(path.to_string_lossy(), self.source).finish();
self.messages
.into_iter()
.map(|msg| Message {
kind: msg.kind,
range: msg.range,
fix: msg.fix,
file: file.clone(),
noqa_offset: msg.noqa_offset,
})
.collect()
};
Diagnostics::new(messages, self.imports)
}
}
#[derive(Deserialize)]
struct MessageHeader {
/// On disk representation of a diagnostic message.
#[derive(Clone, Deserialize, Debug, Serialize)]
struct CacheMessage {
kind: DiagnosticKind,
/// Range into the message's [`FileCache::source`].
range: TextRange,
fix: Option<Fix>,
file_id: usize,
noqa_row: TextSize,
noqa_offset: TextSize,
}
#[derive(Deserialize)]
struct CheckResult {
imports: ImportMap,
messages: Vec<MessageHeader>,
files: FilesBuf,
}
fn content_dir() -> &'static Path {
Path::new("content")
}
fn cache_key(
path: &Path,
package: Option<&Path>,
metadata: &fs::Metadata,
settings: &Settings,
) -> u64 {
/// Returns a hash key based on the `package_root`, `settings` and the crate
/// version.
fn cache_key(package_root: &Path, settings: &Settings) -> u64 {
let mut hasher = CacheKeyHasher::new();
CARGO_PKG_VERSION.cache_key(&mut hasher);
path.absolutize().unwrap().cache_key(&mut hasher);
package
.as_ref()
.map(|path| path.absolutize().unwrap())
.cache_key(&mut hasher);
FileTime::from_last_modification_time(metadata).cache_key(&mut hasher);
#[cfg(unix)]
metadata.permissions().mode().cache_key(&mut hasher);
env!("CARGO_PKG_VERSION").cache_key(&mut hasher);
package_root.cache_key(&mut hasher);
settings.cache_key(&mut hasher);
hasher.finish()
}
#[allow(dead_code)]
/// Initialize the cache at the specified `Path`.
pub(crate) fn init(path: &Path) -> Result<()> {
// Create the cache directories.
fs::create_dir_all(path.join(content_dir()))?;
fs::create_dir_all(path.join("content"))?;
// Add the CACHEDIR.TAG.
if !cachedir::is_tagged(path)? {
@ -181,99 +257,3 @@ pub(crate) fn init(path: &Path) -> Result<()> {
Ok(())
}
fn write_sync(cache_dir: &Path, key: u64, value: &[u8]) -> Result<(), std::io::Error> {
fs::write(
cache_dir.join(content_dir()).join(format!("{key:x}")),
value,
)
}
fn read_sync(cache_dir: &Path, key: u64) -> Result<Vec<u8>, std::io::Error> {
fs::read(cache_dir.join(content_dir()).join(format!("{key:x}")))
}
fn del_sync(cache_dir: &Path, key: u64) -> Result<(), std::io::Error> {
fs::remove_file(cache_dir.join(content_dir()).join(format!("{key:x}")))
}
/// Get a value from the cache.
pub(crate) fn get(
path: &Path,
package: Option<&Path>,
metadata: &fs::Metadata,
settings: &AllSettings,
) -> Option<(Vec<Message>, ImportMap)> {
let encoded = read_sync(
&settings.cli.cache_dir,
cache_key(path, package, metadata, &settings.lib),
)
.ok()?;
match bincode::deserialize::<CheckResult>(&encoded[..]) {
Ok(CheckResult {
messages: headers,
imports,
files: sources,
}) => {
let mut messages = Vec::with_capacity(headers.len());
let source_files: Vec<_> = sources
.into_iter()
.map(|(filename, text)| SourceFileBuilder::new(filename, text).finish())
.collect();
for header in headers {
let Some(source_file) = source_files.get(header.file_id) else {
error!("Failed to retrieve source file for cached entry");
return None;
};
messages.push(Message {
kind: header.kind,
range: header.range,
fix: header.fix,
file: source_file.clone(),
noqa_offset: header.noqa_row,
});
}
Some((messages, imports))
}
Err(e) => {
error!("Failed to deserialize encoded cache entry: {e:?}");
None
}
}
}
/// Set a value in the cache.
pub(crate) fn set(
path: &Path,
package: Option<&Path>,
metadata: &fs::Metadata,
settings: &AllSettings,
messages: &[Message],
imports: &ImportMap,
) {
let check_result = CheckResultRef { imports, messages };
if let Err(e) = write_sync(
&settings.cli.cache_dir,
cache_key(path, package, metadata, &settings.lib),
&bincode::serialize(&check_result).unwrap(),
) {
error!("Failed to write to cache: {e:?}");
}
}
/// Delete a value from the cache.
pub(crate) fn del(
path: &Path,
package: Option<&Path>,
metadata: &fs::Metadata,
settings: &AllSettings,
) {
drop(del_sync(
&settings.cli.cache_dir,
cache_key(path, package, metadata, &settings.lib),
));
}

View File

@ -1,3 +1,5 @@
use std::collections::{hash_map, HashMap};
use std::fmt::Write;
use std::io;
use std::path::{Path, PathBuf};
use std::time::Instant;
@ -20,7 +22,7 @@ use ruff_python_ast::imports::ImportMap;
use ruff_python_ast::source_code::SourceFileBuilder;
use crate::args::Overrides;
use crate::cache;
use crate::cache::{self, PackageCache};
use crate::diagnostics::Diagnostics;
use crate::panic::catch_unwind;
@ -75,6 +77,38 @@ pub(crate) fn run(
pyproject_config,
);
// Create a cache per package, if enabled.
let package_caches = if cache.into() {
let mut caches = HashMap::new();
// TODO(thomas): try to merge this with the detection of package roots
// above or with the parallel iteration below.
for entry in &paths {
let Ok(entry) = entry else { continue };
let path = entry.path();
let package = path
.parent()
.and_then(|parent| package_roots.get(parent))
.and_then(|package| *package);
// For paths not in a package, e.g. scripts, we use the path as
// the package root.
let package_root = package.unwrap_or(path);
let settings = resolver.resolve_all(path, pyproject_config);
if let hash_map::Entry::Vacant(entry) = caches.entry(package_root) {
let cache = PackageCache::open(
&settings.cli.cache_dir,
package_root.to_owned(),
&settings.lib,
)?;
entry.insert(cache);
}
}
Some(caches)
} else {
None
};
let start = Instant::now();
let mut diagnostics: Diagnostics = paths
.par_iter()
@ -86,13 +120,22 @@ pub(crate) fn run(
.parent()
.and_then(|parent| package_roots.get(parent))
.and_then(|package| *package);
let package_cache = package_caches.as_ref().map(|package_caches| {
let package_root = package.unwrap_or(path);
let package_cache = package_caches
.get(package_root)
.expect("failed to get package cache");
package_cache
});
let settings = resolver.resolve_all(path, pyproject_config);
lint_path(path, package, settings, cache, noqa, autofix).map_err(|e| {
lint_path(path, package, settings, package_cache, noqa, autofix).map_err(|e| {
(Some(path.to_owned()), {
let mut error = e.to_string();
for cause in e.chain() {
error += &format!("\n Caused by: {cause}");
write!(&mut error, "\n Caused by: {cause}").unwrap();
}
error
})
@ -145,6 +188,13 @@ pub(crate) fn run(
diagnostics.messages.sort();
// Store the package caches.
if let Some(package_caches) = package_caches {
for package_cache in package_caches.values() {
package_cache.store()?;
}
}
let duration = start.elapsed();
debug!("Checked {:?} files in: {:?}", paths.len(), duration);
@ -157,12 +207,12 @@ fn lint_path(
path: &Path,
package: Option<&Path>,
settings: &AllSettings,
cache: flags::Cache,
package_cache: Option<&PackageCache>,
noqa: flags::Noqa,
autofix: flags::FixMode,
) -> Result<Diagnostics> {
let result = catch_unwind(|| {
crate::diagnostics::lint_path(path, package, settings, cache, noqa, autofix)
crate::diagnostics::lint_path(path, package, settings, package_cache, noqa, autofix)
});
match result {

View File

@ -25,7 +25,7 @@ use ruff_python_ast::imports::ImportMap;
use ruff_python_ast::source_code::{LineIndex, SourceCode, SourceFileBuilder};
use ruff_python_stdlib::path::is_project_toml;
use crate::cache;
use crate::cache::{FileCache, PackageCache};
#[derive(Debug, Default, PartialEq)]
pub(crate) struct Diagnostics {
@ -100,7 +100,7 @@ pub(crate) fn lint_path(
path: &Path,
package: Option<&Path>,
settings: &AllSettings,
cache: flags::Cache,
package_cache: Option<&PackageCache>,
noqa: flags::Noqa,
autofix: flags::FixMode,
) -> Result<Diagnostics> {
@ -110,15 +110,19 @@ pub(crate) fn lint_path(
// to cache `fixer::Mode::Apply`, since a file either has no fixes, or we'll
// write the fixes to disk, thus invalidating the cache. But it's a bit hard
// to reason about. We need to come up with a better solution here.)
let metadata = if cache.into() && noqa.into() && autofix.is_generate() {
let metadata = path.metadata()?;
if let Some((messages, imports)) = cache::get(path, package, &metadata, settings) {
debug!("Cache hit for: {}", path.display());
return Ok(Diagnostics::new(messages, imports));
let caching = match package_cache {
Some(package_cache) if noqa.into() && autofix.is_generate() => {
let relative_path = package_cache
.relative_path(path)
.expect("wrong package cache for file");
let last_modified = path.metadata()?.modified()?;
if let Some(cache) = package_cache.get(relative_path, last_modified) {
return Ok(cache.into_diagnostics(path));
}
Some((package_cache, relative_path, last_modified))
}
Some(metadata)
} else {
None
_ => None,
};
debug!("Checking: {}", path.display());
@ -203,6 +207,17 @@ pub(crate) fn lint_path(
let imports = imports.unwrap_or_default();
if let Some((package_cache, relative_path, file_last_modified)) = caching {
if parse_error.is_some() {
// We don't cache parsing error, so we remove the old file cache (if
// any).
package_cache.remove(relative_path);
} else {
let file_cache = FileCache::new(file_last_modified, &messages, &imports);
package_cache.update(relative_path.to_owned(), file_cache);
}
}
if let Some(err) = parse_error {
error!(
"{}",
@ -212,16 +227,6 @@ pub(crate) fn lint_path(
Some(&source_kind),
)
);
// Purge the cache.
if let Some(metadata) = metadata {
cache::del(path, package, &metadata, settings);
}
} else {
// Re-populate the cache.
if let Some(metadata) = metadata {
cache::set(path, package, &metadata, settings, &messages, &imports);
}
}
Ok(Diagnostics {

View File

@ -202,7 +202,7 @@ impl SourceFile {
.get_or_init(|| LineIndex::from_source_text(self.source_text()))
}
/// Returns `Some` with the source text if set, or `None`.
/// Returns the source code.
#[inline]
pub fn source_text(&self) -> &str {
&self.inner.code