Refactor: Make it possible to share a remote worktree (#12775)

Max Brunsfeld created

This PR is an internal refactor in preparation for remote editing. It
restructures the public interface of `Worktree`, reducing the number of
call sites that assume that a worktree is local or remote.

* The Project no longer calls `worktree.as_local_mut().unwrap()` in code
paths related to basic file operations
* Fewer code paths in the app rely on the worktree's `LocalSnapshot`
* Worktree-related RPC message handling is more fully encapsulated by
the `Worktree` type.

to do:
* [x] file manipulation operations
* [x] sending worktree updates when sharing

for later
* opening buffers
* updating open buffers upon worktree changes

Release Notes:

- N/A

Change summary

Cargo.lock                                   |   1 
crates/client/src/client.rs                  |  25 
crates/collab/src/tests/editor_tests.rs      |  10 
crates/collab/src/tests/integration_tests.rs |  39 
crates/project/src/project.rs                | 501 +++----------
crates/project/src/project_tests.rs          |  23 
crates/rpc/src/peer.rs                       |  52 
crates/semantic_index/src/semantic_index.rs  |  12 
crates/settings/src/settings_store.rs        |   4 
crates/terminal_view/src/terminal_view.rs    |   8 
crates/util/src/paths.rs                     |  20 
crates/worktree/Cargo.toml                   |   1 
crates/worktree/src/worktree.rs              | 776 +++++++++++++++------
crates/worktree/src/worktree_settings.rs     |  58 +
crates/worktree/src/worktree_tests.rs        |  26 
15 files changed, 831 insertions(+), 725 deletions(-)

Detailed changes

Cargo.lock 🔗

@@ -12898,7 +12898,6 @@ dependencies = [
  "gpui",
  "http 0.1.0",
  "ignore",
- "itertools 0.11.0",
  "language",
  "log",
  "parking_lot",

crates/client/src/client.rs 🔗

@@ -1429,6 +1429,31 @@ impl Client {
         }
     }
 
+    pub fn request_dynamic(
+        &self,
+        envelope: proto::Envelope,
+        request_type: &'static str,
+    ) -> impl Future<Output = Result<proto::Envelope>> {
+        let client_id = self.id();
+        log::debug!(
+            "rpc request start. client_id:{}. name:{}",
+            client_id,
+            request_type
+        );
+        let response = self
+            .connection_id()
+            .map(|conn_id| self.peer.request_dynamic(conn_id, envelope, request_type));
+        async move {
+            let response = response?.await;
+            log::debug!(
+                "rpc request finish. client_id:{}. name:{}",
+                client_id,
+                request_type
+            );
+            Ok(response?.0)
+        }
+    }
+
     fn respond<T: RequestMessage>(&self, receipt: Receipt<T>, response: T::Response) -> Result<()> {
         log::debug!("rpc respond. client_id:{}. name:{}", self.id(), T::NAME);
         self.peer.respond(receipt, response)

crates/collab/src/tests/editor_tests.rs 🔗

@@ -83,10 +83,7 @@ async fn test_host_disconnect(
     let project_b = client_b.build_dev_server_project(project_id, cx_b).await;
     cx_a.background_executor.run_until_parked();
 
-    assert!(worktree_a.read_with(cx_a, |tree, _| tree
-        .as_local()
-        .unwrap()
-        .has_update_observer()));
+    assert!(worktree_a.read_with(cx_a, |tree, _| tree.has_update_observer()));
 
     let workspace_b = cx_b
         .add_window(|cx| Workspace::new(None, project_b.clone(), client_b.app_state.clone(), cx));
@@ -123,10 +120,7 @@ async fn test_host_disconnect(
 
     project_b.read_with(cx_b, |project, _| project.is_read_only());
 
-    assert!(worktree_a.read_with(cx_a, |tree, _| !tree
-        .as_local()
-        .unwrap()
-        .has_update_observer()));
+    assert!(worktree_a.read_with(cx_a, |tree, _| !tree.has_update_observer()));
 
     // Ensure client B's edited state is reset and that the whole window is blurred.
 

crates/collab/src/tests/integration_tests.rs 🔗

@@ -1378,10 +1378,7 @@ async fn test_unshare_project(
     let project_b = client_b.build_dev_server_project(project_id, cx_b).await;
     executor.run_until_parked();
 
-    assert!(worktree_a.read_with(cx_a, |tree, _| tree
-        .as_local()
-        .unwrap()
-        .has_update_observer()));
+    assert!(worktree_a.read_with(cx_a, |tree, _| tree.has_update_observer()));
 
     project_b
         .update(cx_b, |p, cx| p.open_buffer((worktree_id, "a.txt"), cx))
@@ -1406,10 +1403,7 @@ async fn test_unshare_project(
         .unwrap();
     executor.run_until_parked();
 
-    assert!(worktree_a.read_with(cx_a, |tree, _| !tree
-        .as_local()
-        .unwrap()
-        .has_update_observer()));
+    assert!(worktree_a.read_with(cx_a, |tree, _| !tree.has_update_observer()));
 
     assert!(project_c.read_with(cx_c, |project, _| project.is_disconnected()));
 
@@ -1421,10 +1415,7 @@ async fn test_unshare_project(
     let project_c2 = client_c.build_dev_server_project(project_id, cx_c).await;
     executor.run_until_parked();
 
-    assert!(worktree_a.read_with(cx_a, |tree, _| tree
-        .as_local()
-        .unwrap()
-        .has_update_observer()));
+    assert!(worktree_a.read_with(cx_a, |tree, _| tree.has_update_observer()));
     project_c2
         .update(cx_c, |p, cx| p.open_buffer((worktree_id, "a.txt"), cx))
         .await
@@ -1531,7 +1522,7 @@ async fn test_project_reconnect(
     executor.run_until_parked();
 
     let worktree1_id = worktree_a1.read_with(cx_a, |worktree, _| {
-        assert!(worktree.as_local().unwrap().has_update_observer());
+        assert!(worktree.has_update_observer());
         worktree.id()
     });
     let (worktree_a2, _) = project_a1
@@ -1543,7 +1534,7 @@ async fn test_project_reconnect(
     executor.run_until_parked();
 
     let worktree2_id = worktree_a2.read_with(cx_a, |tree, _| {
-        assert!(tree.as_local().unwrap().has_update_observer());
+        assert!(tree.has_update_observer());
         tree.id()
     });
     executor.run_until_parked();
@@ -1576,9 +1567,7 @@ async fn test_project_reconnect(
         assert_eq!(project.collaborators().len(), 1);
     });
 
-    worktree_a1.read_with(cx_a, |tree, _| {
-        assert!(tree.as_local().unwrap().has_update_observer())
-    });
+    worktree_a1.read_with(cx_a, |tree, _| assert!(tree.has_update_observer()));
 
     // While client A is disconnected, add and remove files from client A's project.
     client_a
@@ -1620,7 +1609,7 @@ async fn test_project_reconnect(
         .await;
 
     let worktree3_id = worktree_a3.read_with(cx_a, |tree, _| {
-        assert!(!tree.as_local().unwrap().has_update_observer());
+        assert!(!tree.has_update_observer());
         tree.id()
     });
     executor.run_until_parked();
@@ -1643,11 +1632,7 @@ async fn test_project_reconnect(
 
     project_a1.read_with(cx_a, |project, cx| {
         assert!(project.is_shared());
-        assert!(worktree_a1
-            .read(cx)
-            .as_local()
-            .unwrap()
-            .has_update_observer());
+        assert!(worktree_a1.read(cx).has_update_observer());
         assert_eq!(
             worktree_a1
                 .read(cx)
@@ -1665,11 +1650,7 @@ async fn test_project_reconnect(
                 "subdir2/i.txt"
             ]
         );
-        assert!(worktree_a3
-            .read(cx)
-            .as_local()
-            .unwrap()
-            .has_update_observer());
+        assert!(worktree_a3.read(cx).has_update_observer());
         assert_eq!(
             worktree_a3
                 .read(cx)
@@ -1750,7 +1731,7 @@ async fn test_project_reconnect(
     executor.run_until_parked();
 
     let worktree4_id = worktree_a4.read_with(cx_a, |tree, _| {
-        assert!(tree.as_local().unwrap().has_update_observer());
+        assert!(tree.has_update_observer());
         tree.id()
     });
     project_a1.update(cx_a, |project, cx| {

crates/project/src/project.rs 🔗

@@ -27,6 +27,7 @@ use futures::{
         oneshot,
     },
     future::{join_all, try_join_all, Shared},
+    prelude::future::BoxFuture,
     select,
     stream::FuturesUnordered,
     AsyncWriteExt, Future, FutureExt, StreamExt, TryFutureExt,
@@ -38,6 +39,7 @@ use gpui::{
     AnyModel, AppContext, AsyncAppContext, BackgroundExecutor, BorrowAppContext, Context, Entity,
     EventEmitter, Model, ModelContext, PromptLevel, SharedString, Task, WeakModel, WindowContext,
 };
+use http::{HttpClient, Url};
 use itertools::Itertools;
 use language::{
     language_settings::{language_settings, FormatOnSave, Formatter, InlayHintKind},
@@ -66,19 +68,16 @@ use postage::watch;
 use prettier_support::{DefaultPrettier, PrettierInstance};
 use project_settings::{LspSettings, ProjectSettings};
 use rand::prelude::*;
-use search_history::SearchHistory;
-use snippet::Snippet;
-use worktree::{CreatedEntry, LocalSnapshot};
-
-use http::{HttpClient, Url};
 use rpc::{ErrorCode, ErrorExt as _};
 use search::SearchQuery;
+use search_history::SearchHistory;
 use serde::Serialize;
 use settings::{watch_config_file, Settings, SettingsLocation, SettingsStore};
 use sha2::{Digest, Sha256};
 use similar::{ChangeTag, TextDiff};
 use smol::channel::{Receiver, Sender};
 use smol::lock::Semaphore;
+use snippet::Snippet;
 use std::{
     borrow::Cow,
     cmp::{self, Ordering},
@@ -111,7 +110,7 @@ use util::{
     },
     post_inc, ResultExt, TryFutureExt as _,
 };
-use worktree::{Snapshot, Traversal};
+use worktree::{CreatedEntry, RemoteWorktreeClient, Snapshot, Traversal};
 
 pub use fs::*;
 pub use language::Location;
@@ -858,7 +857,13 @@ impl Project {
             // That's because Worktree's identifier is entity id, which should probably be changed.
             let mut worktrees = Vec::new();
             for worktree in response.payload.worktrees {
-                let worktree = Worktree::remote(replica_id, worktree, cx);
+                let worktree = Worktree::remote(
+                    remote_id,
+                    replica_id,
+                    worktree,
+                    Box::new(CollabRemoteWorktreeClient(client.clone())),
+                    cx,
+                );
                 worktrees.push(worktree);
             }
 
@@ -1453,47 +1458,9 @@ impl Project {
                 "No worktree for path {project_path:?}"
             ))));
         };
-        if self.is_local() {
-            worktree.update(cx, |worktree, cx| {
-                worktree
-                    .as_local_mut()
-                    .unwrap()
-                    .create_entry(project_path.path, is_directory, cx)
-            })
-        } else {
-            let client = self.client.clone();
-            let project_id = self.remote_id().unwrap();
-            cx.spawn(move |_, mut cx| async move {
-                let response = client
-                    .request(proto::CreateProjectEntry {
-                        worktree_id: project_path.worktree_id.to_proto(),
-                        project_id,
-                        path: project_path.path.to_string_lossy().into(),
-                        is_directory,
-                    })
-                    .await?;
-                match response.entry {
-                    Some(entry) => worktree
-                        .update(&mut cx, |worktree, cx| {
-                            worktree.as_remote_mut().unwrap().insert_entry(
-                                entry,
-                                response.worktree_scan_id as usize,
-                                cx,
-                            )
-                        })?
-                        .await
-                        .map(CreatedEntry::Included),
-                    None => {
-                        let abs_path = worktree.update(&mut cx, |worktree, _| {
-                            worktree
-                                .absolutize(&project_path.path)
-                                .with_context(|| format!("absolutizing {project_path:?}"))
-                        })??;
-                        Ok(CreatedEntry::Excluded { abs_path })
-                    }
-                }
-            })
-        }
+        worktree.update(cx, |worktree, cx| {
+            worktree.create_entry(project_path.path, is_directory, cx)
+        })
     }
 
     pub fn copy_entry(
@@ -1505,41 +1472,9 @@ impl Project {
         let Some(worktree) = self.worktree_for_entry(entry_id, cx) else {
             return Task::ready(Ok(None));
         };
-        let new_path = new_path.into();
-        if self.is_local() {
-            worktree.update(cx, |worktree, cx| {
-                worktree
-                    .as_local_mut()
-                    .unwrap()
-                    .copy_entry(entry_id, new_path, cx)
-            })
-        } else {
-            let client = self.client.clone();
-            let project_id = self.remote_id().unwrap();
-
-            cx.spawn(move |_, mut cx| async move {
-                let response = client
-                    .request(proto::CopyProjectEntry {
-                        project_id,
-                        entry_id: entry_id.to_proto(),
-                        new_path: new_path.to_string_lossy().into(),
-                    })
-                    .await?;
-                match response.entry {
-                    Some(entry) => worktree
-                        .update(&mut cx, |worktree, cx| {
-                            worktree.as_remote_mut().unwrap().insert_entry(
-                                entry,
-                                response.worktree_scan_id as usize,
-                                cx,
-                            )
-                        })?
-                        .await
-                        .map(Some),
-                    None => Ok(None),
-                }
-            })
-        }
+        worktree.update(cx, |worktree, cx| {
+            worktree.copy_entry(entry_id, new_path, cx)
+        })
     }
 
     pub fn rename_entry(
@@ -1551,48 +1486,9 @@ impl Project {
         let Some(worktree) = self.worktree_for_entry(entry_id, cx) else {
             return Task::ready(Err(anyhow!(format!("No worktree for entry {entry_id:?}"))));
         };
-        let new_path = new_path.into();
-        if self.is_local() {
-            worktree.update(cx, |worktree, cx| {
-                worktree
-                    .as_local_mut()
-                    .unwrap()
-                    .rename_entry(entry_id, new_path, cx)
-            })
-        } else {
-            let client = self.client.clone();
-            let project_id = self.remote_id().unwrap();
-
-            cx.spawn(move |_, mut cx| async move {
-                let response = client
-                    .request(proto::RenameProjectEntry {
-                        project_id,
-                        entry_id: entry_id.to_proto(),
-                        new_path: new_path.to_string_lossy().into(),
-                    })
-                    .await?;
-                match response.entry {
-                    Some(entry) => worktree
-                        .update(&mut cx, |worktree, cx| {
-                            worktree.as_remote_mut().unwrap().insert_entry(
-                                entry,
-                                response.worktree_scan_id as usize,
-                                cx,
-                            )
-                        })?
-                        .await
-                        .map(CreatedEntry::Included),
-                    None => {
-                        let abs_path = worktree.update(&mut cx, |worktree, _| {
-                            worktree
-                                .absolutize(&new_path)
-                                .with_context(|| format!("absolutizing {new_path:?}"))
-                        })??;
-                        Ok(CreatedEntry::Excluded { abs_path })
-                    }
-                }
-            })
-        }
+        worktree.update(cx, |worktree, cx| {
+            worktree.rename_entry(entry_id, new_path, cx)
+        })
     }
 
     pub fn delete_entry(
@@ -1602,38 +1498,10 @@ impl Project {
         cx: &mut ModelContext<Self>,
     ) -> Option<Task<Result<()>>> {
         let worktree = self.worktree_for_entry(entry_id, cx)?;
-
         cx.emit(Event::DeletedEntry(entry_id));
-
-        if self.is_local() {
-            worktree.update(cx, |worktree, cx| {
-                worktree
-                    .as_local_mut()
-                    .unwrap()
-                    .delete_entry(entry_id, trash, cx)
-            })
-        } else {
-            let client = self.client.clone();
-            let project_id = self.remote_id().unwrap();
-            Some(cx.spawn(move |_, mut cx| async move {
-                let response = client
-                    .request(proto::DeleteProjectEntry {
-                        project_id,
-                        entry_id: entry_id.to_proto(),
-                        use_trash: trash,
-                    })
-                    .await?;
-                worktree
-                    .update(&mut cx, move |worktree, cx| {
-                        worktree.as_remote_mut().unwrap().delete_entry(
-                            entry_id,
-                            response.worktree_scan_id as usize,
-                            cx,
-                        )
-                    })?
-                    .await
-            }))
-        }
+        worktree.update(cx, |worktree, cx| {
+            worktree.delete_entry(entry_id, trash, cx)
+        })
     }
 
     pub fn expand_entry(
@@ -1643,31 +1511,7 @@ impl Project {
         cx: &mut ModelContext<Self>,
     ) -> Option<Task<Result<()>>> {
         let worktree = self.worktree_for_id(worktree_id, cx)?;
-        if self.is_local() {
-            worktree.update(cx, |worktree, cx| {
-                worktree.as_local_mut().unwrap().expand_entry(entry_id, cx)
-            })
-        } else {
-            let worktree = worktree.downgrade();
-            let request = self.client.request(proto::ExpandProjectEntry {
-                project_id: self.remote_id().unwrap(),
-                entry_id: entry_id.to_proto(),
-            });
-            Some(cx.spawn(move |_, mut cx| async move {
-                let response = request.await?;
-                if let Some(worktree) = worktree.upgrade() {
-                    worktree
-                        .update(&mut cx, |worktree, _| {
-                            worktree
-                                .as_remote_mut()
-                                .unwrap()
-                                .wait_for_snapshot(response.worktree_scan_id as usize)
-                        })?
-                        .await?;
-                }
-                Ok(())
-            }))
-        }
+        worktree.update(cx, |worktree, cx| worktree.expand_entry(entry_id, cx))
     }
 
     pub fn shared(&mut self, project_id: u64, cx: &mut ModelContext<Self>) -> Result<()> {
@@ -1785,18 +1629,12 @@ impl Project {
                                             }
                                         }
 
-                                        worktree.as_local_mut().unwrap().observe_updates(
-                                            project_id,
-                                            cx,
-                                            {
-                                                let client = client.clone();
-                                                move |update| {
-                                                    client
-                                                        .request(update)
-                                                        .map(|result| result.is_ok())
-                                                }
-                                            },
-                                        );
+                                        worktree.observe_updates(project_id, cx, {
+                                            let client = client.clone();
+                                            move |update| {
+                                                client.request(update).map(|result| result.is_ok())
+                                            }
+                                        });
 
                                         anyhow::Ok(())
                                     })?;
@@ -1947,7 +1785,7 @@ impl Project {
             for worktree_handle in self.worktrees.iter_mut() {
                 if let WorktreeHandle::Strong(worktree) = worktree_handle {
                     let is_visible = worktree.update(cx, |worktree, _| {
-                        worktree.as_local_mut().unwrap().stop_observing_updates();
+                        worktree.stop_observing_updates();
                         worktree.is_visible()
                     });
                     if !is_visible {
@@ -2230,21 +2068,20 @@ impl Project {
         cx: &mut ModelContext<Self>,
     ) -> Task<Result<Model<Buffer>>> {
         let load_buffer = worktree.update(cx, |worktree, cx| {
-            let worktree = worktree.as_local_mut().unwrap();
-            let file = worktree.load_file(path.as_ref(), cx);
+            let load_file = worktree.load_file(path.as_ref(), cx);
             let reservation = cx.reserve_model();
             let buffer_id = BufferId::from(reservation.entity_id().as_non_zero_u64());
             cx.spawn(move |_, mut cx| async move {
-                let (file, contents, diff_base) = file.await?;
+                let loaded = load_file.await?;
                 let text_buffer = cx
                     .background_executor()
-                    .spawn(async move { text::Buffer::new(0, buffer_id, contents) })
+                    .spawn(async move { text::Buffer::new(0, buffer_id, loaded.text) })
                     .await;
                 cx.insert_model(reservation, |_| {
                     Buffer::build(
                         text_buffer,
-                        diff_base,
-                        Some(Arc::new(file)),
+                        loaded.diff_base,
+                        Some(loaded.file),
                         Capability::ReadWrite,
                     )
                 })
@@ -2398,10 +2235,11 @@ impl Project {
         };
         let worktree = file.worktree.clone();
         let path = file.path.clone();
-        worktree.update(cx, |worktree, cx| match worktree {
-            Worktree::Local(worktree) => self.save_local_buffer(&worktree, buffer, path, false, cx),
-            Worktree::Remote(_) => self.save_remote_buffer(buffer, None, cx),
-        })
+        if self.is_local() {
+            self.save_local_buffer(worktree, buffer, path, false, cx)
+        } else {
+            self.save_remote_buffer(buffer, None, cx)
+        }
     }
 
     pub fn save_buffer_as(
@@ -2410,26 +2248,21 @@ impl Project {
         path: ProjectPath,
         cx: &mut ModelContext<Self>,
     ) -> Task<Result<()>> {
-        let old_file = File::from_dyn(buffer.read(cx).file())
-            .filter(|f| f.is_local())
-            .cloned();
+        let old_file = File::from_dyn(buffer.read(cx).file()).cloned();
         let Some(worktree) = self.worktree_for_id(path.worktree_id, cx) else {
             return Task::ready(Err(anyhow!("worktree does not exist")));
         };
 
         cx.spawn(move |this, mut cx| async move {
             this.update(&mut cx, |this, cx| {
-                if let Some(old_file) = &old_file {
-                    this.unregister_buffer_from_language_servers(&buffer, old_file, cx);
-                }
-                worktree.update(cx, |worktree, cx| match worktree {
-                    Worktree::Local(worktree) => {
-                        this.save_local_buffer(worktree, buffer.clone(), path.path, true, cx)
+                if this.is_local() {
+                    if let Some(old_file) = &old_file {
+                        this.unregister_buffer_from_language_servers(&buffer, old_file, cx);
                     }
-                    Worktree::Remote(_) => {
-                        this.save_remote_buffer(buffer.clone(), Some(path.to_proto()), cx)
-                    }
-                })
+                    this.save_local_buffer(worktree, buffer.clone(), path.path, true, cx)
+                } else {
+                    this.save_remote_buffer(buffer.clone(), Some(path.to_proto()), cx)
+                }
             })?
             .await?;
 
@@ -2443,70 +2276,39 @@ impl Project {
 
     pub fn save_local_buffer(
         &self,
-        worktree: &LocalWorktree,
+        worktree: Model<Worktree>,
         buffer_handle: Model<Buffer>,
         path: Arc<Path>,
         mut has_changed_file: bool,
-        cx: &mut ModelContext<Worktree>,
+        cx: &mut ModelContext<Self>,
     ) -> Task<Result<()>> {
         let buffer = buffer_handle.read(cx);
-
-        let rpc = self.client.clone();
-        let buffer_id: u64 = buffer.remote_id().into();
-        let project_id = self.remote_id();
-
+        let buffer_id = buffer.remote_id();
+        let text = buffer.as_rope().clone();
+        let line_ending = buffer.line_ending();
+        let version = buffer.version();
         if buffer.file().is_some_and(|file| !file.is_created()) {
             has_changed_file = true;
         }
 
-        let text = buffer.as_rope().clone();
-        let version = buffer.version();
-        let save = worktree.write_file(path.as_ref(), text, buffer.line_ending(), cx);
-        let fs = Arc::clone(&self.fs);
-        let abs_path = worktree.absolutize(&path);
-        let is_private = worktree.is_path_private(&path);
-
-        cx.spawn(move |this, mut cx| async move {
-            let entry = save.await?;
-            let abs_path = abs_path?;
-            let this = this.upgrade().context("worktree dropped")?;
-
-            let (entry_id, mtime, path, is_private) = match entry {
-                Some(entry) => (Some(entry.id), entry.mtime, entry.path, entry.is_private),
-                None => {
-                    let metadata = fs
-                        .metadata(&abs_path)
-                        .await
-                        .with_context(|| {
-                            format!(
-                                "Fetching metadata after saving the excluded buffer {abs_path:?}"
-                            )
-                        })?
-                        .with_context(|| {
-                            format!("Excluded buffer {path:?} got removed during saving")
-                        })?;
-                    (None, Some(metadata.mtime), path, is_private)
-                }
-            };
+        let save = worktree.update(cx, |worktree, cx| {
+            worktree.write_file(path.as_ref(), text, line_ending, cx)
+        });
 
+        let client = self.client.clone();
+        let project_id = self.remote_id();
+        cx.spawn(move |_, mut cx| async move {
+            let new_file = save.await?;
+            let mtime = new_file.mtime;
             if has_changed_file {
-                let new_file = Arc::new(File {
-                    entry_id,
-                    worktree: this,
-                    path,
-                    mtime,
-                    is_local: true,
-                    is_deleted: false,
-                    is_private,
-                });
-
                 if let Some(project_id) = project_id {
-                    rpc.send(proto::UpdateBufferFile {
-                        project_id,
-                        buffer_id,
-                        file: Some(new_file.to_proto()),
-                    })
-                    .log_err();
+                    client
+                        .send(proto::UpdateBufferFile {
+                            project_id,
+                            buffer_id: buffer_id.into(),
+                            file: Some(new_file.to_proto()),
+                        })
+                        .log_err();
                 }
 
                 buffer_handle.update(&mut cx, |buffer, cx| {
@@ -2517,9 +2319,9 @@ impl Project {
             }
 
             if let Some(project_id) = project_id {
-                rpc.send(proto::BufferSaved {
+                client.send(proto::BufferSaved {
                     project_id,
-                    buffer_id,
+                    buffer_id: buffer_id.into(),
                     version: serialize_version(&version),
                     mtime: mtime.map(|time| time.into()),
                 })?;
@@ -2537,7 +2339,7 @@ impl Project {
         &self,
         buffer_handle: Model<Buffer>,
         new_path: Option<proto::ProjectPath>,
-        cx: &mut ModelContext<Worktree>,
+        cx: &mut ModelContext<Self>,
     ) -> Task<Result<()>> {
         let buffer = buffer_handle.read(cx);
         let buffer_id = buffer.remote_id().into();
@@ -2641,7 +2443,6 @@ impl Project {
 
         self.detect_language_for_buffer(buffer, cx);
         self.register_buffer_with_language_servers(buffer, cx);
-        // self.register_buffer_with_copilot(buffer, cx);
         cx.observe_release(buffer, |this, buffer, cx| {
             if let Some(file) = File::from_dyn(buffer.file()) {
                 if file.is_local() {
@@ -2791,16 +2592,6 @@ impl Project {
         });
     }
 
-    // fn register_buffer_with_copilot(
-    //     &self,
-    //     buffer_handle: &Model<Buffer>,
-    //     cx: &mut ModelContext<Self>,
-    // ) {
-    //     if let Some(copilot) = Copilot::global(cx) {
-    //         copilot.update(cx, |copilot, cx| copilot.register_buffer(buffer_handle, cx));
-    //     }
-    // }
-
     async fn send_buffer_ordered_messages(
         this: WeakModel<Self>,
         rx: UnboundedReceiver<BufferOrderedMessage>,
@@ -5521,7 +5312,7 @@ impl Project {
     ) -> Result<Option<Diff>> {
         let working_dir_path = buffer.update(cx, |buffer, cx| {
             let file = File::from_dyn(buffer.file())?;
-            let worktree = file.worktree.read(cx).as_local()?;
+            let worktree = file.worktree.read(cx);
             let mut worktree_path = worktree.abs_path().to_path_buf();
             if worktree.root_entry()?.is_file() {
                 worktree_path.pop();
@@ -5708,9 +5499,6 @@ impl Project {
                 if !worktree.is_visible() {
                     continue;
                 }
-                let Some(worktree) = worktree.as_local() else {
-                    continue;
-                };
                 let worktree_abs_path = worktree.abs_path().clone();
 
                 let (adapter, language, server) = match self.language_servers.get(server_id) {
@@ -5874,15 +5662,14 @@ impl Project {
 
             let worktree_abs_path = if let Some(worktree_abs_path) = self
                 .worktree_for_id(symbol.path.worktree_id, cx)
-                .and_then(|worktree| worktree.read(cx).as_local())
-                .map(|local_worktree| local_worktree.abs_path())
+                .map(|worktree| worktree.read(cx).abs_path())
             {
                 worktree_abs_path
             } else {
                 return Task::ready(Err(anyhow!("worktree not found for symbol")));
             };
 
-            let symbol_abs_path = resolve_path(worktree_abs_path, &symbol.path.path);
+            let symbol_abs_path = resolve_path(&worktree_abs_path, &symbol.path.path);
             let symbol_uri = if let Ok(uri) = lsp::Url::from_file_path(symbol_abs_path) {
                 uri
             } else {
@@ -7234,8 +7021,8 @@ impl Project {
         let snapshots = self
             .visible_worktrees(cx)
             .filter_map(|tree| {
-                let tree = tree.read(cx).as_local()?;
-                Some(tree.snapshot())
+                let tree = tree.read(cx);
+                Some((tree.snapshot(), tree.as_local()?.settings()))
             })
             .collect::<Vec<_>>();
         let include_root = snapshots.len() > 1;
@@ -7243,11 +7030,11 @@ impl Project {
         let background = cx.background_executor().clone();
         let path_count: usize = snapshots
             .iter()
-            .map(|s| {
+            .map(|(snapshot, _)| {
                 if query.include_ignored() {
-                    s.file_count()
+                    snapshot.file_count()
                 } else {
-                    s.visible_file_count()
+                    snapshot.visible_file_count()
                 }
             })
             .sum();
@@ -7403,7 +7190,7 @@ impl Project {
         query: SearchQuery,
         include_root: bool,
         path_count: usize,
-        snapshots: Vec<LocalSnapshot>,
+        snapshots: Vec<(Snapshot, WorktreeSettings)>,
         matching_paths_tx: Sender<SearchMatchCandidate>,
     ) {
         let fs = &fs;
@@ -7459,13 +7246,14 @@ impl Project {
                 }
 
                 if query.include_ignored() {
-                    for snapshot in snapshots {
+                    for (snapshot, settings) in snapshots {
                         for ignored_entry in snapshot.entries(true).filter(|e| e.is_ignored) {
                             let limiter = Arc::clone(&max_concurrent_workers);
                             scope.spawn(async move {
                                 let _guard = limiter.acquire().await;
                                 search_ignored_entry(
                                     snapshot,
+                                    settings,
                                     ignored_entry,
                                     fs,
                                     query,
@@ -8302,7 +8090,7 @@ impl Project {
         changes: &UpdatedEntriesSet,
         cx: &mut ModelContext<Self>,
     ) {
-        if worktree.read(cx).as_local().is_none() {
+        if worktree.read(cx).is_remote() {
             return;
         }
         let project_id = self.remote_id();
@@ -8607,14 +8395,12 @@ impl Project {
         self.worktree_for_id(project_path.worktree_id, cx)?
             .read(cx)
             .as_local()?
-            .snapshot()
             .local_git_repo(&project_path.path)
     }
 
     pub fn get_first_worktree_root_repo(&self, cx: &AppContext) -> Option<Arc<dyn GitRepository>> {
         let worktree = self.visible_worktrees(cx).next()?.read(cx).as_local()?;
         let root_entry = worktree.root_git_entry()?;
-
         worktree.get_local_repo(&root_entry)?.repo().clone().into()
     }
 
@@ -9016,21 +8802,7 @@ impl Project {
             this.worktree_for_id(worktree_id, cx)
                 .ok_or_else(|| anyhow!("worktree not found"))
         })??;
-        let worktree_scan_id = worktree.update(&mut cx, |worktree, _| worktree.scan_id())?;
-        let entry = worktree
-            .update(&mut cx, |worktree, cx| {
-                let worktree = worktree.as_local_mut().unwrap();
-                let path = PathBuf::from(envelope.payload.path);
-                worktree.create_entry(path, envelope.payload.is_directory, cx)
-            })?
-            .await?;
-        Ok(proto::ProjectEntryResponse {
-            entry: match &entry {
-                CreatedEntry::Included(entry) => Some(entry.into()),
-                CreatedEntry::Excluded { .. } => None,
-            },
-            worktree_scan_id: worktree_scan_id as u64,
-        })
+        Worktree::handle_create_entry(worktree, envelope.payload, cx).await
     }
 
     async fn handle_rename_project_entry(
@@ -9044,23 +8816,7 @@ impl Project {
             this.worktree_for_entry(entry_id, cx)
                 .ok_or_else(|| anyhow!("worktree not found"))
         })??;
-        let worktree_scan_id = worktree.update(&mut cx, |worktree, _| worktree.scan_id())?;
-        let entry = worktree
-            .update(&mut cx, |worktree, cx| {
-                let new_path = PathBuf::from(envelope.payload.new_path);
-                worktree
-                    .as_local_mut()
-                    .unwrap()
-                    .rename_entry(entry_id, new_path, cx)
-            })?
-            .await?;
-        Ok(proto::ProjectEntryResponse {
-            entry: match &entry {
-                CreatedEntry::Included(entry) => Some(entry.into()),
-                CreatedEntry::Excluded { .. } => None,
-            },
-            worktree_scan_id: worktree_scan_id as u64,
-        })
+        Worktree::handle_rename_entry(worktree, envelope.payload, cx).await
     }
 
     async fn handle_copy_project_entry(
@@ -9074,20 +8830,7 @@ impl Project {
             this.worktree_for_entry(entry_id, cx)
                 .ok_or_else(|| anyhow!("worktree not found"))
         })??;
-        let worktree_scan_id = worktree.update(&mut cx, |worktree, _| worktree.scan_id())?;
-        let entry = worktree
-            .update(&mut cx, |worktree, cx| {
-                let new_path = PathBuf::from(envelope.payload.new_path);
-                worktree
-                    .as_local_mut()
-                    .unwrap()
-                    .copy_entry(entry_id, new_path, cx)
-            })?
-            .await?;
-        Ok(proto::ProjectEntryResponse {
-            entry: entry.as_ref().map(|e| e.into()),
-            worktree_scan_id: worktree_scan_id as u64,
-        })
+        Worktree::handle_copy_entry(worktree, envelope.payload, cx).await
     }
 
     async fn handle_delete_project_entry(
@@ -9097,28 +8840,12 @@ impl Project {
         mut cx: AsyncAppContext,
     ) -> Result<proto::ProjectEntryResponse> {
         let entry_id = ProjectEntryId::from_proto(envelope.payload.entry_id);
-        let trash = envelope.payload.use_trash;
-
-        this.update(&mut cx, |_, cx| cx.emit(Event::DeletedEntry(entry_id)))?;
-
         let worktree = this.update(&mut cx, |this, cx| {
             this.worktree_for_entry(entry_id, cx)
                 .ok_or_else(|| anyhow!("worktree not found"))
         })??;
-        let worktree_scan_id = worktree.update(&mut cx, |worktree, _| worktree.scan_id())?;
-        worktree
-            .update(&mut cx, |worktree, cx| {
-                worktree
-                    .as_local_mut()
-                    .unwrap()
-                    .delete_entry(entry_id, trash, cx)
-                    .ok_or_else(|| anyhow!("invalid entry"))
-            })??
-            .await?;
-        Ok(proto::ProjectEntryResponse {
-            entry: None,
-            worktree_scan_id: worktree_scan_id as u64,
-        })
+        this.update(&mut cx, |_, cx| cx.emit(Event::DeletedEntry(entry_id)))?;
+        Worktree::handle_delete_entry(worktree, envelope.payload, cx).await
     }
 
     async fn handle_expand_project_entry(
@@ -9131,17 +8858,7 @@ impl Project {
         let worktree = this
             .update(&mut cx, |this, cx| this.worktree_for_entry(entry_id, cx))?
             .ok_or_else(|| anyhow!("invalid request"))?;
-        worktree
-            .update(&mut cx, |worktree, cx| {
-                worktree
-                    .as_local_mut()
-                    .unwrap()
-                    .expand_entry(entry_id, cx)
-                    .ok_or_else(|| anyhow!("invalid entry"))
-            })??
-            .await?;
-        let worktree_scan_id = worktree.update(&mut cx, |worktree, _| worktree.scan_id())? as u64;
-        Ok(proto::ExpandProjectEntryResponse { worktree_scan_id })
+        Worktree::handle_expand_entry(worktree, envelope.payload, cx).await
     }
 
     async fn handle_update_diagnostic_summary(
@@ -10594,6 +10311,7 @@ impl Project {
         cx: &mut ModelContext<Project>,
     ) -> Result<()> {
         let replica_id = self.replica_id();
+        let remote_id = self.remote_id().ok_or_else(|| anyhow!("invalid project"))?;
 
         let mut old_worktrees_by_id = self
             .worktrees
@@ -10610,8 +10328,16 @@ impl Project {
             {
                 self.worktrees.push(WorktreeHandle::Strong(old_worktree));
             } else {
-                let worktree = Worktree::remote(replica_id, worktree, cx);
-                let _ = self.add_worktree(&worktree, cx);
+                self.add_worktree(
+                    &Worktree::remote(
+                        remote_id,
+                        replica_id,
+                        worktree,
+                        Box::new(CollabRemoteWorktreeClient(self.client.clone())),
+                        cx,
+                    ),
+                    cx,
+                );
             }
         }
 
@@ -11374,7 +11100,7 @@ fn deserialize_code_actions(code_actions: &HashMap<String, bool>) -> Vec<lsp::Co
 
 #[allow(clippy::too_many_arguments)]
 async fn search_snapshots(
-    snapshots: &Vec<LocalSnapshot>,
+    snapshots: &Vec<(Snapshot, WorktreeSettings)>,
     worker_start_ix: usize,
     worker_end_ix: usize,
     query: &SearchQuery,
@@ -11386,7 +11112,7 @@ async fn search_snapshots(
     let mut snapshot_start_ix = 0;
     let mut abs_path = PathBuf::new();
 
-    for snapshot in snapshots {
+    for (snapshot, _) in snapshots {
         let snapshot_end_ix = snapshot_start_ix
             + if query.include_ignored() {
                 snapshot.file_count()
@@ -11452,7 +11178,8 @@ async fn search_snapshots(
 }
 
 async fn search_ignored_entry(
-    snapshot: &LocalSnapshot,
+    snapshot: &Snapshot,
+    settings: &WorktreeSettings,
     ignored_entry: &Entry,
     fs: &Arc<dyn Fs>,
     query: &SearchQuery,
@@ -11486,7 +11213,7 @@ async fn search_ignored_entry(
                 }
             } else if !fs_metadata.is_symlink {
                 if !query.file_matches(Some(&ignored_abs_path))
-                    || snapshot.is_path_excluded(&ignored_entry.path)
+                    || settings.is_path_excluded(&ignored_entry.path)
                 {
                     continue;
                 }
@@ -11562,6 +11289,18 @@ impl OpenBuffer {
     }
 }
 
+pub struct CollabRemoteWorktreeClient(Arc<Client>);
+
+impl RemoteWorktreeClient for CollabRemoteWorktreeClient {
+    fn request(
+        &self,
+        envelope: proto::Envelope,
+        request_type: &'static str,
+    ) -> BoxFuture<'static, Result<proto::Envelope>> {
+        self.0.request_dynamic(envelope, request_type).boxed()
+    }
+}
+
 pub struct PathMatchCandidateSet {
     pub snapshot: Snapshot,
     pub include_ignored: bool,

crates/project/src/project_tests.rs 🔗

@@ -2981,21 +2981,26 @@ async fn test_rescan_and_remote_updates(cx: &mut gpui::TestAppContext) {
 
     // Create a remote copy of this worktree.
     let tree = project.update(cx, |project, _| project.worktrees().next().unwrap());
-
-    let metadata = tree.update(cx, |tree, _| tree.as_local().unwrap().metadata_proto());
+    let metadata = tree.update(cx, |tree, _| tree.metadata_proto());
 
     let updates = Arc::new(Mutex::new(Vec::new()));
     tree.update(cx, |tree, cx| {
-        tree.as_local_mut().unwrap().observe_updates(0, cx, {
-            let updates = updates.clone();
-            move |update| {
-                updates.lock().push(update);
-                async { true }
-            }
+        let updates = updates.clone();
+        tree.observe_updates(0, cx, move |update| {
+            updates.lock().push(update);
+            async { true }
         });
     });
 
-    let remote = cx.update(|cx| Worktree::remote(1, metadata, cx));
+    let remote = cx.update(|cx| {
+        Worktree::remote(
+            0,
+            1,
+            metadata,
+            Box::new(CollabRemoteWorktreeClient(project.read(cx).client())),
+            cx,
+        )
+    });
 
     cx.executor().run_until_parked();
 

crates/rpc/src/peer.rs 🔗

@@ -435,6 +435,7 @@ impl Peer {
         self.connections.write().clear();
     }
 
+    /// Make a request and wait for a response.
     pub fn request<T: RequestMessage>(
         &self,
         receiver_id: ConnectionId,
@@ -462,28 +463,50 @@ impl Peer {
             .map_ok(|envelope| envelope.payload)
     }
 
-    pub fn request_internal<T: RequestMessage>(
+    fn request_internal<T: RequestMessage>(
         &self,
         original_sender_id: Option<ConnectionId>,
         receiver_id: ConnectionId,
         request: T,
     ) -> impl Future<Output = Result<TypedEnvelope<T::Response>>> {
+        let envelope = request.into_envelope(0, None, original_sender_id.map(Into::into));
+        let response = self.request_dynamic(receiver_id, envelope, T::NAME);
+        async move {
+            let (response, received_at) = response.await?;
+            Ok(TypedEnvelope {
+                message_id: response.id,
+                sender_id: receiver_id,
+                original_sender_id: response.original_sender_id,
+                payload: T::Response::from_envelope(response)
+                    .ok_or_else(|| anyhow!("received response of the wrong type"))?,
+                received_at,
+            })
+        }
+    }
+
+    /// Make a request and wait for a response.
+    ///
+    /// The caller must make sure to deserialize the response into the request's
+    /// response type. This interface is only useful in trait objects, where
+    /// generics can't be used. If you have a concrete type, use `request`.
+    pub fn request_dynamic(
+        &self,
+        receiver_id: ConnectionId,
+        mut envelope: proto::Envelope,
+        type_name: &'static str,
+    ) -> impl Future<Output = Result<(proto::Envelope, Instant)>> {
         let (tx, rx) = oneshot::channel();
         let send = self.connection_state(receiver_id).and_then(|connection| {
-            let message_id = connection.next_message_id.fetch_add(1, SeqCst);
+            envelope.id = connection.next_message_id.fetch_add(1, SeqCst);
             connection
                 .response_channels
                 .lock()
                 .as_mut()
                 .ok_or_else(|| anyhow!("connection was closed"))?
-                .insert(message_id, tx);
+                .insert(envelope.id, tx);
             connection
                 .outgoing_tx
-                .unbounded_send(proto::Message::Envelope(request.into_envelope(
-                    message_id,
-                    None,
-                    original_sender_id.map(Into::into),
-                )))
+                .unbounded_send(proto::Message::Envelope(envelope))
                 .map_err(|_| anyhow!("connection was closed"))?;
             Ok(())
         });
@@ -491,19 +514,10 @@ impl Peer {
             send?;
             let (response, received_at, _barrier) =
                 rx.await.map_err(|_| anyhow!("connection was closed"))?;
-
             if let Some(proto::envelope::Payload::Error(error)) = &response.payload {
-                Err(RpcError::from_proto(&error, T::NAME))
-            } else {
-                Ok(TypedEnvelope {
-                    message_id: response.id,
-                    sender_id: receiver_id,
-                    original_sender_id: response.original_sender_id,
-                    payload: T::Response::from_envelope(response)
-                        .ok_or_else(|| anyhow!("received response of the wrong type"))?,
-                    received_at,
-                })
+                return Err(RpcError::from_proto(&error, type_name));
             }
+            Ok((response, received_at))
         }
     }
 

crates/semantic_index/src/semantic_index.rs 🔗

@@ -30,7 +30,7 @@ use std::{
     time::{Duration, SystemTime},
 };
 use util::ResultExt;
-use worktree::LocalSnapshot;
+use worktree::Snapshot;
 
 pub use project_index_debug_view::ProjectIndexDebugView;
 
@@ -583,9 +583,9 @@ impl WorktreeIndex {
     }
 
     fn index_entries_changed_on_disk(&self, cx: &AppContext) -> impl Future<Output = Result<()>> {
-        let worktree = self.worktree.read(cx).as_local().unwrap().snapshot();
+        let worktree = self.worktree.read(cx).snapshot();
         let worktree_abs_path = worktree.abs_path().clone();
-        let scan = self.scan_entries(worktree.clone(), cx);
+        let scan = self.scan_entries(worktree, cx);
         let chunk = self.chunk_files(worktree_abs_path, scan.updated_entries, cx);
         let embed = Self::embed_files(self.embedding_provider.clone(), chunk.files, cx);
         let persist = self.persist_embeddings(scan.deleted_entry_ranges, embed.files, cx);
@@ -600,7 +600,7 @@ impl WorktreeIndex {
         updated_entries: UpdatedEntriesSet,
         cx: &AppContext,
     ) -> impl Future<Output = Result<()>> {
-        let worktree = self.worktree.read(cx).as_local().unwrap().snapshot();
+        let worktree = self.worktree.read(cx).snapshot();
         let worktree_abs_path = worktree.abs_path().clone();
         let scan = self.scan_updated_entries(worktree, updated_entries.clone(), cx);
         let chunk = self.chunk_files(worktree_abs_path, scan.updated_entries, cx);
@@ -612,7 +612,7 @@ impl WorktreeIndex {
         }
     }
 
-    fn scan_entries(&self, worktree: LocalSnapshot, cx: &AppContext) -> ScanEntries {
+    fn scan_entries(&self, worktree: Snapshot, cx: &AppContext) -> ScanEntries {
         let (updated_entries_tx, updated_entries_rx) = channel::bounded(512);
         let (deleted_entry_ranges_tx, deleted_entry_ranges_rx) = channel::bounded(128);
         let db_connection = self.db_connection.clone();
@@ -692,7 +692,7 @@ impl WorktreeIndex {
 
     fn scan_updated_entries(
         &self,
-        worktree: LocalSnapshot,
+        worktree: Snapshot,
         updated_entries: UpdatedEntriesSet,
         cx: &AppContext,
     ) -> ScanEntries {

crates/settings/src/settings_store.rs 🔗

@@ -632,8 +632,6 @@ impl SettingsStore {
             }
 
             // If the global settings file changed, reload the global value for the field.
-            project_settings_stack.clear();
-            paths_stack.clear();
             if changed_local_path.is_none() {
                 if let Some(value) = setting_value
                     .load_setting(
@@ -653,6 +651,8 @@ impl SettingsStore {
             }
 
             // Reload the local values for the setting.
+            paths_stack.clear();
+            project_settings_stack.clear();
             for ((root_id, path), local_settings) in &self.raw_local_settings {
                 // Build a stack of all of the local values for that setting.
                 while let Some(prev_entry) = paths_stack.last() {

crates/terminal_view/src/terminal_view.rs 🔗

@@ -1301,13 +1301,7 @@ mod tests {
             .unwrap();
 
         let entry = cx
-            .update(|cx| {
-                wt.update(cx, |wt, cx| {
-                    wt.as_local()
-                        .unwrap()
-                        .create_entry(Path::new(""), is_dir, cx)
-                })
-            })
+            .update(|cx| wt.update(cx, |wt, cx| wt.create_entry(Path::new(""), is_dir, cx)))
             .await
             .unwrap()
             .to_included()

crates/util/src/paths.rs 🔗

@@ -343,36 +343,40 @@ impl<P> PathLikeWithPosition<P> {
 
 #[derive(Clone, Debug)]
 pub struct PathMatcher {
-    maybe_path: PathBuf,
+    source: String,
     glob: GlobMatcher,
 }
 
 impl std::fmt::Display for PathMatcher {
     fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
-        self.maybe_path.to_string_lossy().fmt(f)
+        self.source.fmt(f)
     }
 }
 
 impl PartialEq for PathMatcher {
     fn eq(&self, other: &Self) -> bool {
-        self.maybe_path.eq(&other.maybe_path)
+        self.source.eq(&other.source)
     }
 }
 
 impl Eq for PathMatcher {}
 
 impl PathMatcher {
-    pub fn new(maybe_glob: &str) -> Result<Self, globset::Error> {
+    pub fn new(source: &str) -> Result<Self, globset::Error> {
         Ok(PathMatcher {
-            glob: Glob::new(maybe_glob)?.compile_matcher(),
-            maybe_path: PathBuf::from(maybe_glob),
+            glob: Glob::new(source)?.compile_matcher(),
+            source: String::from(source),
         })
     }
 
+    pub fn source(&self) -> &str {
+        &self.source
+    }
+
     pub fn is_match<P: AsRef<Path>>(&self, other: P) -> bool {
         let other_path = other.as_ref();
-        other_path.starts_with(&self.maybe_path)
-            || other_path.ends_with(&self.maybe_path)
+        other_path.starts_with(Path::new(&self.source))
+            || other_path.ends_with(Path::new(&self.source))
             || self.glob.is_match(other_path)
             || self.check_with_end_separator(other_path)
     }

crates/worktree/Cargo.toml 🔗

@@ -31,7 +31,6 @@ fuzzy.workspace = true
 git.workspace = true
 gpui.workspace = true
 ignore.workspace = true
-itertools.workspace = true
 language.workspace = true
 log.workspace = true
 parking_lot.workspace = true

crates/worktree/src/worktree.rs 🔗

@@ -13,6 +13,7 @@ use futures::{
         mpsc::{self, UnboundedSender},
         oneshot,
     },
+    future::BoxFuture,
     select_biased,
     stream::select,
     task::Poll,
@@ -29,14 +30,13 @@ use gpui::{
     Task,
 };
 use ignore::IgnoreStack;
-use itertools::Itertools;
 use parking_lot::Mutex;
 use postage::{
     barrier,
     prelude::{Sink as _, Stream as _},
     watch,
 };
-use rpc::proto;
+use rpc::proto::{self, EnvelopedMessage as _, RequestMessage};
 use settings::{Settings, SettingsLocation, SettingsStore};
 use smol::channel::{self, Sender};
 use std::{
@@ -58,11 +58,7 @@ use std::{
 };
 use sum_tree::{Bias, Edit, SeekTarget, SumTree, TreeMap, TreeSet};
 use text::{LineEnding, Rope};
-use util::{
-    paths::{PathMatcher, HOME},
-    ResultExt,
-};
-
+use util::{paths::HOME, ResultExt};
 pub use worktree_settings::WorktreeSettings;
 
 #[cfg(feature = "test-support")]
@@ -98,17 +94,25 @@ pub enum CreatedEntry {
     Excluded { abs_path: PathBuf },
 }
 
+pub struct LoadedFile {
+    pub file: Arc<File>,
+    pub text: String,
+    pub diff_base: Option<String>,
+}
+
 pub struct LocalWorktree {
     snapshot: LocalSnapshot,
     scan_requests_tx: channel::Sender<ScanRequest>,
     path_prefixes_to_scan_tx: channel::Sender<Arc<Path>>,
     is_scanning: (watch::Sender<bool>, watch::Receiver<bool>),
     _background_scanner_tasks: Vec<Task<()>>,
-    update_observer: Option<ShareState>,
+    update_observer: Option<UpdateObservationState>,
     fs: Arc<dyn Fs>,
     fs_case_sensitive: bool,
     visible: bool,
     next_entry_id: Arc<AtomicUsize>,
+    settings: WorktreeSettings,
+    share_private_files: bool,
 }
 
 struct ScanRequest {
@@ -119,13 +123,26 @@ struct ScanRequest {
 pub struct RemoteWorktree {
     snapshot: Snapshot,
     background_snapshot: Arc<Mutex<Snapshot>>,
+    project_id: u64,
+    client: Box<dyn RemoteWorktreeClient>,
     updates_tx: Option<UnboundedSender<proto::UpdateWorktree>>,
+    update_observer: Arc<
+        Mutex<Option<Box<dyn Send + FnMut(proto::UpdateWorktree) -> BoxFuture<'static, bool>>>>,
+    >,
     snapshot_subscriptions: VecDeque<(usize, oneshot::Sender<()>)>,
     replica_id: ReplicaId,
     visible: bool,
     disconnected: bool,
 }
 
+pub trait RemoteWorktreeClient {
+    fn request(
+        &self,
+        envelope: proto::Envelope,
+        request_type: &'static str,
+    ) -> BoxFuture<'static, Result<proto::Envelope>>;
+}
+
 #[derive(Clone)]
 pub struct Snapshot {
     id: WorktreeId,
@@ -275,9 +292,6 @@ pub struct LocalSnapshot {
     /// All of the git repositories in the worktree, indexed by the project entry
     /// id of their parent directory.
     git_repositories: TreeMap<ProjectEntryId, LocalRepositoryEntry>,
-    file_scan_exclusions: Vec<PathMatcher>,
-    private_files: Vec<PathMatcher>,
-    share_private_files: bool,
 }
 
 struct BackgroundScannerState {
@@ -333,7 +347,7 @@ enum ScanState {
     },
 }
 
-struct ShareState {
+struct UpdateObservationState {
     snapshots_tx:
         mpsc::UnboundedSender<(LocalSnapshot, UpdatedEntriesSet, UpdatedGitRepositoriesSet)>,
     resume_updates: watch::Sender<()>,
@@ -346,6 +360,8 @@ pub enum Event {
     UpdatedGitRepositories(UpdatedGitRepositoriesSet),
 }
 
+static EMPTY_PATH: &str = "";
+
 impl EventEmitter<Event> for Worktree {}
 
 impl Worktree {
@@ -356,10 +372,7 @@ impl Worktree {
         next_entry_id: Arc<AtomicUsize>,
         cx: &mut AsyncAppContext,
     ) -> Result<Model<Self>> {
-        // After determining whether the root entry is a file or a directory, populate the
-        // snapshot's "root name", which will be used for the purpose of fuzzy matching.
         let abs_path = path.into();
-
         let metadata = fs
             .metadata(&abs_path)
             .await
@@ -373,80 +386,34 @@ impl Worktree {
         });
 
         cx.new_model(move |cx: &mut ModelContext<Worktree>| {
+            let worktree_id = cx.handle().entity_id().as_u64();
+            let settings_location = Some(SettingsLocation {
+                worktree_id: worktree_id as usize,
+                path: Path::new(EMPTY_PATH),
+            });
+
+            let settings = WorktreeSettings::get(settings_location, cx).clone();
             cx.observe_global::<SettingsStore>(move |this, cx| {
                 if let Self::Local(this) = this {
-                    let new_file_scan_exclusions = path_matchers(
-                        WorktreeSettings::get_global(cx)
-                            .file_scan_exclusions
-                            .as_deref(),
-                        "file_scan_exclusions",
-                    );
-                    let new_private_files = path_matchers(
-                        WorktreeSettings::get(Some(settings::SettingsLocation {
-                            worktree_id: cx.handle().entity_id().as_u64() as usize,
-                            path: Path::new("")
-                        }), cx).private_files.as_deref(),
-                        "private_files",
-                    );
-
-                    if new_file_scan_exclusions != this.snapshot.file_scan_exclusions
-                        || new_private_files != this.snapshot.private_files
-                    {
-                        this.snapshot.file_scan_exclusions = new_file_scan_exclusions;
-                        this.snapshot.private_files = new_private_files;
-
-                        log::info!(
-                            "Re-scanning directories, new scan exclude files: {:?}, new dotenv files: {:?}",
-                            this.snapshot
-                                .file_scan_exclusions
-                                .iter()
-                                .map(ToString::to_string)
-                                .collect::<Vec<_>>(),
-                            this.snapshot
-                                .private_files
-                                .iter()
-                                .map(ToString::to_string)
-                                .collect::<Vec<_>>()
-                        );
-
+                    let settings = WorktreeSettings::get(settings_location, cx).clone();
+                    if settings != this.settings {
+                        this.settings = settings;
                         this.restart_background_scanners(cx);
                     }
                 }
             })
             .detach();
 
-            let root_name = abs_path
-                .file_name()
-                .map_or(String::new(), |f| f.to_string_lossy().to_string());
-
             let mut snapshot = LocalSnapshot {
-                file_scan_exclusions: path_matchers(
-                    WorktreeSettings::get_global(cx)
-                        .file_scan_exclusions
-                        .as_deref(),
-                    "file_scan_exclusions",
-                ),
-                private_files: path_matchers(
-                    WorktreeSettings::get(Some(SettingsLocation {
-                        worktree_id: cx.handle().entity_id().as_u64() as usize,
-                        path: Path::new(""),
-                    }), cx).private_files.as_deref(),
-                    "private_files",
-                ),
-                share_private_files: false,
                 ignores_by_parent_abs_path: Default::default(),
                 git_repositories: Default::default(),
-                snapshot: Snapshot {
-                    id: WorktreeId::from_usize(cx.entity_id().as_u64() as usize),
-                    abs_path: abs_path.to_path_buf().into(),
-                    root_name: root_name.clone(),
-                    root_char_bag: root_name.chars().map(|c| c.to_ascii_lowercase()).collect(),
-                    entries_by_path: Default::default(),
-                    entries_by_id: Default::default(),
-                    repository_entries: Default::default(),
-                    scan_id: 1,
-                    completed_scan_id: 0,
-                },
+                snapshot: Snapshot::new(
+                    cx.entity_id().as_u64(),
+                    abs_path
+                        .file_name()
+                        .map_or(String::new(), |f| f.to_string_lossy().to_string()),
+                    abs_path,
+                ),
             };
 
             if let Some(metadata) = metadata {
@@ -456,7 +423,7 @@ impl Worktree {
                         &metadata,
                         &next_entry_id,
                         snapshot.root_char_bag,
-                        None
+                        None,
                     ),
                     fs.as_ref(),
                 );
@@ -464,68 +431,71 @@ impl Worktree {
 
             let (scan_requests_tx, scan_requests_rx) = channel::unbounded();
             let (path_prefixes_to_scan_tx, path_prefixes_to_scan_rx) = channel::unbounded();
-            let task_snapshot = snapshot.clone();
-            Worktree::Local(LocalWorktree {
-                next_entry_id: Arc::clone(&next_entry_id),
+            let mut worktree = LocalWorktree {
+                share_private_files: false,
+                next_entry_id,
                 snapshot,
                 is_scanning: watch::channel_with(true),
                 update_observer: None,
                 scan_requests_tx,
                 path_prefixes_to_scan_tx,
-                _background_scanner_tasks: start_background_scan_tasks(
-                    &abs_path,
-                    task_snapshot,
-                    scan_requests_rx,
-                    path_prefixes_to_scan_rx,
-                    Arc::clone(&next_entry_id),
-                    Arc::clone(&fs),
-                    cx,
-                ),
+                _background_scanner_tasks: Vec::new(),
                 fs,
                 fs_case_sensitive,
                 visible,
-            })
+                settings,
+            };
+            worktree.start_background_scanner(scan_requests_rx, path_prefixes_to_scan_rx, cx);
+            Worktree::Local(worktree)
         })
     }
 
     pub fn remote(
+        project_id: u64,
         replica_id: ReplicaId,
         worktree: proto::WorktreeMetadata,
+        client: Box<dyn RemoteWorktreeClient>,
         cx: &mut AppContext,
     ) -> Model<Self> {
         cx.new_model(|cx: &mut ModelContext<Self>| {
-            let snapshot = Snapshot {
-                id: WorktreeId(worktree.id as usize),
-                abs_path: Arc::from(PathBuf::from(worktree.abs_path)),
-                root_name: worktree.root_name.clone(),
-                root_char_bag: worktree
-                    .root_name
-                    .chars()
-                    .map(|c| c.to_ascii_lowercase())
-                    .collect(),
-                entries_by_path: Default::default(),
-                entries_by_id: Default::default(),
-                repository_entries: Default::default(),
-                scan_id: 1,
-                completed_scan_id: 0,
-            };
+            let snapshot = Snapshot::new(
+                worktree.id,
+                worktree.root_name,
+                Arc::from(PathBuf::from(worktree.abs_path)),
+            );
 
             let (updates_tx, mut updates_rx) = mpsc::unbounded();
             let background_snapshot = Arc::new(Mutex::new(snapshot.clone()));
             let (mut snapshot_updated_tx, mut snapshot_updated_rx) = watch::channel();
+            let update_observer = Arc::new(Mutex::new(None));
+
+            let worktree = RemoteWorktree {
+                client,
+                project_id,
+                replica_id,
+                snapshot,
+                background_snapshot: background_snapshot.clone(),
+                update_observer: update_observer.clone(),
+                updates_tx: Some(updates_tx),
+                snapshot_subscriptions: Default::default(),
+                visible: worktree.visible,
+                disconnected: false,
+            };
 
             cx.background_executor()
-                .spawn({
-                    let background_snapshot = background_snapshot.clone();
-                    async move {
-                        while let Some(update) = updates_rx.next().await {
-                            if let Err(error) =
-                                background_snapshot.lock().apply_remote_update(update)
-                            {
-                                log::error!("error applying worktree update: {}", error);
-                            }
-                            snapshot_updated_tx.send(()).await.ok();
+                .spawn(async move {
+                    while let Some(update) = updates_rx.next().await {
+                        let call = update_observer
+                            .lock()
+                            .as_mut()
+                            .map(|observer| (observer)(update.clone()));
+                        if let Some(call) = call {
+                            call.await;
                         }
+                        if let Err(error) = background_snapshot.lock().apply_remote_update(update) {
+                            log::error!("error applying worktree update: {}", error);
+                        }
+                        snapshot_updated_tx.send(()).await.ok();
                     }
                 })
                 .detach();
@@ -551,15 +521,7 @@ impl Worktree {
             })
             .detach();
 
-            Worktree::Remote(RemoteWorktree {
-                replica_id,
-                snapshot: snapshot.clone(),
-                background_snapshot,
-                updates_tx: Some(updates_tx),
-                snapshot_subscriptions: Default::default(),
-                visible: worktree.visible,
-                disconnected: false,
-            })
+            Worktree::Remote(worktree)
         })
     }
 
@@ -605,8 +567,8 @@ impl Worktree {
 
     pub fn snapshot(&self) -> Snapshot {
         match self {
-            Worktree::Local(worktree) => worktree.snapshot().snapshot,
-            Worktree::Remote(worktree) => worktree.snapshot(),
+            Worktree::Local(worktree) => worktree.snapshot.snapshot.clone(),
+            Worktree::Remote(worktree) => worktree.snapshot.clone(),
         }
     }
 
@@ -617,6 +579,15 @@ impl Worktree {
         }
     }
 
+    pub fn metadata_proto(&self) -> proto::WorktreeMetadata {
+        proto::WorktreeMetadata {
+            id: self.id().to_proto(),
+            root_name: self.root_name().to_string(),
+            visible: self.is_visible(),
+            abs_path: self.abs_path().as_os_str().to_string_lossy().into(),
+        }
+    }
+
     pub fn completed_scan_id(&self) -> usize {
         match self {
             Worktree::Local(worktree) => worktree.snapshot.completed_scan_id,
@@ -649,101 +620,351 @@ impl Worktree {
         let entry = self.root_entry()?;
         Some(File::for_entry(entry.clone(), cx.handle()))
     }
-}
 
-fn start_background_scan_tasks(
-    abs_path: &Path,
-    snapshot: LocalSnapshot,
-    scan_requests_rx: channel::Receiver<ScanRequest>,
-    path_prefixes_to_scan_rx: channel::Receiver<Arc<Path>>,
-    next_entry_id: Arc<AtomicUsize>,
-    fs: Arc<dyn Fs>,
-    cx: &mut ModelContext<'_, Worktree>,
-) -> Vec<Task<()>> {
-    let (scan_states_tx, mut scan_states_rx) = mpsc::unbounded();
-    let background_scanner = cx.background_executor().spawn({
-        let abs_path = if cfg!(target_os = "windows") {
-            abs_path
-                .canonicalize()
-                .unwrap_or_else(|_| abs_path.to_path_buf())
-        } else {
-            abs_path.to_path_buf()
-        };
-        let background = cx.background_executor().clone();
-        async move {
-            let (events, watcher) = fs.watch(&abs_path, FS_WATCH_LATENCY).await;
-            let case_sensitive = fs.is_case_sensitive().await.unwrap_or_else(|e| {
-                log::error!("Failed to determine whether filesystem is case sensitive: {e:#}");
-                true
-            });
+    pub fn observe_updates<F, Fut>(
+        &mut self,
+        project_id: u64,
+        cx: &mut ModelContext<Worktree>,
+        callback: F,
+    ) where
+        F: 'static + Send + Fn(proto::UpdateWorktree) -> Fut,
+        Fut: 'static + Send + Future<Output = bool>,
+    {
+        match self {
+            Worktree::Local(this) => this.observe_updates(project_id, cx, callback),
+            Worktree::Remote(this) => {
+                this.update_observer
+                    .lock()
+                    .replace(Box::new(move |update| callback(update).boxed()));
+            }
+        }
+    }
 
-            let mut scanner = BackgroundScanner {
-                fs,
-                fs_case_sensitive: case_sensitive,
-                status_updates_tx: scan_states_tx,
-                executor: background,
-                scan_requests_rx,
-                path_prefixes_to_scan_rx,
-                next_entry_id,
-                state: Mutex::new(BackgroundScannerState {
-                    prev_snapshot: snapshot.snapshot.clone(),
-                    snapshot,
-                    scanned_dirs: Default::default(),
-                    path_prefixes_to_scan: Default::default(),
-                    paths_to_scan: Default::default(),
-                    removed_entry_ids: Default::default(),
-                    changed_paths: Default::default(),
-                }),
-                phase: BackgroundScannerPhase::InitialScan,
-                watcher,
-            };
+    pub fn stop_observing_updates(&mut self) {
+        match self {
+            Worktree::Local(this) => {
+                this.update_observer.take();
+            }
+            Worktree::Remote(this) => {
+                this.update_observer.lock().take();
+            }
+        }
+    }
 
-            scanner.run(events).await;
+    #[cfg(any(test, feature = "test-support"))]
+    pub fn has_update_observer(&self) -> bool {
+        match self {
+            Worktree::Local(this) => this.update_observer.is_some(),
+            Worktree::Remote(this) => this.update_observer.lock().is_some(),
         }
-    });
-    let scan_state_updater = cx.spawn(|this, mut cx| async move {
-        while let Some((state, this)) = scan_states_rx.next().await.zip(this.upgrade()) {
-            this.update(&mut cx, |this, cx| {
-                let this = this.as_local_mut().unwrap();
-                match state {
-                    ScanState::Started => {
-                        *this.is_scanning.0.borrow_mut() = true;
+    }
+
+    pub fn load_file(
+        &self,
+        path: &Path,
+        cx: &mut ModelContext<Worktree>,
+    ) -> Task<Result<LoadedFile>> {
+        match self {
+            Worktree::Local(this) => this.load_file(path, cx),
+            Worktree::Remote(_) => {
+                Task::ready(Err(anyhow!("remote worktrees can't yet load files")))
+            }
+        }
+    }
+
+    pub fn write_file(
+        &self,
+        path: &Path,
+        text: Rope,
+        line_ending: LineEnding,
+        cx: &mut ModelContext<Worktree>,
+    ) -> Task<Result<Arc<File>>> {
+        match self {
+            Worktree::Local(this) => this.write_file(path, text, line_ending, cx),
+            Worktree::Remote(_) => {
+                Task::ready(Err(anyhow!("remote worktree can't yet write files")))
+            }
+        }
+    }
+
+    pub fn create_entry(
+        &mut self,
+        path: impl Into<Arc<Path>>,
+        is_directory: bool,
+        cx: &mut ModelContext<Worktree>,
+    ) -> Task<Result<CreatedEntry>> {
+        let path = path.into();
+        let worktree_id = self.id();
+        match self {
+            Worktree::Local(this) => this.create_entry(path, is_directory, cx),
+            Worktree::Remote(this) => {
+                let project_id = this.project_id;
+                let request = this.rpc_request(proto::CreateProjectEntry {
+                    worktree_id: worktree_id.to_proto(),
+                    project_id,
+                    path: path.to_string_lossy().into(),
+                    is_directory,
+                });
+                cx.spawn(move |this, mut cx| async move {
+                    let response = request.await?;
+                    match response.entry {
+                        Some(entry) => this
+                            .update(&mut cx, |worktree, cx| {
+                                worktree.as_remote_mut().unwrap().insert_entry(
+                                    entry,
+                                    response.worktree_scan_id as usize,
+                                    cx,
+                                )
+                            })?
+                            .await
+                            .map(CreatedEntry::Included),
+                        None => {
+                            let abs_path = this.update(&mut cx, |worktree, _| {
+                                worktree
+                                    .absolutize(&path)
+                                    .with_context(|| format!("absolutizing {path:?}"))
+                            })??;
+                            Ok(CreatedEntry::Excluded { abs_path })
+                        }
                     }
-                    ScanState::Updated {
-                        snapshot,
-                        changes,
-                        barrier,
-                        scanning,
-                    } => {
-                        *this.is_scanning.0.borrow_mut() = scanning;
-                        this.set_snapshot(snapshot, changes, cx);
-                        drop(barrier);
+                })
+            }
+        }
+    }
+
+    pub fn delete_entry(
+        &mut self,
+        entry_id: ProjectEntryId,
+        trash: bool,
+        cx: &mut ModelContext<Worktree>,
+    ) -> Option<Task<Result<()>>> {
+        match self {
+            Worktree::Local(this) => this.delete_entry(entry_id, trash, cx),
+            Worktree::Remote(this) => {
+                let response = this.rpc_request(proto::DeleteProjectEntry {
+                    project_id: this.project_id,
+                    entry_id: entry_id.to_proto(),
+                    use_trash: trash,
+                });
+                Some(cx.spawn(move |this, mut cx| async move {
+                    let response = response.await?;
+                    this.update(&mut cx, move |worktree, cx| {
+                        worktree.as_remote_mut().unwrap().delete_entry(
+                            entry_id,
+                            response.worktree_scan_id as usize,
+                            cx,
+                        )
+                    })?
+                    .await
+                }))
+            }
+        }
+    }
+
+    pub fn rename_entry(
+        &mut self,
+        entry_id: ProjectEntryId,
+        new_path: impl Into<Arc<Path>>,
+        cx: &mut ModelContext<Self>,
+    ) -> Task<Result<CreatedEntry>> {
+        let new_path = new_path.into();
+        match self {
+            Worktree::Local(this) => this.rename_entry(entry_id, new_path, cx),
+            Worktree::Remote(this) => {
+                let response = this.rpc_request(proto::RenameProjectEntry {
+                    project_id: this.project_id,
+                    entry_id: entry_id.to_proto(),
+                    new_path: new_path.to_string_lossy().into(),
+                });
+                cx.spawn(move |this, mut cx| async move {
+                    let response = response.await?;
+                    match response.entry {
+                        Some(entry) => this
+                            .update(&mut cx, |this, cx| {
+                                this.as_remote_mut().unwrap().insert_entry(
+                                    entry,
+                                    response.worktree_scan_id as usize,
+                                    cx,
+                                )
+                            })?
+                            .await
+                            .map(CreatedEntry::Included),
+                        None => {
+                            let abs_path = this.update(&mut cx, |worktree, _| {
+                                worktree
+                                    .absolutize(&new_path)
+                                    .with_context(|| format!("absolutizing {new_path:?}"))
+                            })??;
+                            Ok(CreatedEntry::Excluded { abs_path })
+                        }
                     }
-                }
-                cx.notify();
-            })
-            .ok();
+                })
+            }
         }
-    });
-    vec![background_scanner, scan_state_updater]
-}
+    }
 
-fn path_matchers(values: Option<&[String]>, context: &'static str) -> Vec<PathMatcher> {
-    values
-        .unwrap_or(&[])
-        .iter()
-        .sorted()
-        .filter_map(|pattern| {
-            PathMatcher::new(pattern)
-                .map(Some)
-                .unwrap_or_else(|e| {
-                    log::error!(
-                        "Skipping pattern {pattern} in `{}` project settings due to parsing error: {e:#}", context
-                    );
-                    None
+    pub fn copy_entry(
+        &mut self,
+        entry_id: ProjectEntryId,
+        new_path: impl Into<Arc<Path>>,
+        cx: &mut ModelContext<Self>,
+    ) -> Task<Result<Option<Entry>>> {
+        let new_path = new_path.into();
+        match self {
+            Worktree::Local(this) => this.copy_entry(entry_id, new_path, cx),
+            Worktree::Remote(this) => {
+                let response = this.rpc_request(proto::CopyProjectEntry {
+                    project_id: this.project_id,
+                    entry_id: entry_id.to_proto(),
+                    new_path: new_path.to_string_lossy().into(),
+                });
+                cx.spawn(move |this, mut cx| async move {
+                    let response = response.await?;
+                    match response.entry {
+                        Some(entry) => this
+                            .update(&mut cx, |worktree, cx| {
+                                worktree.as_remote_mut().unwrap().insert_entry(
+                                    entry,
+                                    response.worktree_scan_id as usize,
+                                    cx,
+                                )
+                            })?
+                            .await
+                            .map(Some),
+                        None => Ok(None),
+                    }
                 })
+            }
+        }
+    }
+
+    pub fn expand_entry(
+        &mut self,
+        entry_id: ProjectEntryId,
+        cx: &mut ModelContext<Worktree>,
+    ) -> Option<Task<Result<()>>> {
+        match self {
+            Worktree::Local(this) => this.expand_entry(entry_id, cx),
+            Worktree::Remote(this) => {
+                let response = this.rpc_request(proto::ExpandProjectEntry {
+                    project_id: this.project_id,
+                    entry_id: entry_id.to_proto(),
+                });
+                Some(cx.spawn(move |this, mut cx| async move {
+                    let response = response.await?;
+                    this.update(&mut cx, |this, _| {
+                        this.as_remote_mut()
+                            .unwrap()
+                            .wait_for_snapshot(response.worktree_scan_id as usize)
+                    })?
+                    .await?;
+                    Ok(())
+                }))
+            }
+        }
+    }
+
+    pub async fn handle_create_entry(
+        this: Model<Self>,
+        request: proto::CreateProjectEntry,
+        mut cx: AsyncAppContext,
+    ) -> Result<proto::ProjectEntryResponse> {
+        let (scan_id, entry) = this.update(&mut cx, |this, cx| {
+            (
+                this.scan_id(),
+                this.create_entry(PathBuf::from(request.path), request.is_directory, cx),
+            )
+        })?;
+        Ok(proto::ProjectEntryResponse {
+            entry: match &entry.await? {
+                CreatedEntry::Included(entry) => Some(entry.into()),
+                CreatedEntry::Excluded { .. } => None,
+            },
+            worktree_scan_id: scan_id as u64,
+        })
+    }
+
+    pub async fn handle_delete_entry(
+        this: Model<Self>,
+        request: proto::DeleteProjectEntry,
+        mut cx: AsyncAppContext,
+    ) -> Result<proto::ProjectEntryResponse> {
+        let (scan_id, task) = this.update(&mut cx, |this, cx| {
+            (
+                this.scan_id(),
+                this.delete_entry(
+                    ProjectEntryId::from_proto(request.entry_id),
+                    request.use_trash,
+                    cx,
+                ),
+            )
+        })?;
+        task.ok_or_else(|| anyhow!("invalid entry"))?.await?;
+        Ok(proto::ProjectEntryResponse {
+            entry: None,
+            worktree_scan_id: scan_id as u64,
+        })
+    }
+
+    pub async fn handle_expand_entry(
+        this: Model<Self>,
+        request: proto::ExpandProjectEntry,
+        mut cx: AsyncAppContext,
+    ) -> Result<proto::ExpandProjectEntryResponse> {
+        let task = this.update(&mut cx, |this, cx| {
+            this.expand_entry(ProjectEntryId::from_proto(request.entry_id), cx)
+        })?;
+        task.ok_or_else(|| anyhow!("no such entry"))?.await?;
+        let scan_id = this.read_with(&cx, |this, _| this.scan_id())?;
+        Ok(proto::ExpandProjectEntryResponse {
+            worktree_scan_id: scan_id as u64,
+        })
+    }
+
+    pub async fn handle_rename_entry(
+        this: Model<Self>,
+        request: proto::RenameProjectEntry,
+        mut cx: AsyncAppContext,
+    ) -> Result<proto::ProjectEntryResponse> {
+        let (scan_id, task) = this.update(&mut cx, |this, cx| {
+            (
+                this.scan_id(),
+                this.rename_entry(
+                    ProjectEntryId::from_proto(request.entry_id),
+                    PathBuf::from(request.new_path),
+                    cx,
+                ),
+            )
+        })?;
+        Ok(proto::ProjectEntryResponse {
+            entry: match &task.await? {
+                CreatedEntry::Included(entry) => Some(entry.into()),
+                CreatedEntry::Excluded { .. } => None,
+            },
+            worktree_scan_id: scan_id as u64,
+        })
+    }
+
+    pub async fn handle_copy_entry(
+        this: Model<Self>,
+        request: proto::CopyProjectEntry,
+        mut cx: AsyncAppContext,
+    ) -> Result<proto::ProjectEntryResponse> {
+        let (scan_id, task) = this.update(&mut cx, |this, cx| {
+            (
+                this.scan_id(),
+                this.copy_entry(
+                    ProjectEntryId::from_proto(request.entry_id),
+                    PathBuf::from(request.new_path),
+                    cx,
+                ),
+            )
+        })?;
+        Ok(proto::ProjectEntryResponse {
+            entry: task.await?.as_ref().map(|e| e.into()),
+            worktree_scan_id: scan_id as u64,
         })
-        .collect()
+    }
 }
 
 impl LocalWorktree {
@@ -751,32 +972,108 @@ impl LocalWorktree {
         path.starts_with(&self.abs_path)
     }
 
+    pub fn is_path_private(&self, path: &Path) -> bool {
+        !self.share_private_files && self.settings.is_path_private(path)
+    }
+
     fn restart_background_scanners(&mut self, cx: &mut ModelContext<Worktree>) {
         let (scan_requests_tx, scan_requests_rx) = channel::unbounded();
         let (path_prefixes_to_scan_tx, path_prefixes_to_scan_rx) = channel::unbounded();
         self.scan_requests_tx = scan_requests_tx;
         self.path_prefixes_to_scan_tx = path_prefixes_to_scan_tx;
-        self._background_scanner_tasks = start_background_scan_tasks(
-            &self.snapshot.abs_path,
-            self.snapshot(),
-            scan_requests_rx,
-            path_prefixes_to_scan_rx,
-            Arc::clone(&self.next_entry_id),
-            Arc::clone(&self.fs),
-            cx,
-        );
+        self.start_background_scanner(scan_requests_rx, path_prefixes_to_scan_rx, cx);
+    }
+
+    fn start_background_scanner(
+        &mut self,
+        scan_requests_rx: channel::Receiver<ScanRequest>,
+        path_prefixes_to_scan_rx: channel::Receiver<Arc<Path>>,
+        cx: &mut ModelContext<Worktree>,
+    ) {
+        let snapshot = self.snapshot();
+        let share_private_files = self.share_private_files;
+        let next_entry_id = self.next_entry_id.clone();
+        let fs = self.fs.clone();
+        let settings = self.settings.clone();
+        let (scan_states_tx, mut scan_states_rx) = mpsc::unbounded();
+        let background_scanner = cx.background_executor().spawn({
+            let abs_path = &snapshot.abs_path;
+            let abs_path = if cfg!(target_os = "windows") {
+                abs_path
+                    .canonicalize()
+                    .unwrap_or_else(|_| abs_path.to_path_buf())
+            } else {
+                abs_path.to_path_buf()
+            };
+            let background = cx.background_executor().clone();
+            async move {
+                let (events, watcher) = fs.watch(&abs_path, FS_WATCH_LATENCY).await;
+                let fs_case_sensitive = fs.is_case_sensitive().await.unwrap_or_else(|e| {
+                    log::error!("Failed to determine whether filesystem is case sensitive: {e:#}");
+                    true
+                });
+
+                let mut scanner = BackgroundScanner {
+                    fs,
+                    fs_case_sensitive,
+                    status_updates_tx: scan_states_tx,
+                    executor: background,
+                    scan_requests_rx,
+                    path_prefixes_to_scan_rx,
+                    next_entry_id,
+                    state: Mutex::new(BackgroundScannerState {
+                        prev_snapshot: snapshot.snapshot.clone(),
+                        snapshot,
+                        scanned_dirs: Default::default(),
+                        path_prefixes_to_scan: Default::default(),
+                        paths_to_scan: Default::default(),
+                        removed_entry_ids: Default::default(),
+                        changed_paths: Default::default(),
+                    }),
+                    phase: BackgroundScannerPhase::InitialScan,
+                    share_private_files,
+                    settings,
+                    watcher,
+                };
+
+                scanner.run(events).await;
+            }
+        });
+        let scan_state_updater = cx.spawn(|this, mut cx| async move {
+            while let Some((state, this)) = scan_states_rx.next().await.zip(this.upgrade()) {
+                this.update(&mut cx, |this, cx| {
+                    let this = this.as_local_mut().unwrap();
+                    match state {
+                        ScanState::Started => {
+                            *this.is_scanning.0.borrow_mut() = true;
+                        }
+                        ScanState::Updated {
+                            snapshot,
+                            changes,
+                            barrier,
+                            scanning,
+                        } => {
+                            *this.is_scanning.0.borrow_mut() = scanning;
+                            this.set_snapshot(snapshot, changes, cx);
+                            drop(barrier);
+                        }
+                    }
+                    cx.notify();
+                })
+                .ok();
+            }
+        });
+        self._background_scanner_tasks = vec![background_scanner, scan_state_updater];
         self.is_scanning = watch::channel_with(true);
     }
 
     fn set_snapshot(
         &mut self,
-        mut new_snapshot: LocalSnapshot,
+        new_snapshot: LocalSnapshot,
         entry_changes: UpdatedEntriesSet,
         cx: &mut ModelContext<Worktree>,
     ) {
         let repo_changes = self.changed_repos(&self.snapshot, &new_snapshot);
-
-        new_snapshot.share_private_files = self.snapshot.share_private_files;
         self.snapshot = new_snapshot;
 
         if let Some(share) = self.update_observer.as_mut() {
@@ -911,24 +1208,25 @@ impl LocalWorktree {
         self.snapshot.clone()
     }
 
-    pub fn metadata_proto(&self) -> proto::WorktreeMetadata {
-        proto::WorktreeMetadata {
-            id: self.id().to_proto(),
-            root_name: self.root_name().to_string(),
-            visible: self.visible,
-            abs_path: self.abs_path().as_os_str().to_string_lossy().into(),
-        }
+    pub fn settings(&self) -> WorktreeSettings {
+        self.settings.clone()
     }
 
-    pub fn load_file(
-        &self,
-        path: &Path,
-        cx: &mut ModelContext<Worktree>,
-    ) -> Task<Result<(File, String, Option<String>)>> {
+    pub fn local_git_repo(&self, path: &Path) -> Option<Arc<dyn GitRepository>> {
+        self.repo_for_path(path)
+            .map(|(_, entry)| entry.repo_ptr.clone())
+    }
+
+    pub fn get_local_repo(&self, repo: &RepositoryEntry) -> Option<&LocalRepositoryEntry> {
+        self.git_repositories.get(&repo.work_directory.0)
+    }
+
+    fn load_file(&self, path: &Path, cx: &mut ModelContext<Worktree>) -> Task<Result<LoadedFile>> {
         let path = Arc::from(path);
         let abs_path = self.absolutize(&path);
         let fs = self.fs.clone();
         let entry = self.refresh_entry(path.clone(), None, cx);
+        let is_private = self.is_path_private(path.as_ref());
 
         cx.spawn(|this, mut cx| async move {
             let abs_path = abs_path?;
@@ -943,7 +1241,7 @@ impl LocalWorktree {
                             let fs = fs.clone();
                             let abs_path = abs_path.clone();
                             async move {
-                                let abs_path_metadata = fs
+                                let metadata = fs
                                     .metadata(&abs_path)
                                     .await
                                     .with_context(|| {
@@ -951,7 +1249,7 @@ impl LocalWorktree {
                                     })
                                     .log_err()
                                     .flatten()?;
-                                if abs_path_metadata.is_dir || abs_path_metadata.is_symlink {
+                                if metadata.is_dir || metadata.is_symlink {
                                     None
                                 } else {
                                     git_repo.load_index_text(&repo_path)

crates/worktree/src/worktree_settings.rs 🔗

@@ -1,10 +1,37 @@
+use std::{path::Path, sync::Arc};
+
 use gpui::AppContext;
 use schemars::JsonSchema;
 use serde::{Deserialize, Serialize};
 use settings::{Settings, SettingsSources};
+use util::paths::PathMatcher;
 
-#[derive(Clone, Default, Serialize, Deserialize, JsonSchema)]
+#[derive(Clone, PartialEq, Eq)]
 pub struct WorktreeSettings {
+    pub file_scan_exclusions: Arc<[PathMatcher]>,
+    pub private_files: Arc<[PathMatcher]>,
+}
+
+impl WorktreeSettings {
+    pub fn is_path_private(&self, path: &Path) -> bool {
+        path.ancestors().any(|ancestor| {
+            self.private_files
+                .iter()
+                .any(|matcher| matcher.is_match(&ancestor))
+        })
+    }
+
+    pub fn is_path_excluded(&self, path: &Path) -> bool {
+        path.ancestors().any(|ancestor| {
+            self.file_scan_exclusions
+                .iter()
+                .any(|matcher| matcher.is_match(&ancestor))
+        })
+    }
+}
+
+#[derive(Clone, Default, Serialize, Deserialize, JsonSchema)]
+pub struct WorktreeSettingsContent {
     /// Completely ignore files matching globs from `file_scan_exclusions`
     ///
     /// Default: [
@@ -28,12 +55,37 @@ pub struct WorktreeSettings {
 impl Settings for WorktreeSettings {
     const KEY: Option<&'static str> = None;
 
-    type FileContent = Self;
+    type FileContent = WorktreeSettingsContent;
 
     fn load(
         sources: SettingsSources<Self::FileContent>,
         _: &mut AppContext,
     ) -> anyhow::Result<Self> {
-        sources.json_merge()
+        let result: WorktreeSettingsContent = sources.json_merge()?;
+        let mut file_scan_exclusions = result.file_scan_exclusions.unwrap_or_default();
+        let mut private_files = result.private_files.unwrap_or_default();
+        file_scan_exclusions.sort();
+        private_files.sort();
+        Ok(Self {
+            file_scan_exclusions: path_matchers(&file_scan_exclusions, "file_scan_exclusions"),
+            private_files: path_matchers(&private_files, "private_files"),
+        })
     }
 }
+
+fn path_matchers(values: &[String], context: &'static str) -> Arc<[PathMatcher]> {
+    values
+        .iter()
+        .filter_map(|pattern| {
+            PathMatcher::new(pattern)
+                .map(Some)
+                .unwrap_or_else(|e| {
+                    log::error!(
+                        "Skipping pattern {pattern} in `{}` project settings due to parsing error: {e:#}", context
+                    );
+                    None
+                })
+        })
+        .collect::<Vec<_>>()
+        .into()
+}

crates/worktree/src/worktree_tests.rs 🔗

@@ -453,11 +453,9 @@ async fn test_open_gitignored_files(cx: &mut TestAppContext) {
     // Open a file that is nested inside of a gitignored directory that
     // has not yet been expanded.
     let prev_read_dir_count = fs.read_dir_call_count();
-    let (file, _, _) = tree
+    let loaded = tree
         .update(cx, |tree, cx| {
-            tree.as_local_mut()
-                .unwrap()
-                .load_file("one/node_modules/b/b1.js".as_ref(), cx)
+            tree.load_file("one/node_modules/b/b1.js".as_ref(), cx)
         })
         .await
         .unwrap();
@@ -483,7 +481,10 @@ async fn test_open_gitignored_files(cx: &mut TestAppContext) {
             ]
         );
 
-        assert_eq!(file.path.as_ref(), Path::new("one/node_modules/b/b1.js"));
+        assert_eq!(
+            loaded.file.path.as_ref(),
+            Path::new("one/node_modules/b/b1.js")
+        );
 
         // Only the newly-expanded directories are scanned.
         assert_eq!(fs.read_dir_call_count() - prev_read_dir_count, 2);
@@ -492,11 +493,9 @@ async fn test_open_gitignored_files(cx: &mut TestAppContext) {
     // Open another file in a different subdirectory of the same
     // gitignored directory.
     let prev_read_dir_count = fs.read_dir_call_count();
-    let (file, _, _) = tree
+    let loaded = tree
         .update(cx, |tree, cx| {
-            tree.as_local_mut()
-                .unwrap()
-                .load_file("one/node_modules/a/a2.js".as_ref(), cx)
+            tree.load_file("one/node_modules/a/a2.js".as_ref(), cx)
         })
         .await
         .unwrap();
@@ -524,7 +523,10 @@ async fn test_open_gitignored_files(cx: &mut TestAppContext) {
             ]
         );
 
-        assert_eq!(file.path.as_ref(), Path::new("one/node_modules/a/a2.js"));
+        assert_eq!(
+            loaded.file.path.as_ref(),
+            Path::new("one/node_modules/a/a2.js")
+        );
 
         // Only the newly-expanded directory is scanned.
         assert_eq!(fs.read_dir_call_count() - prev_read_dir_count, 1);
@@ -844,7 +846,7 @@ async fn test_write_file(cx: &mut TestAppContext) {
     tree.flush_fs_events(cx).await;
 
     tree.update(cx, |tree, cx| {
-        tree.as_local().unwrap().write_file(
+        tree.write_file(
             Path::new("tracked-dir/file.txt"),
             "hello".into(),
             Default::default(),
@@ -854,7 +856,7 @@ async fn test_write_file(cx: &mut TestAppContext) {
     .await
     .unwrap();
     tree.update(cx, |tree, cx| {
-        tree.as_local().unwrap().write_file(
+        tree.write_file(
             Path::new("ignored-dir/file.txt"),
             "world".into(),
             Default::default(),