mirror of
https://github.com/astral-sh/uv.git
synced 2025-09-26 12:09:12 +00:00
Add an option to bytecode compile during installation (#2086)
Add a `--compile` option to `pip install` and `pip sync`. I chose to implement this as a separate pass over the entire venv. If we wanted to compile during installation, we'd have to make sure that writing is exclusive, to avoid concurrent processes writing broken `.pyc` files. Additionally, this ensures that the entire site-packages are bytecode compiled, even if there are packages that aren't from this `uv` invocation. The disadvantage is that we do not update RECORD and rely on this comment from [PEP 491](https://peps.python.org/pep-0491/): > Uninstallers should be smart enough to remove .pyc even if it is not mentioned in RECORD. If this is a problem we can change it to run during installation and write RECORD entries. Internally, this is implemented as an async work-stealing subprocess worker pool. The producer is a directory traversal over site-packages, sending each `.py` file to a bounded async FIFO queue/channel. Each worker has a long-running python process. It pops the queue to get a single path (or exists if the channel is closed), then sends it to stdin, waits until it's informed that the compilation is done through a line on stdout, and repeat. This is fast, e.g. installing `jupyter plotly` on Python 3.12 it processes 15876 files in 319ms with 32 threads (vs. 3.8s with a single core). The python processes internally calls `compileall.compile_file`, the same as pip. Like pip, we ignore and silence all compilation errors (https://github.com/astral-sh/uv/issues/1559). There is a 10s timeout to handle the case when the workers got stuck. For the reviewers, please check if i missed any spots where we could deadlock, this is the hardest part of this PR. I've added `uv-dev compile <dir>` and `uv-dev clear-compile <dir>` commands, mainly for my own benchmarking. I don't want to expose them in `uv`, they almost certainly not the correct workflow and we don't want to support them. Fixes #1788 Closes #1559 Closes #1928
This commit is contained in:
parent
93b1395daa
commit
2a53e789b0
17 changed files with 583 additions and 17 deletions
47
Cargo.lock
generated
47
Cargo.lock
generated
|
@ -175,6 +175,19 @@ dependencies = [
|
|||
"tempfile",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "async-channel"
|
||||
version = "2.2.0"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "f28243a43d821d11341ab73c80bed182dc015c514b951616cf79bd4af39af0c3"
|
||||
dependencies = [
|
||||
"concurrent-queue",
|
||||
"event-listener",
|
||||
"event-listener-strategy",
|
||||
"futures-core",
|
||||
"pin-project-lite",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "async-compression"
|
||||
version = "0.4.6"
|
||||
|
@ -620,6 +633,15 @@ version = "1.0.0"
|
|||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "acbf1af155f9b9ef647e42cdc158db4b64a1b61f743629225fde6f3e0be2a7c7"
|
||||
|
||||
[[package]]
|
||||
name = "concurrent-queue"
|
||||
version = "2.4.0"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "d16048cd947b08fa32c24458a22f5dc5e835264f689f4f5653210c69fd107363"
|
||||
dependencies = [
|
||||
"crossbeam-utils",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "configparser"
|
||||
version = "3.0.4"
|
||||
|
@ -955,6 +977,27 @@ dependencies = [
|
|||
"windows-sys 0.52.0",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "event-listener"
|
||||
version = "5.2.0"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "2b5fb89194fa3cad959b833185b3063ba881dbfc7030680b314250779fb4cc91"
|
||||
dependencies = [
|
||||
"concurrent-queue",
|
||||
"parking",
|
||||
"pin-project-lite",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "event-listener-strategy"
|
||||
version = "0.5.0"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "feedafcaa9b749175d5ac357452a9d41ea2911da598fde46ce1fe02c37751291"
|
||||
dependencies = [
|
||||
"event-listener",
|
||||
"pin-project-lite",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "fastrand"
|
||||
version = "2.0.1"
|
||||
|
@ -4329,6 +4372,7 @@ dependencies = [
|
|||
"uv-resolver",
|
||||
"uv-traits",
|
||||
"uv-virtualenv",
|
||||
"walkdir",
|
||||
"which",
|
||||
]
|
||||
|
||||
|
@ -4459,6 +4503,7 @@ name = "uv-installer"
|
|||
version = "0.0.1"
|
||||
dependencies = [
|
||||
"anyhow",
|
||||
"async-channel",
|
||||
"distribution-filename",
|
||||
"distribution-types",
|
||||
"fs-err",
|
||||
|
@ -4489,6 +4534,8 @@ dependencies = [
|
|||
"uv-interpreter",
|
||||
"uv-normalize",
|
||||
"uv-traits",
|
||||
"uv-warnings",
|
||||
"walkdir",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
|
|
|
@ -20,6 +20,7 @@ license = "MIT OR Apache-2.0"
|
|||
anstream = { version = "0.6.13" }
|
||||
anyhow = { version = "1.0.80" }
|
||||
async-compression = { version = "0.4.6" }
|
||||
async-channel = { version = "2.2.0" }
|
||||
async-trait = { version = "0.1.77" }
|
||||
async_http_range_reader = { version = "0.7.0" }
|
||||
async_zip = { git = "https://github.com/charliermarsh/rs-async-zip", rev = "d76801da0943de985254fc6255c0e476b57c5836", features = ["deflate"] }
|
||||
|
|
|
@ -61,6 +61,7 @@ tracing-durations-export = { workspace = true, features = ["plot"] }
|
|||
tracing-indicatif = { workspace = true }
|
||||
tracing-subscriber = { workspace = true }
|
||||
url = { workspace = true }
|
||||
walkdir = { workspace = true }
|
||||
which = { workspace = true }
|
||||
|
||||
[target.'cfg(target_os = "windows")'.dependencies]
|
||||
|
|
33
crates/uv-dev/src/clear_compile.rs
Normal file
33
crates/uv-dev/src/clear_compile.rs
Normal file
|
@ -0,0 +1,33 @@
|
|||
use std::path::PathBuf;
|
||||
|
||||
use clap::Parser;
|
||||
use tracing::info;
|
||||
use walkdir::WalkDir;
|
||||
|
||||
#[derive(Parser)]
|
||||
pub(crate) struct ClearCompileArgs {
|
||||
/// Compile all `.py` in this or any subdirectory to bytecode
|
||||
root: PathBuf,
|
||||
}
|
||||
|
||||
pub(crate) fn clear_compile(args: &ClearCompileArgs) -> anyhow::Result<()> {
|
||||
let mut removed_files = 0;
|
||||
let mut removed_directories = 0;
|
||||
for entry in WalkDir::new(&args.root).contents_first(true) {
|
||||
let entry = entry?;
|
||||
let metadata = entry.metadata()?;
|
||||
if metadata.is_file() {
|
||||
if entry.path().extension().is_some_and(|ext| ext == "pyc") {
|
||||
fs_err::remove_file(entry.path())?;
|
||||
removed_files += 1;
|
||||
}
|
||||
} else if metadata.is_dir() {
|
||||
if entry.file_name() == "__pycache__" {
|
||||
fs_err::remove_dir(entry.path())?;
|
||||
removed_directories += 1;
|
||||
}
|
||||
}
|
||||
}
|
||||
info!("Removed {removed_files} files and {removed_directories} directories");
|
||||
Ok(())
|
||||
}
|
37
crates/uv-dev/src/compile.rs
Normal file
37
crates/uv-dev/src/compile.rs
Normal file
|
@ -0,0 +1,37 @@
|
|||
use std::path::PathBuf;
|
||||
|
||||
use clap::Parser;
|
||||
use platform_host::Platform;
|
||||
use tracing::info;
|
||||
use uv_cache::{Cache, CacheArgs};
|
||||
use uv_interpreter::PythonEnvironment;
|
||||
|
||||
#[derive(Parser)]
|
||||
pub(crate) struct CompileArgs {
|
||||
/// Compile all `.py` in this or any subdirectory to bytecode
|
||||
root: PathBuf,
|
||||
python: Option<PathBuf>,
|
||||
#[command(flatten)]
|
||||
cache_args: CacheArgs,
|
||||
}
|
||||
|
||||
pub(crate) async fn compile(args: CompileArgs) -> anyhow::Result<()> {
|
||||
let cache = Cache::try_from(args.cache_args)?;
|
||||
|
||||
let interpreter = if let Some(python) = args.python {
|
||||
python
|
||||
} else {
|
||||
let platform = Platform::current()?;
|
||||
let venv = PythonEnvironment::from_virtualenv(platform, &cache)?;
|
||||
venv.python_executable().to_path_buf()
|
||||
};
|
||||
|
||||
let files = uv_installer::compile_tree(
|
||||
&fs_err::canonicalize(args.root)?,
|
||||
&interpreter,
|
||||
cache.root(),
|
||||
)
|
||||
.await?;
|
||||
info!("Compiled {files} files");
|
||||
Ok(())
|
||||
}
|
|
@ -19,6 +19,8 @@ use tracing_subscriber::EnvFilter;
|
|||
use resolve_many::ResolveManyArgs;
|
||||
|
||||
use crate::build::{build, BuildArgs};
|
||||
use crate::clear_compile::ClearCompileArgs;
|
||||
use crate::compile::CompileArgs;
|
||||
use crate::install_many::InstallManyArgs;
|
||||
use crate::render_benchmarks::RenderBenchmarksArgs;
|
||||
use crate::resolve_cli::ResolveCliArgs;
|
||||
|
@ -41,6 +43,8 @@ static GLOBAL: mimalloc::MiMalloc = mimalloc::MiMalloc;
|
|||
static GLOBAL: tikv_jemallocator::Jemalloc = tikv_jemallocator::Jemalloc;
|
||||
|
||||
mod build;
|
||||
mod clear_compile;
|
||||
mod compile;
|
||||
mod install_many;
|
||||
mod render_benchmarks;
|
||||
mod resolve_cli;
|
||||
|
@ -67,6 +71,10 @@ enum Cli {
|
|||
Resolve(ResolveCliArgs),
|
||||
WheelMetadata(WheelMetadataArgs),
|
||||
RenderBenchmarks(RenderBenchmarksArgs),
|
||||
/// Compile all `.py` to `.pyc` files in the tree.
|
||||
Compile(CompileArgs),
|
||||
/// Remove all `.pyc` in the tree.
|
||||
ClearCompile(ClearCompileArgs),
|
||||
}
|
||||
|
||||
#[instrument] // Anchor span to check for overhead
|
||||
|
@ -88,6 +96,8 @@ async fn run() -> Result<()> {
|
|||
}
|
||||
Cli::WheelMetadata(args) => wheel_metadata::wheel_metadata(args).await?,
|
||||
Cli::RenderBenchmarks(args) => render_benchmarks::render_benchmarks(&args)?,
|
||||
Cli::Compile(args) => compile::compile(args).await?,
|
||||
Cli::ClearCompile(args) => clear_compile::clear_compile(&args)?,
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
|
|
@ -31,8 +31,10 @@ uv-git = { path = "../uv-git", features = ["vendored-openssl"] }
|
|||
uv-interpreter = { path = "../uv-interpreter" }
|
||||
uv-normalize = { path = "../uv-normalize" }
|
||||
uv-traits = { path = "../uv-traits" }
|
||||
uv-warnings = { path = "../uv-warnings" }
|
||||
|
||||
anyhow = { workspace = true }
|
||||
async-channel = { workspace = true }
|
||||
fs-err = { workspace = true }
|
||||
futures = { workspace = true }
|
||||
pyproject-toml = { workspace = true }
|
||||
|
@ -45,3 +47,4 @@ tokio = { workspace = true }
|
|||
toml = { workspace = true }
|
||||
tracing = { workspace = true }
|
||||
url = { workspace = true }
|
||||
walkdir = { workspace = true }
|
||||
|
|
284
crates/uv-installer/src/compile.rs
Normal file
284
crates/uv-installer/src/compile.rs
Normal file
|
@ -0,0 +1,284 @@
|
|||
use std::io;
|
||||
use std::num::NonZeroUsize;
|
||||
use std::path::{Path, PathBuf};
|
||||
use std::process::Stdio;
|
||||
use std::time::Duration;
|
||||
|
||||
use async_channel::{Receiver, SendError};
|
||||
use tempfile::tempdir_in;
|
||||
use thiserror::Error;
|
||||
use tokio::io::{AsyncBufReadExt, AsyncReadExt, AsyncWriteExt, BufReader};
|
||||
use tokio::process::{ChildStdin, ChildStdout, Command};
|
||||
use tokio::task::JoinError;
|
||||
use tracing::{debug, instrument};
|
||||
use walkdir::WalkDir;
|
||||
|
||||
use uv_fs::Simplified;
|
||||
use uv_warnings::warn_user;
|
||||
|
||||
const COMPILEALL_SCRIPT: &str = include_str!("pip_compileall.py");
|
||||
const MAIN_TIMEOUT: Duration = Duration::from_secs(10);
|
||||
|
||||
#[derive(Debug, Error)]
|
||||
pub enum CompileError {
|
||||
#[error("Failed to list files in `site-packages`")]
|
||||
Walkdir(#[from] walkdir::Error),
|
||||
#[error("Failed to send task to worker")]
|
||||
WorkerDisappeared(SendError<PathBuf>),
|
||||
#[error("The task executor is broken, did some other task panic?")]
|
||||
Join(#[from] JoinError),
|
||||
#[error("Failed to start Python interpreter to run compile script")]
|
||||
PythonSubcommand(#[source] io::Error),
|
||||
#[error("Failed to create temporary script file")]
|
||||
TempFile(#[source] io::Error),
|
||||
#[error("Bytecode compilation failed, expected {0:?}, received: {1:?}")]
|
||||
WrongPath(String, String),
|
||||
#[error("Failed to write to Python {device}")]
|
||||
ChildStdio {
|
||||
device: &'static str,
|
||||
#[source]
|
||||
err: io::Error,
|
||||
},
|
||||
#[error("Python process stderr:\n{stderr}")]
|
||||
ErrorWithStderr {
|
||||
stderr: String,
|
||||
#[source]
|
||||
err: Box<Self>,
|
||||
},
|
||||
#[error("Bytecode timed out ({}s)", _0.as_secs_f32())]
|
||||
Timeout(Duration),
|
||||
}
|
||||
|
||||
/// Bytecode compile all file in `dir` using a pool of work-stealing Python interpreters running a
|
||||
/// Python script that calls `compileall.compile_file`.
|
||||
///
|
||||
/// All compilation errors are muted (like pip). There is a 10s timeout to handle the case that
|
||||
/// the workers have gotten stuck. This happens way too easily with channels and subprocesses, e.g.,
|
||||
/// because some pipe is full, we're waiting when it's buffered, or we didn't handle channel closing
|
||||
/// properly.
|
||||
///
|
||||
/// We only compile all files, but we don't update the RECORD, relying on PEP 491:
|
||||
/// > Uninstallers should be smart enough to remove .pyc even if it is not mentioned in RECORD.
|
||||
///
|
||||
/// We've confirmed that both `uv` and `pip` (as of 24.0.0) remove the `__pycache__` directory.
|
||||
#[instrument(skip(python_executable))]
|
||||
pub async fn compile_tree(
|
||||
dir: &Path,
|
||||
python_executable: &Path,
|
||||
cache: &Path,
|
||||
) -> Result<usize, CompileError> {
|
||||
debug_assert!(
|
||||
dir.is_absolute(),
|
||||
"compileall doesn't work with relative paths"
|
||||
);
|
||||
let worker_count = std::thread::available_parallelism().unwrap_or_else(|err| {
|
||||
warn_user!("Couldn't determine number of cores, compiling with a single thread: {err}");
|
||||
NonZeroUsize::MIN
|
||||
});
|
||||
|
||||
let (sender, receiver) = async_channel::bounded::<PathBuf>(worker_count.get() * 10);
|
||||
|
||||
// Running Python with an actual file will produce better error messages..
|
||||
let tempdir = tempdir_in(cache).map_err(CompileError::TempFile)?;
|
||||
let pip_compileall_py = tempdir.path().join("pip_compileall.py");
|
||||
|
||||
// Start the workers.
|
||||
let mut worker_handles = Vec::new();
|
||||
for _ in 0..worker_count.get() {
|
||||
worker_handles.push(tokio::task::spawn(worker(
|
||||
dir.to_path_buf(),
|
||||
python_executable.to_path_buf(),
|
||||
pip_compileall_py.clone(),
|
||||
receiver.clone(),
|
||||
)));
|
||||
}
|
||||
|
||||
// Start the producer, sending all `.py` files to workers.
|
||||
let mut source_files = 0;
|
||||
let mut send_error = None;
|
||||
let walker = WalkDir::new(dir)
|
||||
.into_iter()
|
||||
// Otherwise we stumble over temporary files from `compileall`.
|
||||
.filter_entry(|dir| dir.file_name() != "__pycache__");
|
||||
for entry in walker {
|
||||
let entry = entry?;
|
||||
// https://github.com/pypa/pip/blob/3820b0e52c7fed2b2c43ba731b718f316e6816d1/src/pip/_internal/operations/install/wheel.py#L593-L604
|
||||
if entry.metadata()?.is_file() && entry.path().extension().is_some_and(|ext| ext == "py") {
|
||||
source_files += 1;
|
||||
match tokio::time::timeout(MAIN_TIMEOUT, sender.send(entry.path().to_owned())).await {
|
||||
// The workers are stuck.
|
||||
// If we hit this condition, none of the workers made progress in the last 10s.
|
||||
// For reference, on my desktop compiling a venv with "jupyter plotly django" while
|
||||
// a cargo build was also running the slowest file took 100ms.
|
||||
Err(_) => return Err(CompileError::Timeout(MAIN_TIMEOUT)),
|
||||
// The workers exited.
|
||||
// If e.g. something with the Python interpreter is wrong, the workers have exited
|
||||
// with an error. We try to report this informative error and only if that fails,
|
||||
// report the send error.
|
||||
Ok(Err(err)) => send_error = Some(err),
|
||||
Ok(Ok(())) => {}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// All workers will receive an error after the last item. Note that there are still
|
||||
// up to worker_count * 10 items in the queue.
|
||||
drop(sender);
|
||||
|
||||
// Make sure all workers exit regularly, avoid hiding errors.
|
||||
let results =
|
||||
match tokio::time::timeout(MAIN_TIMEOUT, futures::future::join_all(worker_handles)).await {
|
||||
Err(_) => {
|
||||
// If this happens, we waited more than 10s for n * 10 files on n workers. It
|
||||
// could happen when the user's io is bad (e.g. an overloaded misconfigured
|
||||
// network storage), there are unreasonably sized source files, but more likely
|
||||
// there is a bug in uv (including strange python configurations we need to work
|
||||
// around), which we'd like to know.
|
||||
return Err(CompileError::Timeout(MAIN_TIMEOUT));
|
||||
}
|
||||
Ok(results) => results,
|
||||
};
|
||||
for result in results {
|
||||
match result {
|
||||
// There spawning earlier errored due to a panic in a task.
|
||||
Err(join_err) => return Err(CompileError::Join(join_err)),
|
||||
// The worker reports an error.
|
||||
Ok(Err(compile_error)) => return Err(compile_error),
|
||||
Ok(Ok(())) => {}
|
||||
}
|
||||
}
|
||||
|
||||
if let Some(send_error) = send_error {
|
||||
// This is suspicious: Why did the channel stop working, but all workers exited
|
||||
// successfully?
|
||||
return Err(CompileError::WorkerDisappeared(send_error));
|
||||
}
|
||||
|
||||
Ok(source_files)
|
||||
}
|
||||
|
||||
async fn worker(
|
||||
dir: PathBuf,
|
||||
interpreter: PathBuf,
|
||||
pip_compileall_py: PathBuf,
|
||||
receiver: Receiver<PathBuf>,
|
||||
) -> Result<(), CompileError> {
|
||||
fs_err::tokio::write(&pip_compileall_py, COMPILEALL_SCRIPT)
|
||||
.await
|
||||
.map_err(CompileError::TempFile)?;
|
||||
// We input the paths through stdin and get the successful paths returned through stdout.
|
||||
let mut bytecode_compiler = Command::new(&interpreter)
|
||||
.arg(&pip_compileall_py)
|
||||
.stdin(Stdio::piped())
|
||||
.stdout(Stdio::piped())
|
||||
.stderr(Stdio::piped())
|
||||
.current_dir(dir)
|
||||
// Otherwise stdout is buffered and we'll wait forever for a response
|
||||
.env("PYTHONUNBUFFERED", "1")
|
||||
.spawn()
|
||||
.map_err(CompileError::PythonSubcommand)?;
|
||||
|
||||
// https://stackoverflow.com/questions/49218599/write-to-child-process-stdin-in-rust/49597789#comment120223107_49597789
|
||||
// Unbuffered, we need to write immediately or the python process will get stuck waiting
|
||||
let child_stdin = bytecode_compiler
|
||||
.stdin
|
||||
.take()
|
||||
.expect("Child must have stdin");
|
||||
let mut child_stdout = BufReader::new(
|
||||
bytecode_compiler
|
||||
.stdout
|
||||
.take()
|
||||
.expect("Child must have stdout"),
|
||||
);
|
||||
let mut child_stderr = BufReader::new(
|
||||
bytecode_compiler
|
||||
.stderr
|
||||
.take()
|
||||
.expect("Child must have stderr"),
|
||||
);
|
||||
|
||||
let stderr_reader = tokio::task::spawn(async move {
|
||||
let mut child_stderr_collected: Vec<u8> = Vec::new();
|
||||
child_stderr
|
||||
.read_to_end(&mut child_stderr_collected)
|
||||
.await?;
|
||||
Ok(child_stderr_collected)
|
||||
});
|
||||
|
||||
let result = worker_main_loop(receiver, child_stdin, &mut child_stdout).await;
|
||||
// Reap the process to avoid zombies.
|
||||
let _ = bytecode_compiler.kill().await;
|
||||
|
||||
// If there was something printed to stderr (which shouldn't happen, we muted all errors), tell
|
||||
// the user, otherwise only forward the result.
|
||||
let child_stderr_collected = stderr_reader
|
||||
.await?
|
||||
.map_err(|err| CompileError::ChildStdio {
|
||||
device: "stderr",
|
||||
err,
|
||||
})?;
|
||||
if !child_stderr_collected.is_empty() {
|
||||
let stderr = String::from_utf8_lossy(&child_stderr_collected);
|
||||
return match result {
|
||||
Ok(()) => {
|
||||
debug!(
|
||||
"Bytecode compilation `python` at {} stderr:\n{}\n---",
|
||||
interpreter.simplified_display(),
|
||||
stderr
|
||||
);
|
||||
Ok(())
|
||||
}
|
||||
Err(err) => Err(CompileError::ErrorWithStderr {
|
||||
stderr: stderr.to_string(),
|
||||
err: Box::new(err),
|
||||
}),
|
||||
};
|
||||
}
|
||||
|
||||
result
|
||||
}
|
||||
|
||||
/// We use stdin/stdout as a sort of bounded channel. We write one path to stdin, then wait until
|
||||
/// we get the same path back from stdout. This way we ensure one worker is only working on one
|
||||
/// piece of work at the same time.
|
||||
async fn worker_main_loop(
|
||||
receiver: Receiver<PathBuf>,
|
||||
mut child_stdin: ChildStdin,
|
||||
child_stdout: &mut BufReader<ChildStdout>,
|
||||
) -> Result<(), CompileError> {
|
||||
let mut out_line = String::new();
|
||||
while let Ok(source_file) = receiver.recv().await {
|
||||
let source_file = source_file.display().to_string();
|
||||
if source_file.contains(['\r', '\n']) {
|
||||
warn_user!("Path contains newline, skipping: {source_file:?}");
|
||||
continue;
|
||||
}
|
||||
// Luckily, LF alone works on windows too
|
||||
let bytes = format!("{source_file}\n").into_bytes();
|
||||
|
||||
child_stdin
|
||||
.write_all(&bytes)
|
||||
.await
|
||||
.map_err(|err| CompileError::ChildStdio {
|
||||
device: "stdin",
|
||||
err,
|
||||
})?;
|
||||
|
||||
out_line.clear();
|
||||
child_stdout
|
||||
.read_line(&mut out_line)
|
||||
.await
|
||||
.map_err(|err| CompileError::ChildStdio {
|
||||
device: "stdout",
|
||||
err,
|
||||
})?;
|
||||
|
||||
// This is a sanity check, if we don't get the path back something has gone wrong, e.g.
|
||||
// we're not actually running a python interpreter.
|
||||
let actual = out_line.trim_end_matches(['\n', '\r']);
|
||||
if actual != source_file {
|
||||
return Err(CompileError::WrongPath(source_file, actual.to_string()));
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
|
@ -1,3 +1,4 @@
|
|||
pub use compile::{compile_tree, CompileError};
|
||||
pub use downloader::{Downloader, Reporter as DownloadReporter};
|
||||
pub use editable::{is_dynamic, BuiltEditable, ResolvedEditable};
|
||||
pub use installer::{Installer, Reporter as InstallReporter};
|
||||
|
@ -6,6 +7,7 @@ pub use site_packages::SitePackages;
|
|||
pub use uninstall::uninstall;
|
||||
pub use uv_traits::NoBinary;
|
||||
|
||||
mod compile;
|
||||
mod downloader;
|
||||
mod editable;
|
||||
mod installer;
|
||||
|
|
|
@ -3,9 +3,9 @@ Based on
|
|||
https://github.com/pypa/pip/blob/3820b0e52c7fed2b2c43ba731b718f316e6816d1/src/pip/_internal/operations/install/wheel.py#L612-L623
|
||||
|
||||
pip silently just swallows all pyc compilation errors, but `python -m compileall` does
|
||||
not have such a flag, so we adapt the pip code. This is relevant e.g. for
|
||||
not have such a flag, so we adapt the pip code. This is relevant, e.g., for
|
||||
`debugpy-1.5.1-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64`,
|
||||
which contains some vendored python 2 code which fails to compile
|
||||
which contains some vendored Python 2 code which fails to compile.
|
||||
"""
|
||||
|
||||
import compileall
|
||||
|
@ -14,18 +14,15 @@ import warnings
|
|||
|
||||
with warnings.catch_warnings():
|
||||
warnings.filterwarnings("ignore")
|
||||
# in rust, we give one line per file to compile
|
||||
# we also have to read it before printing to stdout or we risk pipes running full
|
||||
paths = sys.stdin.readlines()
|
||||
for path in paths:
|
||||
# just to be sure
|
||||
# In rust, we provide one line per file to compile.
|
||||
for path in sys.stdin:
|
||||
# Remove trailing newlines.
|
||||
path = path.strip()
|
||||
if not path:
|
||||
continue
|
||||
# Unlike pip, we set quiet=2, so we don't have to capture stdout
|
||||
# I'd like to show those errors, but given that pip thinks that's totally fine
|
||||
# we can't really change that
|
||||
# Unlike pip, we set quiet=2, so we don't have to capture stdout.
|
||||
# We'd like to show those errors, but given that pip thinks that's totally fine,
|
||||
# we can't really change that.
|
||||
success = compileall.compile_file(path, force=True, quiet=2)
|
||||
if success:
|
||||
# return successfully compiled files so we can update RECORD accordingly
|
||||
print(path)
|
||||
# We're ready for the next file.
|
||||
print(path)
|
|
@ -45,7 +45,7 @@ chrono = { workspace = true }
|
|||
clap = { workspace = true, features = ["derive", "string"] }
|
||||
clap_complete_command = { workspace = true }
|
||||
console = { workspace = true }
|
||||
ctrlc = { workspace = true }
|
||||
ctrlc = { workspace = true }
|
||||
dunce = { workspace = true }
|
||||
flate2 = { workspace = true, default-features = false }
|
||||
fs-err = { workspace = true, features = ["tokio"] }
|
||||
|
|
|
@ -1,6 +1,10 @@
|
|||
use std::fmt::Write;
|
||||
use std::process::ExitCode;
|
||||
use std::time::Duration;
|
||||
|
||||
use anyhow::Context;
|
||||
use owo_colors::OwoColorize;
|
||||
|
||||
pub(crate) use cache_clean::cache_clean;
|
||||
pub(crate) use cache_dir::cache_dir;
|
||||
use distribution_types::InstalledMetadata;
|
||||
|
@ -10,9 +14,15 @@ pub(crate) use pip_install::pip_install;
|
|||
pub(crate) use pip_list::pip_list;
|
||||
pub(crate) use pip_sync::pip_sync;
|
||||
pub(crate) use pip_uninstall::pip_uninstall;
|
||||
use uv_cache::Cache;
|
||||
use uv_fs::Simplified;
|
||||
use uv_installer::compile_tree;
|
||||
use uv_interpreter::PythonEnvironment;
|
||||
pub(crate) use venv::venv;
|
||||
pub(crate) use version::version;
|
||||
|
||||
use crate::printer::Printer;
|
||||
|
||||
mod cache_clean;
|
||||
mod cache_dir;
|
||||
mod pip_compile;
|
||||
|
@ -82,3 +92,35 @@ pub(crate) enum VersionFormat {
|
|||
Text,
|
||||
Json,
|
||||
}
|
||||
|
||||
/// Compile all Python source files in site-packages to bytecode, to speed up the
|
||||
/// initial run of any subsequent executions.
|
||||
///
|
||||
/// See the `--compile` option on `pip sync` and `pip install`.
|
||||
pub(super) async fn compile_bytecode(
|
||||
venv: &PythonEnvironment,
|
||||
cache: &Cache,
|
||||
mut printer: Printer,
|
||||
) -> anyhow::Result<()> {
|
||||
let start = std::time::Instant::now();
|
||||
let files = compile_tree(venv.site_packages(), venv.python_executable(), cache.root())
|
||||
.await
|
||||
.with_context(|| {
|
||||
format!(
|
||||
"Failed to bytecode compile {}",
|
||||
venv.site_packages().simplified_display()
|
||||
)
|
||||
})?;
|
||||
let s = if files == 1 { "" } else { "s" };
|
||||
writeln!(
|
||||
printer,
|
||||
"{}",
|
||||
format!(
|
||||
"Bytecode compiled {} in {}",
|
||||
format!("{files} file{s}").bold(),
|
||||
elapsed(start.elapsed())
|
||||
)
|
||||
.dimmed()
|
||||
)?;
|
||||
Ok(())
|
||||
}
|
||||
|
|
|
@ -35,7 +35,7 @@ use uv_resolver::{
|
|||
use uv_traits::{ConfigSettings, InFlight, NoBuild, SetupPyStrategy};
|
||||
|
||||
use crate::commands::reporters::{DownloadReporter, InstallReporter, ResolverReporter};
|
||||
use crate::commands::{elapsed, ChangeEvent, ChangeEventKind, ExitStatus};
|
||||
use crate::commands::{compile_bytecode, elapsed, ChangeEvent, ChangeEventKind, ExitStatus};
|
||||
use crate::printer::Printer;
|
||||
use crate::requirements::{ExtrasSpecification, RequirementsSource, RequirementsSpecification};
|
||||
|
||||
|
@ -55,6 +55,7 @@ pub(crate) async fn pip_install(
|
|||
index_locations: IndexLocations,
|
||||
reinstall: &Reinstall,
|
||||
link_mode: LinkMode,
|
||||
compile: bool,
|
||||
setup_py: SetupPyStrategy,
|
||||
connectivity: Connectivity,
|
||||
config_settings: &ConfigSettings,
|
||||
|
@ -293,6 +294,7 @@ pub(crate) async fn pip_install(
|
|||
reinstall,
|
||||
no_binary,
|
||||
link_mode,
|
||||
compile,
|
||||
&index_locations,
|
||||
tags,
|
||||
&client,
|
||||
|
@ -506,6 +508,7 @@ async fn install(
|
|||
reinstall: &Reinstall,
|
||||
no_binary: &NoBinary,
|
||||
link_mode: LinkMode,
|
||||
compile: bool,
|
||||
index_urls: &IndexLocations,
|
||||
tags: &Tags,
|
||||
client: &RegistryClient,
|
||||
|
@ -639,6 +642,10 @@ async fn install(
|
|||
)?;
|
||||
}
|
||||
|
||||
if compile {
|
||||
compile_bytecode(venv, cache, printer).await?;
|
||||
}
|
||||
|
||||
for event in reinstalls
|
||||
.into_iter()
|
||||
.map(|distribution| ChangeEvent {
|
||||
|
|
|
@ -23,7 +23,7 @@ use uv_resolver::InMemoryIndex;
|
|||
use uv_traits::{ConfigSettings, InFlight, NoBuild, SetupPyStrategy};
|
||||
|
||||
use crate::commands::reporters::{DownloadReporter, FinderReporter, InstallReporter};
|
||||
use crate::commands::{elapsed, ChangeEvent, ChangeEventKind, ExitStatus};
|
||||
use crate::commands::{compile_bytecode, elapsed, ChangeEvent, ChangeEventKind, ExitStatus};
|
||||
use crate::printer::Printer;
|
||||
use crate::requirements::{RequirementsSource, RequirementsSpecification};
|
||||
|
||||
|
@ -33,6 +33,7 @@ pub(crate) async fn pip_sync(
|
|||
sources: &[RequirementsSource],
|
||||
reinstall: &Reinstall,
|
||||
link_mode: LinkMode,
|
||||
compile: bool,
|
||||
index_locations: IndexLocations,
|
||||
setup_py: SetupPyStrategy,
|
||||
connectivity: Connectivity,
|
||||
|
@ -306,6 +307,10 @@ pub(crate) async fn pip_sync(
|
|||
)?;
|
||||
}
|
||||
|
||||
if compile {
|
||||
compile_bytecode(&venv, &cache, printer).await?;
|
||||
}
|
||||
|
||||
// Report on any changes in the environment.
|
||||
for event in extraneous
|
||||
.into_iter()
|
||||
|
|
|
@ -563,6 +563,18 @@ struct PipSyncArgs {
|
|||
#[clap(long, conflicts_with = "no_build")]
|
||||
only_binary: Vec<PackageNameSpecifier>,
|
||||
|
||||
/// Compile Python files to bytecode.
|
||||
///
|
||||
/// By default, does not compile Python (`.py`) files to bytecode (`__pycache__/*.pyc`), instead
|
||||
/// Python lazily does the compilation the first time a module is imported. In cases where the
|
||||
/// first start time matters, such as CLI applications and docker containers, this option can
|
||||
/// trade longer install time for faster startup.
|
||||
///
|
||||
/// The compile option will process the entire site-packages directory for consistency and
|
||||
/// (like pip) ignore all errors.
|
||||
#[clap(long)]
|
||||
compile: bool,
|
||||
|
||||
/// Settings to pass to the PEP 517 build backend, specified as `KEY=VALUE` pairs.
|
||||
#[clap(long, short = 'C', alias = "config-settings")]
|
||||
config_setting: Vec<ConfigSettingEntry>,
|
||||
|
@ -776,6 +788,18 @@ struct PipInstallArgs {
|
|||
#[clap(long, conflicts_with = "no_build")]
|
||||
only_binary: Vec<PackageNameSpecifier>,
|
||||
|
||||
/// Compile Python files to bytecode.
|
||||
///
|
||||
/// By default, does not compile Python (`.py`) files to bytecode (`__pycache__/*.pyc`), instead
|
||||
/// Python lazily does the compilation the first time a module is imported. In cases where the
|
||||
/// first start time matters, such as CLI applications and docker containers, this option can
|
||||
/// trade longer install time for faster startup.
|
||||
///
|
||||
/// The compile option will process the entire site-packages directory for consistency and
|
||||
/// (like pip) ignore all errors.
|
||||
#[clap(long)]
|
||||
compile: bool,
|
||||
|
||||
/// Settings to pass to the PEP 517 build backend, specified as `KEY=VALUE` pairs.
|
||||
#[clap(long, short = 'C', alias = "config-settings")]
|
||||
config_setting: Vec<ConfigSettingEntry>,
|
||||
|
@ -1254,6 +1278,7 @@ async fn run() -> Result<ExitStatus> {
|
|||
&sources,
|
||||
&reinstall,
|
||||
args.link_mode,
|
||||
args.compile,
|
||||
index_urls,
|
||||
setup_py,
|
||||
if args.offline {
|
||||
|
@ -1346,6 +1371,7 @@ async fn run() -> Result<ExitStatus> {
|
|||
index_urls,
|
||||
&reinstall,
|
||||
args.link_mode,
|
||||
args.compile,
|
||||
setup_py,
|
||||
if args.offline {
|
||||
Connectivity::Offline
|
||||
|
|
|
@ -45,6 +45,7 @@ pub struct TestContext {
|
|||
pub temp_dir: assert_fs::TempDir,
|
||||
pub cache_dir: assert_fs::TempDir,
|
||||
pub venv: PathBuf,
|
||||
pub python_version: String,
|
||||
|
||||
// Standard filters for this test context
|
||||
filters: Vec<(String, String)>,
|
||||
|
@ -77,6 +78,7 @@ impl TestContext {
|
|||
temp_dir,
|
||||
cache_dir,
|
||||
venv,
|
||||
python_version: python_version.to_string(),
|
||||
filters,
|
||||
}
|
||||
}
|
||||
|
@ -166,6 +168,12 @@ impl TestContext {
|
|||
.chain(INSTA_FILTERS.iter().copied())
|
||||
.collect()
|
||||
}
|
||||
|
||||
/// For when we add pypy to the test suite.
|
||||
#[allow(clippy::unused_self)]
|
||||
pub fn python_kind(&self) -> &str {
|
||||
"python"
|
||||
}
|
||||
}
|
||||
|
||||
pub fn venv_to_interpreter(venv: &Path) -> PathBuf {
|
||||
|
|
|
@ -2,7 +2,7 @@
|
|||
|
||||
use fs_err as fs;
|
||||
use std::env::consts::EXE_SUFFIX;
|
||||
use std::path::Path;
|
||||
use std::path::{Path, PathBuf};
|
||||
use std::process::Command;
|
||||
|
||||
use anyhow::Result;
|
||||
|
@ -73,6 +73,25 @@ fn uninstall_command(context: &TestContext) -> Command {
|
|||
command
|
||||
}
|
||||
|
||||
/// Returns the site-packages folder inside the venv.
|
||||
fn site_packages(context: &TestContext) -> PathBuf {
|
||||
if cfg!(unix) {
|
||||
context
|
||||
.venv
|
||||
.join("lib")
|
||||
.join(format!(
|
||||
"{}{}",
|
||||
context.python_kind(),
|
||||
context.python_version
|
||||
))
|
||||
.join("site-packages")
|
||||
} else if cfg!(windows) {
|
||||
context.venv.join("Lib").join("site-packages")
|
||||
} else {
|
||||
unimplemented!("Only Windows and Unix are supported")
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn missing_pip() {
|
||||
uv_snapshot!(Command::new(get_bin()).arg("sync"), @r###"
|
||||
|
@ -166,6 +185,13 @@ fn install() -> Result<()> {
|
|||
"###
|
||||
);
|
||||
|
||||
// Counterpart for the `compile()` test.
|
||||
assert!(!site_packages(&context)
|
||||
.join("markupsafe")
|
||||
.join("__pycache__")
|
||||
.join("__init__.cpython-312.pyc")
|
||||
.exists());
|
||||
|
||||
context.assert_command("import markupsafe").success();
|
||||
|
||||
// Removing the cache shouldn't invalidate the virtual environment.
|
||||
|
@ -2893,3 +2919,40 @@ requires-python = ">=3.8"
|
|||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Install with bytecode compilation.
|
||||
#[test]
|
||||
fn compile() -> Result<()> {
|
||||
let context = TestContext::new("3.12");
|
||||
|
||||
let requirements_txt = context.temp_dir.child("requirements.txt");
|
||||
requirements_txt.touch()?;
|
||||
requirements_txt.write_str("MarkupSafe==2.1.3")?;
|
||||
|
||||
uv_snapshot!(command(&context)
|
||||
.arg("requirements.txt")
|
||||
.arg("--compile")
|
||||
.arg("--strict"), @r###"
|
||||
success: true
|
||||
exit_code: 0
|
||||
----- stdout -----
|
||||
|
||||
----- stderr -----
|
||||
Resolved 1 package in [TIME]
|
||||
Downloaded 1 package in [TIME]
|
||||
Installed 1 package in [TIME]
|
||||
Bytecode compiled 3 files in [TIME]
|
||||
+ markupsafe==2.1.3
|
||||
"###
|
||||
);
|
||||
|
||||
assert!(site_packages(&context)
|
||||
.join("markupsafe")
|
||||
.join("__pycache__")
|
||||
.join("__init__.cpython-312.pyc")
|
||||
.exists());
|
||||
|
||||
context.assert_command("import markupsafe").success();
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue