forked from mirrors/jj
working_copy: rename to local_working_copy
It's about time we make the working copy a pluggable backend like we have for the other storage. We will use it at Google for at least two reasons: * To support our virtual file system. That will be a completely separate working copy backend, which will interact with the virtual file system to update and snapshot the working copy. * On local disk, we need to tell our build system where to find the paths that are not in the sparse patterns. We plan to do that by wrapping the standard local working copy backend (the one moved in this commit), writing a symlink that points to the mainline commit where the "background" files can be read from. Let's start by renaming the exising implementation to `local_working_copy`.
This commit is contained in:
parent
1eb8b95a6c
commit
187ba9430a
11 changed files with 48 additions and 41 deletions
|
@ -40,6 +40,10 @@ use jj_lib::git_backend::GitBackend;
|
|||
use jj_lib::gitignore::GitIgnoreFile;
|
||||
use jj_lib::hex_util::to_reverse_hex;
|
||||
use jj_lib::id_prefix::IdPrefixContext;
|
||||
use jj_lib::local_working_copy::{
|
||||
CheckoutStats, LocalWorkingCopy, LockedLocalWorkingCopy, ResetError, SnapshotError,
|
||||
SnapshotOptions, TreeStateError,
|
||||
};
|
||||
use jj_lib::matchers::{EverythingMatcher, Matcher, PrefixMatcher, Visit};
|
||||
use jj_lib::merged_tree::{MergedTree, MergedTreeBuilder};
|
||||
use jj_lib::op_heads_store::{self, OpHeadResolutionError, OpHeadsStore};
|
||||
|
@ -58,10 +62,6 @@ use jj_lib::revset::{
|
|||
use jj_lib::settings::{ConfigResultExt as _, UserSettings};
|
||||
use jj_lib::transaction::Transaction;
|
||||
use jj_lib::tree::TreeMergeError;
|
||||
use jj_lib::working_copy::{
|
||||
CheckoutStats, LockedWorkingCopy, ResetError, SnapshotError, SnapshotOptions, TreeStateError,
|
||||
WorkingCopy,
|
||||
};
|
||||
use jj_lib::workspace::{Workspace, WorkspaceInitError, WorkspaceLoadError, WorkspaceLoader};
|
||||
use jj_lib::{dag_walk, file_util, git, revset};
|
||||
use once_cell::unsync::OnceCell;
|
||||
|
@ -846,13 +846,13 @@ impl WorkspaceCommandHelper {
|
|||
&self.user_repo.repo
|
||||
}
|
||||
|
||||
pub fn working_copy(&self) -> &WorkingCopy {
|
||||
pub fn working_copy(&self) -> &LocalWorkingCopy {
|
||||
self.workspace.working_copy()
|
||||
}
|
||||
|
||||
pub fn unchecked_start_working_copy_mutation(
|
||||
&mut self,
|
||||
) -> Result<(LockedWorkingCopy, Commit), CommandError> {
|
||||
) -> Result<(LockedLocalWorkingCopy, Commit), CommandError> {
|
||||
self.check_working_copy_writable()?;
|
||||
let wc_commit = if let Some(wc_commit_id) = self.get_wc_commit_id() {
|
||||
self.repo().store().get_commit(wc_commit_id)?
|
||||
|
@ -867,7 +867,7 @@ impl WorkspaceCommandHelper {
|
|||
|
||||
pub fn start_working_copy_mutation(
|
||||
&mut self,
|
||||
) -> Result<(LockedWorkingCopy, Commit), CommandError> {
|
||||
) -> Result<(LockedLocalWorkingCopy, Commit), CommandError> {
|
||||
let (locked_working_copy, wc_commit) = self.unchecked_start_working_copy_mutation()?;
|
||||
if wc_commit.tree_id() != locked_working_copy.old_tree_id() {
|
||||
return Err(user_error("Concurrent working copy operation. Try again."));
|
||||
|
@ -1716,7 +1716,7 @@ pub enum StaleWorkingCopyError {
|
|||
|
||||
#[instrument(skip_all)]
|
||||
pub fn check_stale_working_copy(
|
||||
locked_wc: &LockedWorkingCopy,
|
||||
locked_wc: &LockedLocalWorkingCopy,
|
||||
wc_commit: &Commit,
|
||||
repo: &ReadonlyRepo,
|
||||
) -> Result<Option<Operation>, StaleWorkingCopyError> {
|
||||
|
@ -1990,7 +1990,7 @@ pub fn resolve_multiple_nonempty_revsets_default_single(
|
|||
|
||||
pub fn update_working_copy(
|
||||
repo: &Arc<ReadonlyRepo>,
|
||||
wc: &mut WorkingCopy,
|
||||
wc: &mut LocalWorkingCopy,
|
||||
old_commit: Option<&Commit>,
|
||||
new_commit: &Commit,
|
||||
) -> Result<Option<CheckoutStats>, CommandError> {
|
||||
|
|
|
@ -35,6 +35,7 @@ use jj_lib::backend::{CommitId, ObjectId, TreeValue};
|
|||
use jj_lib::commit::Commit;
|
||||
use jj_lib::dag_walk::topo_order_reverse;
|
||||
use jj_lib::git_backend::GitBackend;
|
||||
use jj_lib::local_working_copy::SnapshotOptions;
|
||||
use jj_lib::matchers::EverythingMatcher;
|
||||
use jj_lib::merge::Merge;
|
||||
use jj_lib::merged_tree::{MergedTree, MergedTreeBuilder};
|
||||
|
@ -47,7 +48,6 @@ use jj_lib::revset_graph::{
|
|||
};
|
||||
use jj_lib::rewrite::{back_out_commit, merge_commit_trees, rebase_commit, DescendantRebaser};
|
||||
use jj_lib::settings::UserSettings;
|
||||
use jj_lib::working_copy::SnapshotOptions;
|
||||
use jj_lib::workspace::Workspace;
|
||||
use jj_lib::{conflicts, file_util, revset};
|
||||
use maplit::{hashmap, hashset};
|
||||
|
|
|
@ -10,13 +10,13 @@ use itertools::Itertools;
|
|||
use jj_lib::backend::{FileId, MergedTreeId, TreeValue};
|
||||
use jj_lib::conflicts::{self, materialize_merge_result};
|
||||
use jj_lib::gitignore::GitIgnoreFile;
|
||||
use jj_lib::local_working_copy::{CheckoutError, SnapshotOptions, TreeState, TreeStateError};
|
||||
use jj_lib::matchers::Matcher;
|
||||
use jj_lib::merge::Merge;
|
||||
use jj_lib::merged_tree::{MergedTree, MergedTreeBuilder};
|
||||
use jj_lib::repo_path::RepoPath;
|
||||
use jj_lib::settings::UserSettings;
|
||||
use jj_lib::store::Store;
|
||||
use jj_lib::working_copy::{CheckoutError, SnapshotOptions, TreeState, TreeStateError};
|
||||
use regex::{Captures, Regex};
|
||||
use tempfile::TempDir;
|
||||
use thiserror::Error;
|
||||
|
|
|
@ -21,11 +21,11 @@ use config::ConfigError;
|
|||
use jj_lib::backend::MergedTreeId;
|
||||
use jj_lib::conflicts::extract_as_single_hunk;
|
||||
use jj_lib::gitignore::GitIgnoreFile;
|
||||
use jj_lib::local_working_copy::SnapshotError;
|
||||
use jj_lib::matchers::Matcher;
|
||||
use jj_lib::merged_tree::MergedTree;
|
||||
use jj_lib::repo_path::RepoPath;
|
||||
use jj_lib::settings::{ConfigResultExt as _, UserSettings};
|
||||
use jj_lib::working_copy::SnapshotError;
|
||||
use thiserror::Error;
|
||||
|
||||
use self::builtin::{edit_diff_builtin, edit_merge_builtin, BuiltinToolError};
|
||||
|
|
|
@ -41,6 +41,7 @@ pub mod hex_util;
|
|||
pub mod id_prefix;
|
||||
pub mod index;
|
||||
pub mod local_backend;
|
||||
pub mod local_working_copy;
|
||||
pub mod lock;
|
||||
pub mod matchers;
|
||||
pub mod merge;
|
||||
|
@ -66,5 +67,4 @@ pub mod transaction;
|
|||
pub mod tree;
|
||||
pub mod tree_builder;
|
||||
pub mod view;
|
||||
pub mod working_copy;
|
||||
pub mod workspace;
|
||||
|
|
|
@ -1352,7 +1352,7 @@ struct CheckoutState {
|
|||
workspace_id: WorkspaceId,
|
||||
}
|
||||
|
||||
pub struct WorkingCopy {
|
||||
pub struct LocalWorkingCopy {
|
||||
store: Arc<Store>,
|
||||
working_copy_path: PathBuf,
|
||||
state_path: PathBuf,
|
||||
|
@ -1360,7 +1360,7 @@ pub struct WorkingCopy {
|
|||
tree_state: OnceCell<TreeState>,
|
||||
}
|
||||
|
||||
impl WorkingCopy {
|
||||
impl LocalWorkingCopy {
|
||||
/// Initializes a new working copy at `working_copy_path`. The working
|
||||
/// copy's state will be stored in the `state_path` directory. The working
|
||||
/// copy will have the empty tree checked out.
|
||||
|
@ -1370,7 +1370,7 @@ impl WorkingCopy {
|
|||
state_path: PathBuf,
|
||||
operation_id: OperationId,
|
||||
workspace_id: WorkspaceId,
|
||||
) -> Result<WorkingCopy, TreeStateError> {
|
||||
) -> Result<LocalWorkingCopy, TreeStateError> {
|
||||
let proto = crate::protos::working_copy::Checkout {
|
||||
operation_id: operation_id.to_bytes(),
|
||||
workspace_id: workspace_id.as_str().to_string(),
|
||||
|
@ -1384,7 +1384,7 @@ impl WorkingCopy {
|
|||
file.write_all(&proto.encode_to_vec()).unwrap();
|
||||
let tree_state =
|
||||
TreeState::init(store.clone(), working_copy_path.clone(), state_path.clone())?;
|
||||
Ok(WorkingCopy {
|
||||
Ok(LocalWorkingCopy {
|
||||
store,
|
||||
working_copy_path,
|
||||
state_path,
|
||||
|
@ -1393,8 +1393,12 @@ impl WorkingCopy {
|
|||
})
|
||||
}
|
||||
|
||||
pub fn load(store: Arc<Store>, working_copy_path: PathBuf, state_path: PathBuf) -> WorkingCopy {
|
||||
WorkingCopy {
|
||||
pub fn load(
|
||||
store: Arc<Store>,
|
||||
working_copy_path: PathBuf,
|
||||
state_path: PathBuf,
|
||||
) -> LocalWorkingCopy {
|
||||
LocalWorkingCopy {
|
||||
store,
|
||||
working_copy_path,
|
||||
state_path,
|
||||
|
@ -1489,7 +1493,7 @@ impl WorkingCopy {
|
|||
});
|
||||
}
|
||||
|
||||
pub fn start_mutation(&mut self) -> Result<LockedWorkingCopy, TreeStateError> {
|
||||
pub fn start_mutation(&mut self) -> Result<LockedLocalWorkingCopy, TreeStateError> {
|
||||
let lock_path = self.state_path.join("working_copy.lock");
|
||||
let lock = FileLock::lock(lock_path);
|
||||
|
||||
|
@ -1501,7 +1505,7 @@ impl WorkingCopy {
|
|||
let old_operation_id = self.operation_id().clone();
|
||||
let old_tree_id = self.current_tree_id()?.clone();
|
||||
|
||||
Ok(LockedWorkingCopy {
|
||||
Ok(LockedLocalWorkingCopy {
|
||||
wc: self,
|
||||
lock,
|
||||
old_operation_id,
|
||||
|
@ -1542,8 +1546,8 @@ impl WorkingCopy {
|
|||
|
||||
/// A working copy that's locked on disk. The lock is held until you call
|
||||
/// `finish()` or `discard()`.
|
||||
pub struct LockedWorkingCopy<'a> {
|
||||
wc: &'a mut WorkingCopy,
|
||||
pub struct LockedLocalWorkingCopy<'a> {
|
||||
wc: &'a mut LocalWorkingCopy,
|
||||
#[allow(dead_code)]
|
||||
lock: FileLock,
|
||||
old_operation_id: OperationId,
|
||||
|
@ -1552,7 +1556,7 @@ pub struct LockedWorkingCopy<'a> {
|
|||
closed: bool,
|
||||
}
|
||||
|
||||
impl LockedWorkingCopy<'_> {
|
||||
impl LockedLocalWorkingCopy<'_> {
|
||||
/// The operation at the time the lock was taken
|
||||
pub fn old_operation_id(&self) -> &OperationId {
|
||||
&self.old_operation_id
|
||||
|
@ -1627,7 +1631,7 @@ impl LockedWorkingCopy<'_> {
|
|||
}
|
||||
}
|
||||
|
||||
impl Drop for LockedWorkingCopy<'_> {
|
||||
impl Drop for LockedLocalWorkingCopy<'_> {
|
||||
fn drop(&mut self) {
|
||||
if !self.closed {
|
||||
// Undo the changes in memory
|
|
@ -27,6 +27,7 @@ use crate::file_util::{IoResultExt as _, PathError};
|
|||
use crate::git_backend::GitBackend;
|
||||
use crate::index::IndexStore;
|
||||
use crate::local_backend::LocalBackend;
|
||||
use crate::local_working_copy::{LocalWorkingCopy, TreeStateError};
|
||||
use crate::op_heads_store::OpHeadsStore;
|
||||
use crate::op_store::{OpStore, WorkspaceId};
|
||||
use crate::repo::{
|
||||
|
@ -35,7 +36,6 @@ use crate::repo::{
|
|||
};
|
||||
use crate::settings::UserSettings;
|
||||
use crate::submodule_store::SubmoduleStore;
|
||||
use crate::working_copy::{TreeStateError, WorkingCopy};
|
||||
|
||||
#[derive(Error, Debug)]
|
||||
pub enum WorkspaceInitError {
|
||||
|
@ -74,7 +74,7 @@ pub struct Workspace {
|
|||
// working copy files live.
|
||||
workspace_root: PathBuf,
|
||||
repo_loader: RepoLoader,
|
||||
working_copy: WorkingCopy,
|
||||
working_copy: LocalWorkingCopy,
|
||||
}
|
||||
|
||||
fn create_jj_dir(workspace_root: &Path) -> Result<PathBuf, WorkspaceInitError> {
|
||||
|
@ -94,7 +94,7 @@ fn init_working_copy(
|
|||
workspace_root: &Path,
|
||||
jj_dir: &Path,
|
||||
workspace_id: WorkspaceId,
|
||||
) -> Result<(WorkingCopy, Arc<ReadonlyRepo>), WorkspaceInitError> {
|
||||
) -> Result<(LocalWorkingCopy, Arc<ReadonlyRepo>), WorkspaceInitError> {
|
||||
let working_copy_state_path = jj_dir.join("working_copy");
|
||||
std::fs::create_dir(&working_copy_state_path).context(&working_copy_state_path)?;
|
||||
|
||||
|
@ -109,7 +109,7 @@ fn init_working_copy(
|
|||
)?;
|
||||
let repo = tx.commit();
|
||||
|
||||
let working_copy = WorkingCopy::init(
|
||||
let working_copy = LocalWorkingCopy::init(
|
||||
repo.store().clone(),
|
||||
workspace_root.to_path_buf(),
|
||||
working_copy_state_path,
|
||||
|
@ -122,7 +122,7 @@ fn init_working_copy(
|
|||
impl Workspace {
|
||||
fn new(
|
||||
workspace_root: &Path,
|
||||
working_copy: WorkingCopy,
|
||||
working_copy: LocalWorkingCopy,
|
||||
repo_loader: RepoLoader,
|
||||
) -> Result<Workspace, PathError> {
|
||||
let workspace_root = workspace_root.canonicalize().context(workspace_root)?;
|
||||
|
@ -277,11 +277,11 @@ impl Workspace {
|
|||
&self.repo_loader
|
||||
}
|
||||
|
||||
pub fn working_copy(&self) -> &WorkingCopy {
|
||||
pub fn working_copy(&self) -> &LocalWorkingCopy {
|
||||
&self.working_copy
|
||||
}
|
||||
|
||||
pub fn working_copy_mut(&mut self) -> &mut WorkingCopy {
|
||||
pub fn working_copy_mut(&mut self) -> &mut LocalWorkingCopy {
|
||||
&mut self.working_copy
|
||||
}
|
||||
}
|
||||
|
@ -338,7 +338,7 @@ impl WorkspaceLoader {
|
|||
store_factories: &StoreFactories,
|
||||
) -> Result<Workspace, WorkspaceLoadError> {
|
||||
let repo_loader = RepoLoader::init(user_settings, &self.repo_dir, store_factories)?;
|
||||
let working_copy = WorkingCopy::load(
|
||||
let working_copy = LocalWorkingCopy::load(
|
||||
repo_loader.store().clone(),
|
||||
self.workspace_root.clone(),
|
||||
self.working_copy_state_path.clone(),
|
||||
|
|
|
@ -27,13 +27,15 @@ use std::sync::Arc;
|
|||
use itertools::Itertools;
|
||||
use jj_lib::backend::{MergedTreeId, ObjectId, TreeId, TreeValue};
|
||||
use jj_lib::fsmonitor::FsmonitorKind;
|
||||
use jj_lib::local_working_copy::{
|
||||
LocalWorkingCopy, LockedLocalWorkingCopy, SnapshotError, SnapshotOptions,
|
||||
};
|
||||
use jj_lib::merge::Merge;
|
||||
use jj_lib::merged_tree::{MergedTree, MergedTreeBuilder};
|
||||
use jj_lib::op_store::{OperationId, WorkspaceId};
|
||||
use jj_lib::repo::{ReadonlyRepo, Repo};
|
||||
use jj_lib::repo_path::{RepoPath, RepoPathComponent, RepoPathJoin};
|
||||
use jj_lib::settings::UserSettings;
|
||||
use jj_lib::working_copy::{LockedWorkingCopy, SnapshotError, SnapshotOptions, WorkingCopy};
|
||||
use test_case::test_case;
|
||||
use testutils::{create_tree, write_random_commit, TestRepoBackend, TestWorkspace};
|
||||
|
||||
|
@ -402,7 +404,8 @@ fn test_checkout_discard() {
|
|||
// The change should be reflected in the working copy but not saved
|
||||
assert!(!file1_path.to_fs_path(&workspace_root).is_file());
|
||||
assert!(file2_path.to_fs_path(&workspace_root).is_file());
|
||||
let reloaded_wc = WorkingCopy::load(store.clone(), workspace_root.clone(), state_path.clone());
|
||||
let reloaded_wc =
|
||||
LocalWorkingCopy::load(store.clone(), workspace_root.clone(), state_path.clone());
|
||||
assert!(reloaded_wc.file_states().unwrap().contains_key(&file1_path));
|
||||
assert!(!reloaded_wc.file_states().unwrap().contains_key(&file2_path));
|
||||
drop(locked_wc);
|
||||
|
@ -412,7 +415,7 @@ fn test_checkout_discard() {
|
|||
assert!(!wc.file_states().unwrap().contains_key(&file2_path));
|
||||
assert!(!file1_path.to_fs_path(&workspace_root).is_file());
|
||||
assert!(file2_path.to_fs_path(&workspace_root).is_file());
|
||||
let reloaded_wc = WorkingCopy::load(store.clone(), workspace_root, state_path);
|
||||
let reloaded_wc = LocalWorkingCopy::load(store.clone(), workspace_root, state_path);
|
||||
assert!(reloaded_wc.file_states().unwrap().contains_key(&file1_path));
|
||||
assert!(!reloaded_wc.file_states().unwrap().contains_key(&file2_path));
|
||||
}
|
||||
|
@ -827,7 +830,7 @@ fn test_fsmonitor() {
|
|||
testutils::write_working_copy_file(&workspace_root, &ignored_path, "ignored\n");
|
||||
testutils::write_working_copy_file(&workspace_root, &gitignore_path, "to/ignored\n");
|
||||
|
||||
let snapshot = |locked_wc: &mut LockedWorkingCopy, paths: &[&RepoPath]| {
|
||||
let snapshot = |locked_wc: &mut LockedLocalWorkingCopy, paths: &[&RepoPath]| {
|
||||
let fs_paths = paths
|
||||
.iter()
|
||||
.map(|p| p.to_fs_path(&workspace_root))
|
|
@ -16,9 +16,9 @@ use std::cmp::max;
|
|||
use std::thread;
|
||||
|
||||
use assert_matches::assert_matches;
|
||||
use jj_lib::local_working_copy::{CheckoutError, SnapshotOptions};
|
||||
use jj_lib::repo::Repo;
|
||||
use jj_lib::repo_path::RepoPath;
|
||||
use jj_lib::working_copy::{CheckoutError, SnapshotOptions};
|
||||
use jj_lib::workspace::Workspace;
|
||||
use testutils::{create_tree, write_working_copy_file, TestRepo, TestWorkspace};
|
||||
|
|
@ -13,10 +13,10 @@
|
|||
// limitations under the License.
|
||||
|
||||
use itertools::Itertools;
|
||||
use jj_lib::local_working_copy::{CheckoutStats, LocalWorkingCopy};
|
||||
use jj_lib::matchers::EverythingMatcher;
|
||||
use jj_lib::repo::Repo;
|
||||
use jj_lib::repo_path::RepoPath;
|
||||
use jj_lib::working_copy::{CheckoutStats, WorkingCopy};
|
||||
use testutils::{create_tree, TestWorkspace};
|
||||
|
||||
#[test]
|
||||
|
@ -84,7 +84,7 @@ fn test_sparse_checkout() {
|
|||
assert_eq!(wc.sparse_patterns().unwrap(), sparse_patterns);
|
||||
|
||||
// Reload the state to check that it was persisted
|
||||
let mut wc = WorkingCopy::load(
|
||||
let mut wc = LocalWorkingCopy::load(
|
||||
repo.store().clone(),
|
||||
wc.working_copy_path().to_path_buf(),
|
||||
wc.state_path().to_path_buf(),
|
|
@ -23,6 +23,7 @@ use jj_lib::commit::Commit;
|
|||
use jj_lib::commit_builder::CommitBuilder;
|
||||
use jj_lib::git_backend::GitBackend;
|
||||
use jj_lib::local_backend::LocalBackend;
|
||||
use jj_lib::local_working_copy::{SnapshotError, SnapshotOptions};
|
||||
use jj_lib::merged_tree::MergedTree;
|
||||
use jj_lib::repo::{MutableRepo, ReadonlyRepo, Repo, RepoLoader, StoreFactories};
|
||||
use jj_lib::repo_path::RepoPath;
|
||||
|
@ -32,7 +33,6 @@ use jj_lib::store::Store;
|
|||
use jj_lib::transaction::Transaction;
|
||||
use jj_lib::tree::Tree;
|
||||
use jj_lib::tree_builder::TreeBuilder;
|
||||
use jj_lib::working_copy::{SnapshotError, SnapshotOptions};
|
||||
use jj_lib::workspace::Workspace;
|
||||
use tempfile::TempDir;
|
||||
|
||||
|
|
Loading…
Reference in a new issue