Parse less JSON on null builds

This commit fixes a performance pathology in Cargo today. Whenever Cargo
generates a lock file (which happens on all invocations of `cargo build`
for example) Cargo will parse the crates.io index to learn about
dependencies. Currently, however, when it parses a crate it parses the
JSON blob for every single version of the crate. With a lock file,
however, or with incremental builds only one of these lines of JSON is
relevant. Measured today Cargo building Cargo parses 3700 JSON
dependencies in the registry.

This commit implements an optimization that brings down the number of
parsed JSON lines in the registry to precisely the right number
necessary to build a project. For example Cargo has 150 crates in its
lock file, so now it only parses 150 JSON lines (a 20x reduction from
3700). This in turn can greatly improve Cargo's null build time. Cargo
building Cargo dropped from 120ms to 60ms on a Linux machine and 400ms
to 200ms on a Mac.

The commit internally has a lot more details about how this is done but
the general idea is to have a cache which is optimized for Cargo to read
which is maintained automatically by Cargo.

Closes #6866
This commit is contained in:
Alex Crichton 2019-04-25 14:21:27 -07:00
parent c7e1b68918
commit 6babe72e7c
5 changed files with 523 additions and 136 deletions

View file

@ -44,6 +44,7 @@ lazycell = "1.2.0"
libc = "0.2"
log = "0.4.6"
libgit2-sys = "0.7.9"
memchr = "2.1.3"
num_cpus = "1.0"
opener = "0.3.0"
rustfix = "0.4.4"

View file

@ -14,6 +14,7 @@ pub use self::shell::{Shell, Verbosity};
pub use self::source::{GitReference, Source, SourceId, SourceMap};
pub use self::summary::{FeatureMap, FeatureValue, Summary};
pub use self::workspace::{Members, Workspace, WorkspaceConfig, WorkspaceRootConfig};
pub use self::interning::InternedString;
pub mod compiler;
pub mod dependency;

View file

