##// END OF EJS Templates
rust-status: don't trigger dirstate v1 rewrite when only v2 data is changed...
rust-status: don't trigger dirstate v1 rewrite when only v2 data is changed The assumption that we need to rewrite (or append to) the dirstate if the ignore pattern hash has changed or if any cached directory mtimes have changed is only valid when using dirstate-v2. In dirstate-v1, neither of these things are written to disk.

File last commit:

r50232:6cd24955 stable
r50232:6cd24955 stable
Show More
status.rs
864 lines | 32.8 KiB | application/rls-services+xml | RustLexer
Simon Sapin
dirstate-v2: Truncate directory mtimes to 31 bits of seconds...
r49007 use crate::dirstate::entry::TruncatedTimestamp;
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 use crate::dirstate::status::IgnoreFnType;
Simon Sapin
rhg: Add support for `rhg status --copies`...
r49285 use crate::dirstate::status::StatusPath;
Simon Sapin
dirstate-tree: Change status() results to not borrow DirstateMap...
r48136 use crate::dirstate_tree::dirstate_map::BorrowedPath;
Simon Sapin
dirstate-tree: Add `NodeRef` and `ChildNodesRef` enums...
r48124 use crate::dirstate_tree::dirstate_map::ChildNodesRef;
Simon Sapin
dirstate-tree: Give to `status()` mutable access to the `DirstateMap`...
r47882 use crate::dirstate_tree::dirstate_map::DirstateMap;
Raphaël Gomès
rust-status: don't trigger dirstate v1 rewrite when only v2 data is changed...
r50232 use crate::dirstate_tree::dirstate_map::DirstateVersion;
Simon Sapin
dirstate-v2: Skip readdir in status based on directory mtime...
r48138 use crate::dirstate_tree::dirstate_map::NodeData;
Simon Sapin
dirstate-tree: Add `NodeRef` and `ChildNodesRef` enums...
r48124 use crate::dirstate_tree::dirstate_map::NodeRef;
Simon Sapin
dirstate-v2: Make more APIs fallible, returning Result...
r48126 use crate::dirstate_tree::on_disk::DirstateV2ParseError;
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 use crate::matchers::get_ignore_function;
Simon Sapin
dirstate-tree: Give to `status()` mutable access to the `DirstateMap`...
r47882 use crate::matchers::Matcher;
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 use crate::utils::files::get_bytes_from_os_string;
Simon Sapin
dirstate-tree: Skip readdir() in `hg status -mard`...
r48129 use crate::utils::files::get_path_from_bytes;
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 use crate::utils::hg_path::HgPath;
Simon Sapin
dirstate-tree: Handle I/O errors in status...
r47885 use crate::BadMatch;
Simon Sapin
dirstate-tree: Give to `status()` mutable access to the `DirstateMap`...
r47882 use crate::DirstateStatus;
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 use crate::EntryState;
use crate::HgPathBuf;
Simon Sapin
rhg: Add support for `rhg status --copies`...
r49285 use crate::HgPathCow;
Simon Sapin
dirstate-tree: Give to `status()` mutable access to the `DirstateMap`...
r47882 use crate::PatternFileWarning;
use crate::StatusError;
use crate::StatusOptions;
Simon Sapin
dirstate-tree: Add #[timed] attribute to `status` and `DirstateMap::read`...
r47888 use micro_timer::timed;
Simon Sapin
dirstate-tree: Paralellize the status algorithm with Rayon...
r47887 use rayon::prelude::*;
Simon Sapin
dirstate-v2: Store a hash of ignore patterns (.hgignore)...
r48202 use sha1::{Digest, Sha1};
Simon Sapin
dirstate-v2: Skip readdir in status based on directory mtime...
r48138 use std::borrow::Cow;
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 use std::io;
use std::path::Path;
Simon Sapin
dirstate-tree: Give to `status()` mutable access to the `DirstateMap`...
r47882 use std::path::PathBuf;
Simon Sapin
dirstate-tree: Paralellize the status algorithm with Rayon...
r47887 use std::sync::Mutex;
Simon Sapin
dirstate-v2: Skip readdir in status based on directory mtime...
r48138 use std::time::SystemTime;
Simon Sapin
dirstate-tree: Give to `status()` mutable access to the `DirstateMap`...
r47882
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 /// Returns the status of the working directory compared to its parent
/// changeset.
///
/// This algorithm is based on traversing the filesystem tree (`fs` in function
/// and variable names) and dirstate tree at the same time. The core of this
/// traversal is the recursive `traverse_fs_directory_and_dirstate` function
/// and its use of `itertools::merge_join_by`. When reaching a path that only
/// exists in one of the two trees, depending on information requested by
/// `options` we may need to traverse the remaining subtree.
Simon Sapin
dirstate-tree: Add #[timed] attribute to `status` and `DirstateMap::read`...
r47888 #[timed]
Raphaël Gomès
rust: fix unsound `OwningDirstateMap`...
r49864 pub fn status<'dirstate>(
dmap: &'dirstate mut DirstateMap,
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 matcher: &(dyn Matcher + Sync),
root_dir: PathBuf,
ignore_files: Vec<PathBuf>,
options: StatusOptions,
Raphaël Gomès
rust: fix unsound `OwningDirstateMap`...
r49864 ) -> Result<(DirstateStatus<'dirstate>, Vec<PatternFileWarning>), StatusError>
{
Raphaël Gomès
rust-status: cap the number of concurrent threads to 16...
r49830 // Force the global rayon threadpool to not exceed 16 concurrent threads.
// This is a stop-gap measure until we figure out why using more than 16
// threads makes `status` slower for each additional thread.
// We use `ok()` in case the global threadpool has already been
// instantiated in `rhg` or some other caller.
// TODO find the underlying cause and fix it, then remove this.
rayon::ThreadPoolBuilder::new()
.num_threads(16)
.build_global()
.ok();
Simon Sapin
dirstate-v2: Store a hash of ignore patterns (.hgignore)...
r48202 let (ignore_fn, warnings, patterns_changed): (IgnoreFnType, _, _) =
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 if options.list_ignored || options.list_unknown {
Raphaël Gomès
rust-status: don't trigger dirstate v1 rewrite when only v2 data is changed...
r50232 let (ignore_fn, warnings, changed) = match dmap.dirstate_version {
DirstateVersion::V1 => {
let (ignore_fn, warnings) = get_ignore_function(
ignore_files,
&root_dir,
&mut |_pattern_bytes| {},
)?;
(ignore_fn, warnings, None)
}
DirstateVersion::V2 => {
let mut hasher = Sha1::new();
let (ignore_fn, warnings) = get_ignore_function(
ignore_files,
&root_dir,
&mut |pattern_bytes| hasher.update(pattern_bytes),
)?;
let new_hash = *hasher.finalize().as_ref();
let changed = new_hash != dmap.ignore_patterns_hash;
dmap.ignore_patterns_hash = new_hash;
(ignore_fn, warnings, Some(changed))
}
};
(ignore_fn, warnings, changed)
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 } else {
Simon Sapin
dirstate-v2: Store a hash of ignore patterns (.hgignore)...
r48202 (Box::new(|&_| true), vec![], None)
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 };
Simon Sapin
dirstate-v2: Apply SECOND_AMBIGUOUS to directory mtimes too...
r49332 let filesystem_time_at_status_start =
filesystem_now(&root_dir).ok().map(TruncatedTimestamp::from);
Simon Sapin
status: prefer relative paths in Rust code...
r49591
// If the repository is under the current directory, prefer using a
// relative path, so the kernel needs to traverse fewer directory in every
// call to `read_dir` or `symlink_metadata`.
// This is effective in the common case where the current directory is the
// repository root.
// TODO: Better yet would be to use libc functions like `openat` and
// `fstatat` to remove such repeated traversals entirely, but the standard
// library does not provide APIs based on those.
// Maybe with a crate like https://crates.io/crates/openat instead?
let root_dir = if let Some(relative) = std::env::current_dir()
.ok()
.and_then(|cwd| root_dir.strip_prefix(cwd).ok())
{
relative
} else {
&root_dir
};
Simon Sapin
rhg: Update the dirstate on disk after status...
r49250 let outcome = DirstateStatus {
filesystem_time_at_status_start,
..Default::default()
};
Simon Sapin
dirstate-tree: Paralellize the status algorithm with Rayon...
r47887 let common = StatusCommon {
Simon Sapin
dirstate-v2: Make the dirstate bytes buffer available in more places...
r48127 dmap,
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 options,
matcher,
ignore_fn,
Simon Sapin
rhg: Update the dirstate on disk after status...
r49250 outcome: Mutex::new(outcome),
Simon Sapin
dirstate-v2: Store a hash of ignore patterns (.hgignore)...
r48202 ignore_patterns_have_changed: patterns_changed,
new_cachable_directories: Default::default(),
Simon Sapin
dirstate-v2: Drop cached read_dir results after .hgignore changes...
r48268 outated_cached_directories: Default::default(),
Simon Sapin
rhg: Update the dirstate on disk after status...
r49250 filesystem_time_at_status_start,
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 };
let is_at_repo_root = true;
Simon Sapin
dirstate-tree: Change status() results to not borrow DirstateMap...
r48136 let hg_path = &BorrowedPath::OnDisk(HgPath::new(""));
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 let has_ignored_ancestor = false;
Simon Sapin
dirstate-v2: Skip readdir in status based on directory mtime...
r48138 let root_cached_mtime = None;
let root_dir_metadata = None;
// If the path we have for the repository root is a symlink, do follow it.
// (As opposed to symlinks within the working directory which are not
// followed, using `std::fs::symlink_metadata`.)
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 common.traverse_fs_directory_and_dirstate(
has_ignored_ancestor,
Simon Sapin
dirstate-tree: Add `NodeRef` and `ChildNodesRef` enums...
r48124 dmap.root.as_ref(),
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 hg_path,
&root_dir,
Simon Sapin
dirstate-v2: Skip readdir in status based on directory mtime...
r48138 root_dir_metadata,
root_cached_mtime,
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 is_at_repo_root,
Simon Sapin
dirstate-v2: Make more APIs fallible, returning Result...
r48126 )?;
Simon Sapin
dirstate-v2: Write .hg/dirstate back to disk on directory cache changes...
r48139 let mut outcome = common.outcome.into_inner().unwrap();
Simon Sapin
dirstate-v2: Store a hash of ignore patterns (.hgignore)...
r48202 let new_cachable = common.new_cachable_directories.into_inner().unwrap();
Simon Sapin
dirstate-v2: Drop cached read_dir results after .hgignore changes...
r48268 let outdated = common.outated_cached_directories.into_inner().unwrap();
Simon Sapin
dirstate-v2: Store a hash of ignore patterns (.hgignore)...
r48202
outcome.dirty = common.ignore_patterns_have_changed == Some(true)
Simon Sapin
dirstate-v2: Drop cached read_dir results after .hgignore changes...
r48268 || !outdated.is_empty()
Raphaël Gomès
rust-status: don't trigger dirstate v1 rewrite when only v2 data is changed...
r50232 || (!new_cachable.is_empty()
&& dmap.dirstate_version == DirstateVersion::V2);
Simon Sapin
dirstate-v2: Store a hash of ignore patterns (.hgignore)...
r48202
Simon Sapin
dirstate-v2: Drop cached read_dir results after .hgignore changes...
r48268 // Remove outdated mtimes before adding new mtimes, in case a given
// directory is both
for path in &outdated {
let node = dmap.get_or_insert(path)?;
if let NodeData::CachedDirectory { .. } = &node.data {
node.data = NodeData::None
}
}
Simon Sapin
dirstate-v2: Store a hash of ignore patterns (.hgignore)...
r48202 for (path, mtime) in &new_cachable {
Simon Sapin
dirstate-v2: Drop cached read_dir results after .hgignore changes...
r48268 let node = dmap.get_or_insert(path)?;
Simon Sapin
dirstate-v2: Skip readdir in status based on directory mtime...
r48138 match &node.data {
NodeData::Entry(_) => {} // Don’t overwrite an entry
NodeData::CachedDirectory { .. } | NodeData::None => {
node.data = NodeData::CachedDirectory { mtime: *mtime }
}
}
}
Simon Sapin
dirstate-v2: Store a hash of ignore patterns (.hgignore)...
r48202
Simon Sapin
dirstate-v2: Skip readdir in status based on directory mtime...
r48138 Ok((outcome, warnings))
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 }
/// Bag of random things needed by various parts of the algorithm. Reduces the
/// number of parameters passed to functions.
Simon Sapin
dirstate-tree: Change status() results to not borrow DirstateMap...
r48136 struct StatusCommon<'a, 'tree, 'on_disk: 'tree> {
Simon Sapin
dirstate-v2: Make the dirstate bytes buffer available in more places...
r48127 dmap: &'tree DirstateMap<'on_disk>,
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 options: StatusOptions,
matcher: &'a (dyn Matcher + Sync),
ignore_fn: IgnoreFnType<'a>,
Simon Sapin
dirstate-tree: Change status() results to not borrow DirstateMap...
r48136 outcome: Mutex<DirstateStatus<'on_disk>>,
Simon Sapin
dirstate-v2: Truncate directory mtimes to 31 bits of seconds...
r49007 new_cachable_directories:
Mutex<Vec<(Cow<'on_disk, HgPath>, TruncatedTimestamp)>>,
Simon Sapin
dirstate-v2: Drop cached read_dir results after .hgignore changes...
r48268 outated_cached_directories: Mutex<Vec<Cow<'on_disk, HgPath>>>,
Simon Sapin
dirstate-v2: Store a hash of ignore patterns (.hgignore)...
r48202
/// Whether ignore files like `.hgignore` have changed since the previous
/// time a `status()` call wrote their hash to the dirstate. `None` means
/// we don’t know as this run doesn’t list either ignored or uknown files
/// and therefore isn’t reading `.hgignore`.
ignore_patterns_have_changed: Option<bool>,
Simon Sapin
dirstate-v2: Skip readdir in status based on directory mtime...
r48138
/// The current time at the start of the `status()` algorithm, as measured
/// and possibly truncated by the filesystem.
Simon Sapin
dirstate-v2: Apply SECOND_AMBIGUOUS to directory mtimes too...
r49332 filesystem_time_at_status_start: Option<TruncatedTimestamp>,
Simon Sapin
dirstate-tree: Give to `status()` mutable access to the `DirstateMap`...
r47882 }
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883
Simon Sapin
rhg: Add support for `rhg status --copies`...
r49285 enum Outcome {
Modified,
Added,
Removed,
Deleted,
Clean,
Ignored,
Unknown,
Unsure,
Simon Sapin
dirstate-tree: Give to `status()` mutable access to the `DirstateMap`...
r47882 }
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883
Simon Sapin
dirstate-tree: Change status() results to not borrow DirstateMap...
r48136 impl<'a, 'tree, 'on_disk> StatusCommon<'a, 'tree, 'on_disk> {
Simon Sapin
rhg: Add support for `rhg status --copies`...
r49285 fn push_outcome(
&self,
which: Outcome,
dirstate_node: &NodeRef<'tree, 'on_disk>,
) -> Result<(), DirstateV2ParseError> {
let path = dirstate_node
.full_path_borrowed(self.dmap.on_disk)?
.detach_from_tree();
let copy_source = if self.options.list_copies {
dirstate_node
.copy_source_borrowed(self.dmap.on_disk)?
.map(|source| source.detach_from_tree())
} else {
None
};
self.push_outcome_common(which, path, copy_source);
Ok(())
}
fn push_outcome_without_copy_source(
&self,
which: Outcome,
path: &BorrowedPath<'_, 'on_disk>,
) {
self.push_outcome_common(which, path.detach_from_tree(), None)
}
fn push_outcome_common(
&self,
which: Outcome,
path: HgPathCow<'on_disk>,
copy_source: Option<HgPathCow<'on_disk>>,
) {
let mut outcome = self.outcome.lock().unwrap();
let vec = match which {
Outcome::Modified => &mut outcome.modified,
Outcome::Added => &mut outcome.added,
Outcome::Removed => &mut outcome.removed,
Outcome::Deleted => &mut outcome.deleted,
Outcome::Clean => &mut outcome.clean,
Outcome::Ignored => &mut outcome.ignored,
Outcome::Unknown => &mut outcome.unknown,
Outcome::Unsure => &mut outcome.unsure,
};
vec.push(StatusPath { path, copy_source });
}
Simon Sapin
dirstate-tree: Handle I/O errors in status...
r47885 fn read_dir(
Simon Sapin
dirstate-tree: Paralellize the status algorithm with Rayon...
r47887 &self,
Simon Sapin
dirstate-tree: Handle I/O errors in status...
r47885 hg_path: &HgPath,
fs_path: &Path,
is_at_repo_root: bool,
) -> Result<Vec<DirEntry>, ()> {
Simon Sapin
dirstate-tree: Skip readdir() in `hg status -mard`...
r48129 DirEntry::read_dir(fs_path, is_at_repo_root)
.map_err(|error| self.io_error(error, hg_path))
}
fn io_error(&self, error: std::io::Error, hg_path: &HgPath) {
let errno = error.raw_os_error().expect("expected real OS error");
self.outcome
.lock()
.unwrap()
.bad
.push((hg_path.to_owned().into(), BadMatch::OsError(errno)))
Simon Sapin
dirstate-tree: Handle I/O errors in status...
r47885 }
Simon Sapin
dirstate-v2: Drop cached read_dir results after .hgignore changes...
r48268 fn check_for_outdated_directory_cache(
&self,
dirstate_node: &NodeRef<'tree, 'on_disk>,
) -> Result<(), DirstateV2ParseError> {
if self.ignore_patterns_have_changed == Some(true)
Simon Sapin
dirstate-v2: Truncate directory mtimes to 31 bits of seconds...
r49007 && dirstate_node.cached_directory_mtime()?.is_some()
Simon Sapin
dirstate-v2: Drop cached read_dir results after .hgignore changes...
r48268 {
self.outated_cached_directories.lock().unwrap().push(
dirstate_node
.full_path_borrowed(self.dmap.on_disk)?
.detach_from_tree(),
)
}
Ok(())
}
Simon Sapin
dirstate-v2: Skip readdir in status based on directory mtime...
r48138 /// If this returns true, we can get accurate results by only using
/// `symlink_metadata` for child nodes that exist in the dirstate and don’t
/// need to call `read_dir`.
fn can_skip_fs_readdir(
&self,
directory_metadata: Option<&std::fs::Metadata>,
Simon Sapin
dirstate-v2: Truncate directory mtimes to 31 bits of seconds...
r49007 cached_directory_mtime: Option<TruncatedTimestamp>,
Simon Sapin
dirstate-v2: Skip readdir in status based on directory mtime...
r48138 ) -> bool {
if !self.options.list_unknown && !self.options.list_ignored {
// All states that we care about listing have corresponding
// dirstate entries.
// This happens for example with `hg status -mard`.
return true;
}
Simon Sapin
status: Extend read_dir caching to directories with ignored files...
r48269 if !self.options.list_ignored
&& self.ignore_patterns_have_changed == Some(false)
{
if let Some(cached_mtime) = cached_directory_mtime {
// The dirstate contains a cached mtime for this directory, set
// by a previous run of the `status` algorithm which found this
// directory eligible for `read_dir` caching.
if let Some(meta) = directory_metadata {
Simon Sapin
status: Extract TruncatedTimestamp from fs::Metadata without SystemTime...
r49032 if cached_mtime
Simon Sapin
dirstate: rename a `very_likely_equal` method to `likely_equal`...
r49076 .likely_equal_to_mtime_of(meta)
Simon Sapin
status: Extract TruncatedTimestamp from fs::Metadata without SystemTime...
r49032 .unwrap_or(false)
{
// The mtime of that directory has not changed
// since then, which means that the results of
// `read_dir` should also be unchanged.
return true;
Simon Sapin
dirstate-v2: Skip readdir in status based on directory mtime...
r48138 }
}
}
}
false
}
Simon Sapin
status: Extend read_dir caching to directories with ignored files...
r48269 /// Returns whether all child entries of the filesystem directory have a
/// corresponding dirstate node or are ignored.
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 fn traverse_fs_directory_and_dirstate(
Simon Sapin
dirstate-tree: Paralellize the status algorithm with Rayon...
r47887 &self,
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 has_ignored_ancestor: bool,
Simon Sapin
dirstate-tree: Change status() results to not borrow DirstateMap...
r48136 dirstate_nodes: ChildNodesRef<'tree, 'on_disk>,
directory_hg_path: &BorrowedPath<'tree, 'on_disk>,
Simon Sapin
dirstate-tree: Handle I/O errors in status...
r47885 directory_fs_path: &Path,
Simon Sapin
dirstate-v2: Skip readdir in status based on directory mtime...
r48138 directory_metadata: Option<&std::fs::Metadata>,
Simon Sapin
dirstate-v2: Truncate directory mtimes to 31 bits of seconds...
r49007 cached_directory_mtime: Option<TruncatedTimestamp>,
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 is_at_repo_root: bool,
Simon Sapin
dirstate-v2: Skip readdir in status based on directory mtime...
r48138 ) -> Result<bool, DirstateV2ParseError> {
if self.can_skip_fs_readdir(directory_metadata, cached_directory_mtime)
{
dirstate_nodes
Simon Sapin
dirstate-tree: Skip readdir() in `hg status -mard`...
r48129 .par_iter()
.map(|dirstate_node| {
let fs_path = directory_fs_path.join(get_path_from_bytes(
dirstate_node.base_name(self.dmap.on_disk)?.as_bytes(),
));
match std::fs::symlink_metadata(&fs_path) {
Ok(fs_metadata) => self.traverse_fs_and_dirstate(
&fs_path,
&fs_metadata,
dirstate_node,
has_ignored_ancestor,
),
Err(e) if e.kind() == std::io::ErrorKind::NotFound => {
self.traverse_dirstate_only(dirstate_node)
}
Err(error) => {
let hg_path =
dirstate_node.full_path(self.dmap.on_disk)?;
Ok(self.io_error(error, hg_path))
}
}
})
Simon Sapin
dirstate-v2: Skip readdir in status based on directory mtime...
r48138 .collect::<Result<_, _>>()?;
Simon Sapin
status: Extend read_dir caching to directories with ignored files...
r48269 // We don’t know, so conservatively say this isn’t the case
let children_all_have_dirstate_node_or_are_ignored = false;
Simon Sapin
dirstate-v2: Skip readdir in status based on directory mtime...
r48138
Simon Sapin
status: Extend read_dir caching to directories with ignored files...
r48269 return Ok(children_all_have_dirstate_node_or_are_ignored);
Simon Sapin
dirstate-tree: Skip readdir() in `hg status -mard`...
r48129 }
Simon Sapin
dirstate-tree: Handle I/O errors in status...
r47885 let mut fs_entries = if let Ok(entries) = self.read_dir(
directory_hg_path,
directory_fs_path,
is_at_repo_root,
) {
entries
} else {
Simon Sapin
dirstate-tree: Fix status algorithm with unreadable directory...
r48135 // Treat an unreadable directory (typically because of insufficient
// permissions) like an empty directory. `self.read_dir` has
// already called `self.io_error` so a warning will be emitted.
Vec::new()
Simon Sapin
dirstate-tree: Handle I/O errors in status...
r47885 };
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883
// `merge_join_by` requires both its input iterators to be sorted:
Simon Sapin
dirstate-tree: Add `NodeRef` and `ChildNodesRef` enums...
r48124 let dirstate_nodes = dirstate_nodes.sorted();
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 // `sort_unstable_by_key` doesn’t allow keys borrowing from the value:
// https://github.com/rust-lang/rust/issues/34162
fs_entries.sort_unstable_by(|e1, e2| e1.base_name.cmp(&e2.base_name));
Simon Sapin
dirstate-v2: Make more APIs fallible, returning Result...
r48126 // Propagate here any error that would happen inside the comparison
// callback below
for dirstate_node in &dirstate_nodes {
Simon Sapin
dirstate-v2: Make the dirstate bytes buffer available in more places...
r48127 dirstate_node.base_name(self.dmap.on_disk)?;
Simon Sapin
dirstate-v2: Make more APIs fallible, returning Result...
r48126 }
Simon Sapin
dirstate-tree: Paralellize the status algorithm with Rayon...
r47887 itertools::merge_join_by(
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 dirstate_nodes,
&fs_entries,
Simon Sapin
dirstate-tree: Add `NodeRef` and `ChildNodesRef` enums...
r48124 |dirstate_node, fs_entry| {
Simon Sapin
dirstate-v2: Make more APIs fallible, returning Result...
r48126 // This `unwrap` never panics because we already propagated
// those errors above
Simon Sapin
dirstate-v2: Make the dirstate bytes buffer available in more places...
r48127 dirstate_node
.base_name(self.dmap.on_disk)
.unwrap()
.cmp(&fs_entry.base_name)
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 },
Simon Sapin
dirstate-tree: Paralellize the status algorithm with Rayon...
r47887 )
.par_bridge()
Simon Sapin
dirstate-v2: Make more APIs fallible, returning Result...
r48126 .map(|pair| {
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 use itertools::EitherOrBoth::*;
Simon Sapin
status: Extend read_dir caching to directories with ignored files...
r48269 let has_dirstate_node_or_is_ignored;
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 match pair {
Simon Sapin
status: Extend read_dir caching to directories with ignored files...
r48269 Both(dirstate_node, fs_entry) => {
self.traverse_fs_and_dirstate(
Simon Sapin
dirstate-tree: Skip readdir() in `hg status -mard`...
r48129 &fs_entry.full_path,
&fs_entry.metadata,
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 dirstate_node,
has_ignored_ancestor,
Simon Sapin
status: Extend read_dir caching to directories with ignored files...
r48269 )?;
has_dirstate_node_or_is_ignored = true
}
Simon Sapin
dirstate-tree: Add `NodeRef` and `ChildNodesRef` enums...
r48124 Left(dirstate_node) => {
Simon Sapin
status: Extend read_dir caching to directories with ignored files...
r48269 self.traverse_dirstate_only(dirstate_node)?;
has_dirstate_node_or_is_ignored = true;
Simon Sapin
dirstate-tree: Add `NodeRef` and `ChildNodesRef` enums...
r48124 }
Simon Sapin
status: Extend read_dir caching to directories with ignored files...
r48269 Right(fs_entry) => {
has_dirstate_node_or_is_ignored = self.traverse_fs_only(
has_ignored_ancestor,
directory_hg_path,
fs_entry,
)
}
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 }
Simon Sapin
status: Extend read_dir caching to directories with ignored files...
r48269 Ok(has_dirstate_node_or_is_ignored)
Simon Sapin
dirstate-tree: Paralellize the status algorithm with Rayon...
r47887 })
Simon Sapin
status: Extend read_dir caching to directories with ignored files...
r48269 .try_reduce(|| true, |a, b| Ok(a && b))
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 }
fn traverse_fs_and_dirstate(
Simon Sapin
dirstate-tree: Paralellize the status algorithm with Rayon...
r47887 &self,
Simon Sapin
dirstate-tree: Skip readdir() in `hg status -mard`...
r48129 fs_path: &Path,
fs_metadata: &std::fs::Metadata,
Simon Sapin
dirstate-tree: Change status() results to not borrow DirstateMap...
r48136 dirstate_node: NodeRef<'tree, 'on_disk>,
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 has_ignored_ancestor: bool,
Simon Sapin
dirstate-v2: Make more APIs fallible, returning Result...
r48126 ) -> Result<(), DirstateV2ParseError> {
Simon Sapin
dirstate-v2: Drop cached read_dir results after .hgignore changes...
r48268 self.check_for_outdated_directory_cache(&dirstate_node)?;
Simon Sapin
dirstate-tree: Change status() results to not borrow DirstateMap...
r48136 let hg_path = &dirstate_node.full_path_borrowed(self.dmap.on_disk)?;
Simon Sapin
dirstate-tree: Skip readdir() in `hg status -mard`...
r48129 let file_type = fs_metadata.file_type();
Simon Sapin
dirstate-tree: Ignore FIFOs etc. in the status algorithm...
r47884 let file_or_symlink = file_type.is_file() || file_type.is_symlink();
if !file_or_symlink {
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 // If we previously had a file here, it was removed (with
// `hg rm` or similar) or deleted before it could be
Simon Sapin
dirstate-tree: Ignore FIFOs etc. in the status algorithm...
r47884 // replaced by a directory or something else.
Simon Sapin
rhg: Add support for `rhg status --copies`...
r49285 self.mark_removed_or_deleted_if_file(&dirstate_node)?;
Simon Sapin
dirstate-tree: Ignore FIFOs etc. in the status algorithm...
r47884 }
if file_type.is_dir() {
if self.options.collect_traversed_dirs {
Simon Sapin
dirstate-tree: Change status() results to not borrow DirstateMap...
r48136 self.outcome
.lock()
.unwrap()
.traversed
.push(hg_path.detach_from_tree())
Simon Sapin
dirstate-tree: Ignore FIFOs etc. in the status algorithm...
r47884 }
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 let is_ignored = has_ignored_ancestor || (self.ignore_fn)(hg_path);
let is_at_repo_root = false;
Simon Sapin
status: Extend read_dir caching to directories with ignored files...
r48269 let children_all_have_dirstate_node_or_are_ignored = self
Simon Sapin
dirstate-v2: Skip readdir in status based on directory mtime...
r48138 .traverse_fs_directory_and_dirstate(
is_ignored,
dirstate_node.children(self.dmap.on_disk)?,
hg_path,
fs_path,
Some(fs_metadata),
Simon Sapin
dirstate-v2: Truncate directory mtimes to 31 bits of seconds...
r49007 dirstate_node.cached_directory_mtime()?,
Simon Sapin
dirstate-v2: Skip readdir in status based on directory mtime...
r48138 is_at_repo_root,
)?;
self.maybe_save_directory_mtime(
Simon Sapin
status: Extend read_dir caching to directories with ignored files...
r48269 children_all_have_dirstate_node_or_are_ignored,
Simon Sapin
dirstate-v2: Skip readdir in status based on directory mtime...
r48138 fs_metadata,
dirstate_node,
Simon Sapin
dirstate-v2: Make more APIs fallible, returning Result...
r48126 )?
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 } else {
Simon Sapin
dirstate-tree: Ignore FIFOs etc. in the status algorithm...
r47884 if file_or_symlink && self.matcher.matches(hg_path) {
Simon Sapin
dirstate-v2: Make more APIs fallible, returning Result...
r48126 if let Some(state) = dirstate_node.state()? {
Simon Sapin
dirstate-tree: Add `NodeRef` and `ChildNodesRef` enums...
r48124 match state {
Simon Sapin
rhg: Add support for `rhg status --copies`...
r49285 EntryState::Added => {
self.push_outcome(Outcome::Added, &dirstate_node)?
}
Simon Sapin
dirstate-tree: Paralellize the status algorithm with Rayon...
r47887 EntryState::Removed => self
Simon Sapin
rhg: Add support for `rhg status --copies`...
r49285 .push_outcome(Outcome::Removed, &dirstate_node)?,
Simon Sapin
dirstate-tree: Paralellize the status algorithm with Rayon...
r47887 EntryState::Merged => self
Simon Sapin
rhg: Add support for `rhg status --copies`...
r49285 .push_outcome(Outcome::Modified, &dirstate_node)?,
Simon Sapin
dirstate-tree: Skip readdir() in `hg status -mard`...
r48129 EntryState::Normal => self
.handle_normal_file(&dirstate_node, fs_metadata)?,
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 }
} else {
// `node.entry.is_none()` indicates a "directory"
// node, but the filesystem has a file
Simon Sapin
status: Extend read_dir caching to directories with ignored files...
r48269 self.mark_unknown_or_ignored(
has_ignored_ancestor,
hg_path,
);
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 }
}
Simon Sapin
dirstate-v2: Make the dirstate bytes buffer available in more places...
r48127 for child_node in dirstate_node.children(self.dmap.on_disk)?.iter()
{
Simon Sapin
dirstate-v2: Make more APIs fallible, returning Result...
r48126 self.traverse_dirstate_only(child_node)?
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 }
}
Simon Sapin
dirstate-v2: Make more APIs fallible, returning Result...
r48126 Ok(())
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 }
Simon Sapin
dirstate-v2: Skip readdir in status based on directory mtime...
r48138 fn maybe_save_directory_mtime(
&self,
Simon Sapin
status: Extend read_dir caching to directories with ignored files...
r48269 children_all_have_dirstate_node_or_are_ignored: bool,
Simon Sapin
dirstate-v2: Skip readdir in status based on directory mtime...
r48138 directory_metadata: &std::fs::Metadata,
dirstate_node: NodeRef<'tree, 'on_disk>,
) -> Result<(), DirstateV2ParseError> {
Simon Sapin
dirstate-v2: Apply SECOND_AMBIGUOUS to directory mtimes too...
r49332 if !children_all_have_dirstate_node_or_are_ignored {
return Ok(());
}
// All filesystem directory entries from `read_dir` have a
// corresponding node in the dirstate, so we can reconstitute the
// names of those entries without calling `read_dir` again.
// TODO: use let-else here and below when available:
// https://github.com/rust-lang/rust/issues/87335
let status_start = if let Some(status_start) =
&self.filesystem_time_at_status_start
{
status_start
} else {
return Ok(());
};
// Although the Rust standard library’s `SystemTime` type
// has nanosecond precision, the times reported for a
// directory’s (or file’s) modified time may have lower
// resolution based on the filesystem (for example ext3
// only stores integer seconds), kernel (see
// https://stackoverflow.com/a/14393315/1162888), etc.
let directory_mtime = if let Ok(option) =
TruncatedTimestamp::for_reliable_mtime_of(
directory_metadata,
status_start,
Simon Sapin
dirstate-v2: Skip readdir in status based on directory mtime...
r48138 ) {
Simon Sapin
dirstate-v2: Apply SECOND_AMBIGUOUS to directory mtimes too...
r49332 if let Some(directory_mtime) = option {
directory_mtime
} else {
// The directory was modified too recently,
// don’t cache its `read_dir` results.
//
// 1. A change to this directory (direct child was
// added or removed) cause its mtime to be set
// (possibly truncated) to `directory_mtime`
// 2. This `status` algorithm calls `read_dir`
// 3. An other change is made to the same directory is
// made so that calling `read_dir` agin would give
// different results, but soon enough after 1. that
// the mtime stays the same
//
// On a system where the time resolution poor, this
// scenario is not unlikely if all three steps are caused
// by the same script.
return Ok(());
Simon Sapin
dirstate-v2: Skip readdir in status based on directory mtime...
r48138 }
Simon Sapin
dirstate-v2: Apply SECOND_AMBIGUOUS to directory mtimes too...
r49332 } else {
// OS/libc does not support mtime?
return Ok(());
};
// We’ve observed (through `status_start`) that time has
// “progressed” since `directory_mtime`, so any further
// change to this directory is extremely likely to cause a
// different mtime.
//
// Having the same mtime again is not entirely impossible
// since the system clock is not monotonous. It could jump
// backward to some point before `directory_mtime`, then a
// directory change could potentially happen during exactly
// the wrong tick.
//
// We deem this scenario (unlike the previous one) to be
// unlikely enough in practice.
let is_up_to_date =
if let Some(cached) = dirstate_node.cached_directory_mtime()? {
cached.likely_equal(directory_mtime)
} else {
false
};
if !is_up_to_date {
let hg_path = dirstate_node
.full_path_borrowed(self.dmap.on_disk)?
.detach_from_tree();
self.new_cachable_directories
.lock()
.unwrap()
.push((hg_path, directory_mtime))
Simon Sapin
dirstate-v2: Skip readdir in status based on directory mtime...
r48138 }
Ok(())
}
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 /// A file with `EntryState::Normal` in the dirstate was found in the
/// filesystem
fn handle_normal_file(
Simon Sapin
dirstate-tree: Paralellize the status algorithm with Rayon...
r47887 &self,
Simon Sapin
dirstate-tree: Change status() results to not borrow DirstateMap...
r48136 dirstate_node: &NodeRef<'tree, 'on_disk>,
Simon Sapin
dirstate-tree: Skip readdir() in `hg status -mard`...
r48129 fs_metadata: &std::fs::Metadata,
Simon Sapin
dirstate-v2: Make more APIs fallible, returning Result...
r48126 ) -> Result<(), DirstateV2ParseError> {
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 // Keep the low 31 bits
fn truncate_u64(value: u64) -> i32 {
(value & 0x7FFF_FFFF) as i32
}
Simon Sapin
dirstate-tree: Add `NodeRef` and `ChildNodesRef` enums...
r48124 let entry = dirstate_node
Simon Sapin
dirstate-v2: Make more APIs fallible, returning Result...
r48126 .entry()?
Simon Sapin
dirstate-tree: Add `NodeRef` and `ChildNodesRef` enums...
r48124 .expect("handle_normal_file called with entry-less node");
Simon Sapin
dirstate-tree: Skip readdir() in `hg status -mard`...
r48129 let mode_changed =
|| self.options.check_exec && entry.mode_changed(fs_metadata);
Simon Sapin
rust: Make the fields of DirstateEntry private...
r48834 let size = entry.size();
let size_changed = size != truncate_u64(fs_metadata.len());
if size >= 0 && size_changed && fs_metadata.file_type().is_symlink() {
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 // issue6456: Size returned may be longer due to encryption
// on EXT-4 fscrypt. TODO maybe only do it on EXT4?
Simon Sapin
rhg: Add support for `rhg status --copies`...
r49285 self.push_outcome(Outcome::Unsure, dirstate_node)?
Simon Sapin
dirstate-v2: Make more APIs fallible, returning Result...
r48126 } else if dirstate_node.has_copy_source()
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 || entry.is_from_other_parent()
Simon Sapin
rust: Make the fields of DirstateEntry private...
r48834 || (size >= 0 && (size_changed || mode_changed()))
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 {
Simon Sapin
rhg: Add support for `rhg status --copies`...
r49285 self.push_outcome(Outcome::Modified, dirstate_node)?
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 } else {
Simon Sapin
dirstate: store mtimes with nanosecond precision in memory...
r49079 let mtime_looks_clean;
if let Some(dirstate_mtime) = entry.truncated_mtime() {
let fs_mtime = TruncatedTimestamp::for_mtime_of(fs_metadata)
Simon Sapin
dirstate-v2: actually use sub-second mtime precision...
r49082 .expect("OS/libc does not support mtime?");
dirstate: remove `lastnormaltime` mechanism...
r49220 // There might be a change in the future if for example the
// internal clock become off while process run, but this is a
// case where the issues the user would face
// would be a lot worse and there is nothing we
// can really do.
Simon Sapin
dirstate: store mtimes with nanosecond precision in memory...
r49079 mtime_looks_clean = fs_mtime.likely_equal(dirstate_mtime)
} else {
// No mtime in the dirstate entry
mtime_looks_clean = false
};
if !mtime_looks_clean {
Simon Sapin
rhg: Add support for `rhg status --copies`...
r49285 self.push_outcome(Outcome::Unsure, dirstate_node)?
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 } else if self.options.list_clean {
Simon Sapin
rhg: Add support for `rhg status --copies`...
r49285 self.push_outcome(Outcome::Clean, dirstate_node)?
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 }
}
Simon Sapin
dirstate-v2: Make more APIs fallible, returning Result...
r48126 Ok(())
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 }
/// A node in the dirstate tree has no corresponding filesystem entry
Simon Sapin
dirstate-v2: Make more APIs fallible, returning Result...
r48126 fn traverse_dirstate_only(
&self,
Simon Sapin
dirstate-tree: Change status() results to not borrow DirstateMap...
r48136 dirstate_node: NodeRef<'tree, 'on_disk>,
Simon Sapin
dirstate-v2: Make more APIs fallible, returning Result...
r48126 ) -> Result<(), DirstateV2ParseError> {
Simon Sapin
dirstate-v2: Drop cached read_dir results after .hgignore changes...
r48268 self.check_for_outdated_directory_cache(&dirstate_node)?;
Simon Sapin
rhg: Add support for `rhg status --copies`...
r49285 self.mark_removed_or_deleted_if_file(&dirstate_node)?;
Simon Sapin
dirstate-tree: Add `NodeRef` and `ChildNodesRef` enums...
r48124 dirstate_node
Simon Sapin
dirstate-v2: Make the dirstate bytes buffer available in more places...
r48127 .children(self.dmap.on_disk)?
Simon Sapin
dirstate-tree: Add `NodeRef` and `ChildNodesRef` enums...
r48124 .par_iter()
Simon Sapin
dirstate-v2: Make more APIs fallible, returning Result...
r48126 .map(|child_node| self.traverse_dirstate_only(child_node))
.collect()
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 }
/// A node in the dirstate tree has no corresponding *file* on the
/// filesystem
///
/// Does nothing on a "directory" node
fn mark_removed_or_deleted_if_file(
Simon Sapin
dirstate-tree: Paralellize the status algorithm with Rayon...
r47887 &self,
Simon Sapin
rhg: Add support for `rhg status --copies`...
r49285 dirstate_node: &NodeRef<'tree, 'on_disk>,
) -> Result<(), DirstateV2ParseError> {
if let Some(state) = dirstate_node.state()? {
let path = dirstate_node.full_path(self.dmap.on_disk)?;
if self.matcher.matches(path) {
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 if let EntryState::Removed = state {
Simon Sapin
rhg: Add support for `rhg status --copies`...
r49285 self.push_outcome(Outcome::Removed, dirstate_node)?
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 } else {
Simon Sapin
rhg: Add support for `rhg status --copies`...
r49285 self.push_outcome(Outcome::Deleted, &dirstate_node)?
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 }
}
}
Simon Sapin
rhg: Add support for `rhg status --copies`...
r49285 Ok(())
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 }
/// Something in the filesystem has no corresponding dirstate node
Simon Sapin
status: Extend read_dir caching to directories with ignored files...
r48269 ///
/// Returns whether that path is ignored
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 fn traverse_fs_only(
Simon Sapin
dirstate-tree: Paralellize the status algorithm with Rayon...
r47887 &self,
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 has_ignored_ancestor: bool,
directory_hg_path: &HgPath,
fs_entry: &DirEntry,
Simon Sapin
status: Extend read_dir caching to directories with ignored files...
r48269 ) -> bool {
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 let hg_path = directory_hg_path.join(&fs_entry.base_name);
Simon Sapin
dirstate-tree: Ignore FIFOs etc. in the status algorithm...
r47884 let file_type = fs_entry.metadata.file_type();
let file_or_symlink = file_type.is_file() || file_type.is_symlink();
if file_type.is_dir() {
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 let is_ignored =
has_ignored_ancestor || (self.ignore_fn)(&hg_path);
let traverse_children = if is_ignored {
// Descendants of an ignored directory are all ignored
self.options.list_ignored
} else {
// Descendants of an unknown directory may be either unknown or
// ignored
self.options.list_unknown || self.options.list_ignored
};
if traverse_children {
let is_at_repo_root = false;
Simon Sapin
dirstate-tree: Handle I/O errors in status...
r47885 if let Ok(children_fs_entries) = self.read_dir(
&hg_path,
&fs_entry.full_path,
is_at_repo_root,
) {
Simon Sapin
dirstate-tree: Paralellize the status algorithm with Rayon...
r47887 children_fs_entries.par_iter().for_each(|child_fs_entry| {
Simon Sapin
dirstate-tree: Handle I/O errors in status...
r47885 self.traverse_fs_only(
is_ignored,
&hg_path,
Simon Sapin
dirstate-tree: Paralellize the status algorithm with Rayon...
r47887 child_fs_entry,
Simon Sapin
status: Extend read_dir caching to directories with ignored files...
r48269 );
Simon Sapin
dirstate-tree: Paralellize the status algorithm with Rayon...
r47887 })
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 }
}
if self.options.collect_traversed_dirs {
Simon Sapin
dirstate-tree: Paralellize the status algorithm with Rayon...
r47887 self.outcome.lock().unwrap().traversed.push(hg_path.into())
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 }
Simon Sapin
status: Extend read_dir caching to directories with ignored files...
r48269 is_ignored
} else {
if file_or_symlink {
if self.matcher.matches(&hg_path) {
self.mark_unknown_or_ignored(
has_ignored_ancestor,
&BorrowedPath::InMemory(&hg_path),
)
} else {
// We haven’t computed whether this path is ignored. It
// might not be, and a future run of status might have a
// different matcher that matches it. So treat it as not
// ignored. That is, inhibit readdir caching of the parent
// directory.
false
}
} else {
// This is neither a directory, a plain file, or a symlink.
// Treat it like an ignored file.
true
}
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 }
}
Simon Sapin
status: Extend read_dir caching to directories with ignored files...
r48269 /// Returns whether that path is ignored
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 fn mark_unknown_or_ignored(
Simon Sapin
dirstate-tree: Paralellize the status algorithm with Rayon...
r47887 &self,
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 has_ignored_ancestor: bool,
Simon Sapin
dirstate-tree: Change status() results to not borrow DirstateMap...
r48136 hg_path: &BorrowedPath<'_, 'on_disk>,
Simon Sapin
status: Extend read_dir caching to directories with ignored files...
r48269 ) -> bool {
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 let is_ignored = has_ignored_ancestor || (self.ignore_fn)(&hg_path);
if is_ignored {
if self.options.list_ignored {
Simon Sapin
rhg: Add support for `rhg status --copies`...
r49285 self.push_outcome_without_copy_source(
Outcome::Ignored,
hg_path,
)
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 }
} else {
if self.options.list_unknown {
Simon Sapin
rhg: Add support for `rhg status --copies`...
r49285 self.push_outcome_without_copy_source(
Outcome::Unknown,
hg_path,
)
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 }
}
Simon Sapin
status: Extend read_dir caching to directories with ignored files...
r48269 is_ignored
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 }
}
struct DirEntry {
base_name: HgPathBuf,
full_path: PathBuf,
metadata: std::fs::Metadata,
}
impl DirEntry {
/// Returns **unsorted** entries in the given directory, with name and
/// metadata.
///
/// If a `.hg` sub-directory is encountered:
///
/// * At the repository root, ignore that sub-directory
/// * Elsewhere, we’re listing the content of a sub-repo. Return an empty
/// list instead.
fn read_dir(path: &Path, is_at_repo_root: bool) -> io::Result<Vec<Self>> {
Simon Sapin
status: prefer relative paths in Rust code...
r49591 // `read_dir` returns a "not found" error for the empty path
let at_cwd = path == Path::new("");
let read_dir_path = if at_cwd { Path::new(".") } else { path };
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 let mut results = Vec::new();
Simon Sapin
status: prefer relative paths in Rust code...
r49591 for entry in read_dir_path.read_dir()? {
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 let entry = entry?;
Arseniy Alekseyev
status: fix hg status race against file deletion...
r49645 let metadata = match entry.metadata() {
Ok(v) => v,
Err(e) => {
// race with file deletion?
if e.kind() == std::io::ErrorKind::NotFound {
continue;
} else {
return Err(e);
}
}
};
Simon Sapin
status: prefer relative paths in Rust code...
r49591 let file_name = entry.file_name();
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 // FIXME don't do this when cached
Simon Sapin
status: prefer relative paths in Rust code...
r49591 if file_name == ".hg" {
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 if is_at_repo_root {
// Skip the repo’s own .hg (might be a symlink)
continue;
} else if metadata.is_dir() {
// A .hg sub-directory at another location means a subrepo,
// skip it entirely.
return Ok(Vec::new());
}
}
Simon Sapin
status: prefer relative paths in Rust code...
r49591 let full_path = if at_cwd {
file_name.clone().into()
} else {
entry.path()
};
let base_name = get_bytes_from_os_string(file_name).into();
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 results.push(DirEntry {
Simon Sapin
status: prefer relative paths in Rust code...
r49591 base_name,
full_path,
Simon Sapin
dirstate-tree: Add the new `status()` algorithm...
r47883 metadata,
})
}
Ok(results)
}
}
Simon Sapin
dirstate-v2: Skip readdir in status based on directory mtime...
r48138
/// Return the `mtime` of a temporary file newly-created in the `.hg` directory
/// of the give repository.
///
/// This is similar to `SystemTime::now()`, with the result truncated to the
/// same time resolution as other files’ modification times. Using `.hg`
/// instead of the system’s default temporary directory (such as `/tmp`) makes
/// it more likely the temporary file is in the same disk partition as contents
/// of the working directory, which can matter since different filesystems may
/// store timestamps with different resolutions.
///
/// This may fail, typically if we lack write permissions. In that case we
/// should continue the `status()` algoritm anyway and consider the current
/// date/time to be unknown.
fn filesystem_now(repo_root: &Path) -> Result<SystemTime, io::Error> {
tempfile::tempfile_in(repo_root.join(".hg"))?
.metadata()?
.modified()
}