@ -1,12 +1,83 @@
//! Management of the index of a registry source
//!
//! This module contains management of the index and various operations, such as
//! actually parsing the index, looking for crates, etc. This is intended to be
//! abstract over remote indices (downloaded via git) and local registry indices
//! (which are all just present on the filesystem).
//!
//! ## Index Performance
//!
//! One important aspect of the index is that we want to optimize the "happy
//! path" as much as possible. Whenever you type `cargo build` Cargo will
//! *always* reparse the registry and learn about dependency information. This
//! is done because Cargo needs to learn about the upstream crates.io crates
//! that you're using and ensure that the preexisting `Cargo.lock` still matches
//! the current state of the world.
//!
//! Consequently, Cargo "null builds" (the index that Cargo adds to each build
//! itself) need to be fast when accessing the index. The primary performance
//! optimization here is to avoid parsing JSON blobs from the registry if we
//! don't need them. Most secondary optimizations are centered around removing
//! allocations and such, but avoiding parsing JSON is the #1 optimization.
//!
//! When we get queries from the resolver we're given a `Dependency`. This
//! dependency in turn has a version requirement, and with lock files that
//! already exist these version requirements are exact version requirements
//! `=a.b.c`. This means that we in theory only need to parse one line of JSON
//! per query in the registry, the one that matches version `a.b.c`.
//!
//! The crates.io index, however, is not amenable to this form of query. Instead
//! the crates.io index simply is a file where each line is a JSON blob. To
//! learn about the versions in each JSON blob we would need to parse the JSON,
//! defeating the purpose of trying to parse as little as possible.
//!
//! > Note that as a small aside even *loading* the JSON from the registry is
//! > actually pretty slow. For crates.io and remote registries we don't
//! > actually check out the git index on disk because that takes quite some
//! > time and is quite large. Instead we use `libgit2` to read the JSON from
//! > the raw git objects. This in turn can be slow (aka show up high in
//! > profiles) because libgit2 has to do deflate decompression and such.
//!
//! To solve all these issues a strategy is employed here where Cargo basically
//! creates an index into the index. The first time a package is queried about
//! (first time being for an entire computer) Cargo will load the contents
//! (slowly via libgit2) from the registry. It will then (slowly) parse every
//! single line to learn about its versions. Afterwards, however, Cargo will
//! emit a new file (a cache) which is amenable for speedily parsing in future
//! invocations.
//!
//! This cache file is currently organized by basically having the semver
//! version extracted from each JSON blob. That way Cargo can quickly and easily
//! parse all versions contained and which JSON blob they're associated with.
//! The JSON blob then doesn't actually need to get parsed unless the version is
//! parsed.
//!
//! Altogether the initial measurements of this shows a massive improvement for
//! Cargo null build performance. It's expected that the improvements earned
//! here will continue to grow over time in the sense that the previous
//! implementation (parse all lines each time) actually continues to slow down
//! over time as new versions of a crate are published. In any case when first
//! implemented a null build of Cargo itself would parse 3700 JSON blobs from
//! the registry and load 150 blobs from git. Afterwards it parses 150 JSON
//! blobs and loads 0 files git. Removing 200ms or more from Cargo's startup
//! time is certainly nothing to sneeze at!
//!
//! Note that this is just a high-level overview, there's of course lots of
//! details like invalidating caches and whatnot which are handled below, but
//! hopefully those are more obvious inline in the code itself.
use std::collections::{HashMap, HashSet};
use std::fs::{self, File};
use std::io::Read;
use std::path::Path;
use std::str;
use log::{info, trace};
use semver::Version;
use filetime::FileTime;
use log::info;
use semver::{Version, VersionReq};
use crate::core::dependency::Dependency;
use crate::core::{PackageId, SourceId, Summary};
use crate::core::{InternedString, PackageId, SourceId, Summary};
use crate::sources::registry::RegistryData;
use crate::sources::registry::{RegistryPackage, INDEX_LOCK};
use crate::util::{internal, CargoResult, Config, Filesystem, ToSemver};
@ -99,13 +170,68 @@ fn overflow_hyphen() {
pub struct RegistryIndex<'cfg> {
source_id: SourceId,
path: Filesystem,
cache: HashMap<&'static str, Vec<(Summary, bool)>>,
// `(name, vers)` -> `checksum`
hashes: HashMap<&'static str, HashMap<Version, String>>,
summaries_cache: HashMap<InternedString, Summaries>,
config: &'cfg Config,
locked: bool,
}
/// An internal cache of summaries for a particular package.
///
/// A list of summaries are loaded from disk via one of two methods:
///
/// 1. Primarily Cargo will parse the corresponding file for a crate in the
/// upstream crates.io registry. That's just a JSON blob per line which we
/// can parse, extract the version, and then store here.
///
/// 2. Alternatively, if Cargo has previously run, we'll have a cached index of
/// dependencies for the upstream index. This is a file that Cargo maintains
/// lazily on the local filesystem and is much faster to parse since it
/// doesn't involve parsing all of the JSON.
///
/// The outward-facing interface of this doesn't matter too much where it's
/// loaded from, but it's important when reading the implementation to note that
/// we try to parse as little as possible!
#[derive(Default)]
struct Summaries {
/// A raw vector of uninterpreted bytes. This is what `Unparsed` start/end
/// fields are indexes into. If a `Summaries` is loaded from the crates.io
/// index then this field will be empty since nothing is `Unparsed`.
raw_data: Vec<u8>,
/// All known versions of a crate, keyed from their `Version` to the
/// possibly parsed or unparsed version of the full summary.
versions: HashMap<Version, MaybeIndexSummary>,
}
/// A lazily parsed `IndexSummary`.
enum MaybeIndexSummary {
/// A summary which has not been parsed, The `start` and `end` are pointers
/// into `Summaries::raw_data` which this is an entry of.
Unparsed { start: usize, end: usize },
/// An actually parsed summary.
Parsed(IndexSummary),
}
/// A parsed representation of a summary from the index.
///
/// In addition to a full `Summary` we have a few auxiliary pieces of
/// information liked `yanked` and what the checksum hash is.
pub struct IndexSummary {
pub summary: Summary,
pub yanked: bool,
pub hash: String,
}
/// A representation of the cache on disk that Cargo maintains of summaries.
/// Cargo will initially parse all summaries in the registry and will then
/// serialize that into this form and place it in a new location on disk,
/// ensuring that access in the future is much speedier.
#[derive(Default)]
struct SummariesCache<'a> {
versions: Vec<(Version, &'a [u8])>,
}
impl<'cfg> RegistryIndex<'cfg> {
pub fn new(
source_id: SourceId,
@ -116,8 +242,7 @@ impl<'cfg> RegistryIndex<'cfg> {
RegistryIndex {
source_id,
path: path.clone(),
cache: HashMap::new(),
hashes: HashMap::new(),
summaries_cache: HashMap::new(),
config,
locked,
}
@ -125,141 +250,138 @@ impl<'cfg> RegistryIndex<'cfg> {
/// Returns the hash listed for a specified `PackageId`.
pub fn hash(&mut self, pkg: PackageId, load: &mut dyn RegistryData) -> CargoResult<String> {
let name = pkg.name().as_str();
let version = pkg.version();
if let Some(s) = self.hashes.get(name).and_then(|v| v.get(version)) {
return Ok(s.clone());
}
// Ok, we're missing the key, so parse the index file to load it.
self.summaries(name, load)?;
self.hashes
.get(name)
.and_then(|v| v.get(version))
.ok_or_else(|| internal(format!("no hash listed for {}", pkg)))
.map(|s| s.clone())
let req = VersionReq::exact(pkg.version());
let summary = self
.summaries(pkg.name(), &req, load)?
.next()
.ok_or_else(|| internal(format!("no hash listed for {}", pkg)))?;
Ok(summary.hash.clone())
}
/// Parses the on-disk metadata for the package provided.
/// Load a list of summaries for `name` package in this registry which
/// match `req`
///
/// Returns a list of pairs of `(summary, yanked)` for the package name specified.
pub fn summaries(
&mut self,
name: &'static str,
/// This function will semantically parse the on-disk index, match all
/// versions, and then return an iterator over all summaries which matched.
/// Internally there's quite a few layer of caching to amortize this cost
/// though since this method is called quite a lot on null builds in Cargo.
pub fn summaries<'a, 'b>(
&'a mut self,
name: InternedString,
req: &'b VersionReq,
load: &mut dyn RegistryData,
) -> CargoResult<&Vec<(Summary, bool)>> {
if self.cache.contains_key(name) {
return Ok(&self.cache[name]);
}
) -> CargoResult<impl Iterator<Item = &'a IndexSummary> + 'b>
where
'a: 'b,
{
let source_id = self.source_id.clone();
// First up actually parse what summaries we have available. If Cargo
// has run previously this will parse a Cargo-specific cache file rather
// than the registry itself. In effect this is intended to be a quite
// cheap operation.
let summaries = self.load_summaries(name, load)?;
self.cache.insert(name, summaries);
Ok(&self.cache[name])
// Iterate over our summaries, extract all relevant ones which match our
// version requirement, and then parse all corresponding rows in the
// registry. As a reminder this `summaries` method is called for each
// entry in a lock file on every build, so we want to absolutely
// minimize the amount of work being done here and parse as little as
// necessary.
let raw_data = &summaries.raw_data;
Ok(summaries
.versions
.iter_mut()
.filter_map(move |(k, v)| if req.matches(k) { Some(v) } else { None })
.filter_map(move |maybe| match maybe.parse(raw_data, source_id) {
Ok(summary) => Some(summary),
Err(e) => {
info!("failed to parse `{}` registry package: {}", name, e);
None
}
}))
}
fn load_summaries(
&mut self,
name: &str,
name: InternedString,
load: &mut dyn RegistryData,
) -> CargoResult<Vec<(Summary, bool)>> {
) -> CargoResult<&mut Summaries> {
// If we've previously loaded what versions are present for `name`, just
// return that since our cache should still be valid.
if self.summaries_cache.contains_key(&name) {
return Ok(self.summaries_cache.get_mut(&name).unwrap());
}
// Prepare the `RegistryData` which will lazily initialize internal data
// structures. Note that this is also importantly needed to initialize
// to avoid deadlocks where we acquire a lock below but the `load`
// function inside *also* wants to acquire a lock. See an instance of
// this on #5551.
load.prepare()?;
let (root, _lock) = if self.locked {
// Synchronize access to the index. For remote indices we want to make
// sure that while we're reading the index no one is trying to update
// it.
let (root, lock) = if self.locked {
let lock = self
.path
.open_ro(Path::new(INDEX_LOCK), self.config, "the registry index");
match lock {
Ok(lock) => (lock.path().parent().unwrap().to_path_buf(), Some(lock)),
Err(_) => return Ok(Vec::new()),
Err(_) => {
self.summaries_cache.insert(name, Summaries::default());
return Ok(self.summaries_cache.get_mut(&name).unwrap());
}
}
} else {
(self.path.clone().into_path_unlocked(), None)
};
let cache_root = root.join(".cache");
// TODO: comment
let lock_mtime = lock
.as_ref()
.and_then(|l| l.file().metadata().ok())
.map(|t| FileTime::from_last_modification_time(&t));
// See module comment in `registry/mod.rs` for why this is structured
// the way it is.
let fs_name = name
.chars()
.flat_map(|c| c.to_lowercase())
.collect::<String>();
// See module comment for why this is structured the way it is.
let raw_path = match fs_name.len() {
1 => format!("1/{}", fs_name),
2 => format!("2/{}", fs_name),
3 => format!("3/{}/{}", &fs_name[..1], fs_name),
_ => format!("{}/{}/{}", &fs_name[0..2], &fs_name[2..4], fs_name),
};
let mut ret = Vec::new();
// Attempt to handle misspellings by searching for a chain of related
// names to the original `raw_path` name. Only return summaries
// associated with the first hit, however. The resolver will later
// reject any candidates that have the wrong name, and with this it'll
// along the way produce helpful "did you mean?" suggestions.
for path in UncanonicalizedIter::new(&raw_path).take(1024) {
let mut hit_closure = false;
let err = load.load(&root, Path::new(&path), &mut |contents| {
hit_closure = true;
let contents = str::from_utf8(contents)
.map_err(|_| failure::format_err!("registry index file was not valid utf-8"))?;
ret.reserve(contents.lines().count());
let lines = contents.lines().map(|s| s.trim()).filter(|l| !l.is_empty());
// Attempt forwards-compatibility on the index by ignoring
// everything that we ourselves don't understand, that should
// allow future cargo implementations to break the
// interpretation of each line here and older cargo will simply
// ignore the new lines.
ret.extend(lines.filter_map(|line| {
let (summary, locked) = match self.parse_registry_package(line) {
Ok(p) => p,
Err(e) => {
info!("failed to parse `{}` registry package: {}", name, e);
trace!("line: {}", line);
return None;
}
};
Some((summary, locked))
}));
Ok(())
});
// We ignore lookup failures as those are just crates which don't exist
// or we haven't updated the registry yet. If we actually ran the
// closure though then we care about those errors.
if hit_closure {
err?;
// Crates.io ensures that there is only one hyphen and underscore equivalent
// result in the index so return when we find it.
return Ok(ret);
let summaries = Summaries::parse(
lock_mtime,
&root,
&cache_root,
path.as_ref(),
self.source_id,
load,
)?;
if let Some(summaries) = summaries {
self.summaries_cache.insert(name, summaries);
return Ok(self.summaries_cache.get_mut(&name).unwrap());
}
}
Ok(ret)
}
/// Parses a line from the registry's index file into a `Summary` for a package.
///
/// The returned boolean is whether or not the summary has been yanked.
fn parse_registry_package(&mut self, line: &str) -> CargoResult<(Summary, bool)> {
let RegistryPackage {
name,
vers,
cksum,
deps,
features,
yanked,
links,
} = serde_json::from_str(line)?;
let pkgid = PackageId::new(&name, &vers, self.source_id)?;
let name = pkgid.name();
let deps = deps
.into_iter()
.map(|dep| dep.into_dep(self.source_id))
.collect::<CargoResult<Vec<_>>>()?;
let mut summary = Summary::new(pkgid, deps, &features, links, false)?;
summary.set_checksum(cksum.clone());
self.hashes
.entry(name.as_str())
.or_insert_with(HashMap::new)
.insert(vers, cksum);
Ok((summary, yanked.unwrap_or(false)))
// If nothing was found then this crate doesn't exists, so just use an
// empty `Summaries` list.
self.summaries_cache.insert(name, Summaries::default());
Ok(self.summaries_cache.get_mut(&name).unwrap())
}
pub fn query_inner(
@ -295,31 +417,31 @@ impl<'cfg> RegistryIndex<'cfg> {
online: bool,
) -> CargoResult<usize> {
let source_id = self.source_id;
let name = dep.package_name().as_str();
let summaries = self.summaries(name, load)?;
let summaries = summaries
.iter()
.filter(|&(summary, yanked)| {
// Note: This particular logic can cause problems with
// optional dependencies when offline. If at least 1 version
// of an optional dependency is downloaded, but that version
// does not satisfy the requirements, then resolution will
// fail. Unfortunately, whether or not something is optional
// is not known here.
(online || load.is_crate_downloaded(summary.package_id()))
&& (!yanked || {
log::debug!("{:?}", yanked_whitelist);
log::debug!("{:?}", summary.package_id());
yanked_whitelist.contains(&summary.package_id())
})
})
.map(|s| s.0.clone());
let summaries = self
.summaries(dep.package_name(), dep.version_req(), load)?
// First filter summaries for `--offline`. If we're online then
// everything is a candidate, otherwise if we're offline we're only
// going to consider candidates which are actually present on disk.
//
// Note: This particular logic can cause problems with
// optional dependencies when offline. If at least 1 version
// of an optional dependency is downloaded, but that version
// does not satisfy the requirements, then resolution will
// fail. Unfortunately, whether or not something is optional
// is not known here.
.filter(|s| (online || load.is_crate_downloaded(s.summary.package_id())))
// Next filter out all yanked packages. Some yanked packages may
// leak throguh if they're in a whitelist (aka if they were
// previously in `Cargo.lock`
.filter(|s| !s.yanked || yanked_whitelist.contains(&s.summary.package_id()))
.map(|s| s.summary.clone());
// Handle `cargo update --precise` here. If specified, our own source
// will have a precise version listed of the form
// `<pkg>=<p_req>o-><f_req>` where `<pkg>` is the name of a crate on
// this source, `<p_req>` is the version installed and `<f_req> is the
// version requested (argument to `--precise`).
let name = dep.package_name().as_str();
let summaries = summaries.filter(|s| match source_id.precise() {
Some(p) if p.starts_with(name) && p[name.len()..].starts_with('=') => {
let mut vers = p[name.len() + 1..].splitn(2, "->");
@ -344,11 +466,270 @@ impl<'cfg> RegistryIndex<'cfg> {
}
pub fn is_yanked(&mut self, pkg: PackageId, load: &mut dyn RegistryData) -> CargoResult<bool> {
let summaries = self.summaries(pkg.name().as_str(), load)?;
let found = summaries
.iter()
.filter(|&(summary, _yanked)| summary.version() == pkg.version())
.any(|(_summary, yanked)| *yanked);
let req = VersionReq::exact(pkg.version());
let found = self
.summaries(pkg.name(), &req, load)?
.any(|summary| summary.yanked);
Ok(found)
}
}
impl Summaries {
/// Parse out a `Summaries` instances from on-disk state.
///
/// This will attempt to prefer parsing a previous cache file that already
/// exists from a previous invocation of Cargo (aka you're typing `cargo
/// build` again after typing it previously). If parsing fails or the cache
/// isn't found, then we take a slower path which loads the full descriptor
/// for `relative` from the underlying index (aka typically libgit2 with
/// crates.io) and then parse everything in there.
///
/// * `lock_mtime` - this is a file modification time where if any cache
/// file is older than this the cache should be considered out of date and
/// needs to be rebuilt.
/// * `root` - this is the root argument passed to `load`
/// * `cache_root` - this is the root on the filesystem itself of where to
/// store cache files.
/// * `relative` - this is the file we're loading from cache or the index
/// data
/// * `source_id` - the registry's SourceId used when parsing JSON blobs to
/// create summaries.
/// * `load` - the actual index implementation which may be very slow to
/// call. We avoid this if we can.
pub fn parse(
lock_mtime: Option<FileTime>,
root: &Path,
cache_root: &Path,
relative: &Path,
source_id: SourceId,
load: &mut dyn RegistryData,
) -> CargoResult<Option<Summaries>> {
// First up, attempt to load the cache. This could fail for all manner
// of reasons, but consider all of them non-fatal and just log their
// occurrence in case anyone is debugging anything.
let cache_path = cache_root.join(relative);
if let Some(lock_mtime) = lock_mtime {
match File::open(&cache_path) {
Ok(file) => {
let metadata = file.metadata()?;
let cache_mtime = FileTime::from_last_modification_time(&metadata);
if cache_mtime > lock_mtime {
log::debug!("cache for {:?} is fresh", relative);
match Summaries::parse_cache(&file, &metadata) {
Ok(s) => return Ok(Some(s)),
Err(e) => {
log::debug!("failed to parse {:?} cache: {}", relative, e);
}
}
} else {
log::debug!("cache for {:?} is out of date", relative);
}
}
Err(e) => log::debug!("cache for {:?} error: {}", relative, e),
}
}
// This is the fallback path where we actually talk to libgit2 to load
// information. Here we parse every single line in the index (as we need
// to find the versions)
log::debug!("slow path for {:?}", relative);
let mut ret = Summaries::default();
let mut hit_closure = false;
let mut cache_bytes = Vec::new();
let err = load.load(root, relative, &mut |contents| {
ret.raw_data = contents.to_vec();
let mut cache = SummariesCache::default();
hit_closure = true;
let mut start = 0;
for end in memchr::Memchr::new(b'\n', contents) {
// Attempt forwards-compatibility on the index by ignoring
// everything that we ourselves don't understand, that should
// allow future cargo implementations to break the
// interpretation of each line here and older cargo will simply
// ignore the new lines.
let line = &contents[start..end];
let summary = match IndexSummary::parse(line, source_id) {
Ok(summary) => summary,
Err(e) => {
log::info!("failed to parse {:?} registry package: {}", relative, e);
continue;
}
};
let version = summary.summary.package_id().version().clone();
cache.versions.push((version.clone(), line));
ret.versions.insert(version, summary.into());
start = end + 1;
}
cache_bytes = cache.serialize();
Ok(())
});
// We ignore lookup failures as those are just crates which don't exist
// or we haven't updated the registry yet. If we actually ran the
// closure though then we care about those errors.
if !hit_closure {
return Ok(None);
}
err?;
// Once we have our `cache_bytes` which represents the `Summaries` we're
// about to return, write that back out to disk so future Cargo
// invocations can use it.
//
// This is opportunistic so we ignore failure here but are sure to log
// something in case of error.
if fs::create_dir_all(cache_path.parent().unwrap()).is_ok() {
// TODO: somehow need to globally synchronize this
if let Err(e) = fs::write(cache_path, cache_bytes) {
log::info!("failed to write cache: {}", e);
}
}
Ok(Some(ret))
}
/// Parses an open `File` which represents information previously cached by
/// Cargo.
pub fn parse_cache(mut file: &File, meta: &fs::Metadata) -> CargoResult<Summaries> {
let mut contents = Vec::new();
contents.reserve(meta.len() as usize + 1);
file.read_to_end(&mut contents)?;
let cache = SummariesCache::parse(&contents)?;
let mut ret = Summaries::default();
for (version, summary) in cache.versions {
let (start, end) = subslice_bounds(&contents, summary);
ret.versions
.insert(version, MaybeIndexSummary::Unparsed { start, end });
}
ret.raw_data = contents;
return Ok(ret);
// Returns the start/end offsets of `inner` with `outer`. Asserts that
// `inner` is a subslice of `outer`.
fn subslice_bounds(outer: &[u8], inner: &[u8]) -> (usize, usize) {
let outer_start = outer.as_ptr() as usize;
let outer_end = outer_start + outer.len();
let inner_start = inner.as_ptr() as usize;
let inner_end = inner_start + inner.len();
assert!(inner_start >= outer_start);
assert!(inner_end <= outer_end);
(inner_start - outer_start, inner_end - outer_start)
}
}
}
// Implementation of serializing/deserializing the cache of summaries on disk.
// Currently the format looks like:
//
// +--------------+----------------+---+----------------+---+
// | version byte | version string | 0 | JSON blob ... | 0 | ...
// +--------------+----------------+---+----------------+---+
//
// The idea is that this is a very easy file for Cargo to parse in future
// invocations. The read from disk should be quite fast and then afterwards all
// we need to know is what versions correspond to which JSON blob.
//
// The leading version byte is intended to ensure that there's some level of
// future compatibility against changes to this cache format so if different
// versions of Cargo share the same cache they don't get too confused.
const CURRENT_CACHE_VERSION: u8 = 1;
impl<'a> SummariesCache<'a> {
fn parse(data: &'a [u8]) -> CargoResult<SummariesCache<'a>> {
// NB: keep this method in sync with `serialize` below
let (first_byte, rest) = data
.split_first()
.ok_or_else(|| failure::format_err!("malformed cache"))?;
if *first_byte != CURRENT_CACHE_VERSION {
failure::bail!("looks like a different Cargo's cache, bailing out");
}
let mut iter = memchr::Memchr::new(0, rest);
let mut start = 0;
let mut ret = SummariesCache::default();
while let Some(version_end) = iter.next() {
let version = &rest[start..version_end];
let version = str::from_utf8(version)?;
let version = Version::parse(version)?;
let summary_end = iter.next().unwrap();
let summary = &rest[version_end + 1..summary_end];
ret.versions.push((version, summary));
start = summary_end + 1;
}
Ok(ret)
}
fn serialize(&self) -> Vec<u8> {
// NB: keep this method in sync with `parse` above
let size = self
.versions
.iter()
.map(|(_version, data)| (10 + data.len()))
.sum();
let mut contents = Vec::with_capacity(size);
contents.push(CURRENT_CACHE_VERSION);
for (version, data) in self.versions.iter() {
contents.extend_from_slice(version.to_string().as_bytes());
contents.push(0);
contents.extend_from_slice(data);
contents.push(0);
}
return contents;
}
}
impl MaybeIndexSummary {
/// Parses this "maybe a summary" into a `Parsed` for sure variant.
///
/// Does nothing if this is already `Parsed`, and otherwise the `raw_data`
/// passed in is sliced with the bounds in `Unparsed` and then actually
/// parsed.
fn parse(&mut self, raw_data: &[u8], source_id: SourceId) -> CargoResult<&IndexSummary> {
let (start, end) = match self {
MaybeIndexSummary::Unparsed { start, end } => (*start, *end),
MaybeIndexSummary::Parsed(summary) => return Ok(summary),
};
let summary = IndexSummary::parse(&raw_data[start..end], source_id)?;
*self = MaybeIndexSummary::Parsed(summary);
match self {
MaybeIndexSummary::Unparsed { .. } => unreachable!(),
MaybeIndexSummary::Parsed(summary) => Ok(summary),
}
}
}
impl From<IndexSummary> for MaybeIndexSummary {
fn from(summary: IndexSummary) -> MaybeIndexSummary {
MaybeIndexSummary::Parsed(summary)
}
}
impl IndexSummary {
/// Parses a line from the registry's index file into an `IndexSummary` for
/// a package.
///
/// The `line` provided is expected to be valid JSON.
fn parse(line: &[u8], source_id: SourceId) -> CargoResult<IndexSummary> {
let RegistryPackage {
name,
vers,
cksum,
deps,
features,
yanked,
links,
} = serde_json::from_slice(line)?;
log::trace!("json parsed registry {}/{}", name, vers);
let pkgid = PackageId::new(&name, &vers, source_id)?;
let deps = deps
.into_iter()
.map(|dep| dep.into_dep(source_id))
.collect::<CargoResult<Vec<_>>>()?;
let mut summary = Summary::new(pkgid, deps, &features, links, false)?;
summary.set_checksum(cksum.clone());
Ok(IndexSummary {
summary,
yanked: yanked.unwrap_or(false),
hash: cksum,
})
}
}

View file

@ -166,7 +166,7 @@ use std::path::{Path, PathBuf};
use flate2::read::GzDecoder;
use log::debug;
use semver::Version;
use semver::{Version, VersionReq};
use serde::Deserialize;
use tar::Archive;
@ -358,7 +358,7 @@ pub trait RegistryData {
fn index_path(&self) -> &Filesystem;
fn load(
&self,
_root: &Path,
root: &Path,
path: &Path,
data: &mut dyn FnMut(&[u8]) -> CargoResult<()>,
) -> CargoResult<()>;
@ -548,13 +548,12 @@ impl<'cfg> RegistrySource<'cfg> {
// After we've loaded the package configure it's summary's `checksum`
// field with the checksum we know for this `PackageId`.
let summaries = self
let req = VersionReq::exact(package.version());
let summary_with_cksum = self
.index
.summaries(package.name().as_str(), &mut *self.ops)?;
let summary_with_cksum = summaries
.iter()
.map(|s| &s.0)
.find(|s| s.package_id() == package)
.summaries(package.name(), &req, &mut *self.ops)?
.map(|s| s.summary.clone())
.next()
.expect("summary not found");
if let Some(cksum) = summary_with_cksum.checksum() {
pkg.manifest_mut()

View file

@ -213,7 +213,7 @@ impl<'cfg> RegistryData for RemoteRegistry<'cfg> {
self.prepare()?;
self.head.set(None);
*self.tree.borrow_mut() = None;
let _lock =
let lock =
self.index_path
.open_rw(Path::new(INDEX_LOCK), self.config, "the registry index")?;
self.config
@ -227,6 +227,11 @@ impl<'cfg> RegistryData for RemoteRegistry<'cfg> {
git::fetch(repo, url, refspec, self.config)
.chain_err(|| format!("failed to fetch `{}`", url))?;
self.config.updated_sources().insert(self.source_id);
// Make a write to the lock file to record the mtime on the filesystem
// of when the last update happened.
lock.file().set_len(0)?;
lock.file().write(&[0])?;
Ok(())
}