Merge pull request #2187 from zed-industries/save-untitled-buffer-bugs

Max Brunsfeld created

Fix newly-discovered bugs in saving untitled buffers

Change summary

crates/collab/src/tests/integration_tests.rs            |  41 +
crates/collab/src/tests/randomized_integration_tests.rs |   5 
crates/editor/src/items.rs                              |  48 -
crates/editor/src/multi_buffer.rs                       |  15 
crates/language/src/buffer.rs                           |  84 --
crates/project/src/project.rs                           | 306 ++++++----
crates/project/src/project_tests.rs                     |  68 +
crates/project/src/worktree.rs                          | 148 ++--
8 files changed, 370 insertions(+), 345 deletions(-)

Detailed changes

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

@@ -2244,7 +2244,7 @@ async fn test_propagate_saves_and_fs_changes(
     });
 
     // Edit the buffer as the host and concurrently save as guest B.
-    let save_b = buffer_b.update(cx_b, |buf, cx| buf.save(cx));
+    let save_b = project_b.update(cx_b, |project, cx| project.save_buffer(buffer_b.clone(), cx));
     buffer_a.update(cx_a, |buf, cx| buf.edit([(0..0, "hi-a, ")], None, cx));
     save_b.await.unwrap();
     assert_eq!(
@@ -2313,6 +2313,41 @@ async fn test_propagate_saves_and_fs_changes(
         assert_eq!(buffer.file().unwrap().path().to_str(), Some("file1.js"));
         assert_eq!(&*buffer.language().unwrap().name(), "JavaScript");
     });
+
+    let new_buffer_a = project_a
+        .update(cx_a, |p, cx| p.create_buffer("", None, cx))
+        .unwrap();
+    let new_buffer_id = new_buffer_a.read_with(cx_a, |buffer, _| buffer.remote_id());
+    let new_buffer_b = project_b
+        .update(cx_b, |p, cx| p.open_buffer_by_id(new_buffer_id, cx))
+        .await
+        .unwrap();
+    new_buffer_b.read_with(cx_b, |buffer, _| {
+        assert!(buffer.file().is_none());
+    });
+
+    new_buffer_a.update(cx_a, |buffer, cx| {
+        buffer.edit([(0..0, "ok")], None, cx);
+    });
+    project_a
+        .update(cx_a, |project, cx| {
+            project.save_buffer_as(new_buffer_a.clone(), "/a/file3.rs".into(), cx)
+        })
+        .await
+        .unwrap();
+
+    deterministic.run_until_parked();
+    new_buffer_b.read_with(cx_b, |buffer_b, _| {
+        assert_eq!(
+            buffer_b.file().unwrap().path().as_ref(),
+            Path::new("file3.rs")
+        );
+
+        new_buffer_a.read_with(cx_a, |buffer_a, _| {
+            assert_eq!(buffer_b.saved_mtime(), buffer_a.saved_mtime());
+            assert_eq!(buffer_b.saved_version(), buffer_a.saved_version());
+        });
+    });
 }
 
 #[gpui::test(iterations = 10)]
@@ -2882,7 +2917,9 @@ async fn test_buffer_conflict_after_save(
         assert!(!buf.has_conflict());
     });
 
-    buffer_b.update(cx_b, |buf, cx| buf.save(cx)).await.unwrap();
+    project_b.update(cx_b, |project, cx| project.save_buffer(buffer_b.clone(), cx))
+        .await
+        .unwrap();
     cx_a.foreground().forbid_parking();
     buffer_b.read_with(cx_b, |buffer_b, _| assert!(!buffer_b.is_dirty()));
     buffer_b.read_with(cx_b, |buf, _| {

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

@@ -1064,15 +1064,16 @@ async fn randomly_query_and_mutate_buffers(
             }
         }
         30..=39 if buffer.read_with(cx, |buffer, _| buffer.is_dirty()) => {
-            let (requested_version, save) = buffer.update(cx, |buffer, cx| {
+            let requested_version = buffer.update(cx, |buffer, cx| {
                 log::info!(
                     "{}: saving buffer {} ({:?})",
                     client.username,
                     buffer.remote_id(),
                     buffer.file().unwrap().full_path(cx)
                 );
-                (buffer.version(), buffer.save(cx))
+                buffer.version()
             });
+            let save = project.update(cx, |project, cx| project.save_buffer(buffer, cx));
             let save = cx.background().spawn(async move {
                 let (saved_version, _, _) = save
                     .await

crates/editor/src/items.rs 🔗

@@ -2,12 +2,10 @@ use crate::{
     display_map::ToDisplayPoint, link_go_to_definition::hide_link_definition,
     movement::surrounding_word, persistence::DB, scroll::ScrollAnchor, Anchor, Autoscroll, Editor,
     Event, ExcerptId, ExcerptRange, MultiBuffer, MultiBufferSnapshot, NavigationData, ToPoint as _,
-    FORMAT_TIMEOUT,
 };
 use anyhow::{anyhow, Context, Result};
 use collections::HashSet;
 use futures::future::try_join_all;
-use futures::FutureExt;
 use gpui::{
     elements::*, geometry::vector::vec2f, AppContext, Entity, ModelHandle, MutableAppContext,
     RenderContext, Subscription, Task, View, ViewContext, ViewHandle, WeakViewHandle,
@@ -16,7 +14,7 @@ use language::{
     proto::serialize_anchor as serialize_text_anchor, Bias, Buffer, OffsetRangeExt, Point,
     SelectionGoal,
 };
-use project::{FormatTrigger, Item as _, Project, ProjectPath};
+use project::{Item as _, Project, ProjectPath};
 use rpc::proto::{self, update_view};
 use settings::Settings;
 use smallvec::SmallVec;
@@ -610,32 +608,12 @@ impl Item for Editor {
         cx: &mut ViewContext<Self>,
     ) -> Task<Result<()>> {
         self.report_event("save editor", cx);
-
-        let buffer = self.buffer().clone();
-        let buffers = buffer.read(cx).all_buffers();
-        let mut timeout = cx.background().timer(FORMAT_TIMEOUT).fuse();
-        let format = project.update(cx, |project, cx| {
-            project.format(buffers, true, FormatTrigger::Save, cx)
-        });
-        cx.spawn(|_, mut cx| async move {
-            let transaction = futures::select_biased! {
-                _ = timeout => {
-                    log::warn!("timed out waiting for formatting");
-                    None
-                }
-                transaction = format.log_err().fuse() => transaction,
-            };
-
-            buffer
-                .update(&mut cx, |buffer, cx| {
-                    if let Some(transaction) = transaction {
-                        if !buffer.is_singleton() {
-                            buffer.push_transaction(&transaction.0);
-                        }
-                    }
-
-                    buffer.save(cx)
-                })
+        let format = self.perform_format(project.clone(), cx);
+        let buffers = self.buffer().clone().read(cx).all_buffers();
+        cx.as_mut().spawn(|mut cx| async move {
+            format.await?;
+            project
+                .update(&mut cx, |project, cx| project.save_buffers(buffers, cx))
                 .await?;
             Ok(())
         })
@@ -1159,7 +1137,6 @@ fn path_for_file<'a>(
 mod tests {
     use super::*;
     use gpui::MutableAppContext;
-    use language::RopeFingerprint;
     use std::{
         path::{Path, PathBuf},
         sync::Arc,
@@ -1205,17 +1182,6 @@ mod tests {
             todo!()
         }
 
-        fn save(
-            &self,
-            _: u64,
-            _: language::Rope,
-            _: clock::Global,
-            _: project::LineEnding,
-            _: &mut MutableAppContext,
-        ) -> gpui::Task<anyhow::Result<(clock::Global, RopeFingerprint, SystemTime)>> {
-            todo!()
-        }
-
         fn as_any(&self) -> &dyn std::any::Any {
             todo!()
         }

crates/editor/src/multi_buffer.rs 🔗

@@ -1,7 +1,6 @@
 mod anchor;
 
 pub use anchor::{Anchor, AnchorRangeExt};
-use anyhow::Result;
 use clock::ReplicaId;
 use collections::{BTreeMap, Bound, HashMap, HashSet};
 use futures::{channel::mpsc, SinkExt};
@@ -1288,20 +1287,6 @@ impl MultiBuffer {
             .map(|state| state.buffer.clone())
     }
 
-    pub fn save(&mut self, cx: &mut ModelContext<Self>) -> Task<Result<()>> {
-        let mut save_tasks = Vec::new();
-        for BufferState { buffer, .. } in self.buffers.borrow().values() {
-            save_tasks.push(buffer.update(cx, |buffer, cx| buffer.save(cx)));
-        }
-
-        cx.spawn(|_, _| async move {
-            for save in save_tasks {
-                save.await?;
-            }
-            Ok(())
-        })
-    }
-
     pub fn is_completion_trigger<T>(&self, position: T, text: &str, cx: &AppContext) -> bool
     where
         T: ToOffset,

crates/language/src/buffer.rs 🔗

@@ -214,15 +214,6 @@ pub trait File: Send + Sync {
 
     fn is_deleted(&self) -> bool;
 
-    fn save(
-        &self,
-        buffer_id: u64,
-        text: Rope,
-        version: clock::Global,
-        line_ending: LineEnding,
-        cx: &mut MutableAppContext,
-    ) -> Task<Result<(clock::Global, RopeFingerprint, SystemTime)>>;
-
     fn as_any(&self) -> &dyn Any;
 
     fn to_proto(&self) -> rpc::proto::File;
@@ -529,33 +520,6 @@ impl Buffer {
         self.file.as_ref()
     }
 
-    pub fn save(
-        &mut self,
-        cx: &mut ModelContext<Self>,
-    ) -> Task<Result<(clock::Global, RopeFingerprint, SystemTime)>> {
-        let file = if let Some(file) = self.file.as_ref() {
-            file
-        } else {
-            return Task::ready(Err(anyhow!("buffer has no file")));
-        };
-        let text = self.as_rope().clone();
-        let version = self.version();
-        let save = file.save(
-            self.remote_id(),
-            text,
-            version,
-            self.line_ending(),
-            cx.as_mut(),
-        );
-        cx.spawn(|this, mut cx| async move {
-            let (version, fingerprint, mtime) = save.await?;
-            this.update(&mut cx, |this, cx| {
-                this.did_save(version.clone(), fingerprint, mtime, None, cx);
-            });
-            Ok((version, fingerprint, mtime))
-        })
-    }
-
     pub fn saved_version(&self) -> &clock::Global {
         &self.saved_version
     }
@@ -585,16 +549,11 @@ impl Buffer {
         version: clock::Global,
         fingerprint: RopeFingerprint,
         mtime: SystemTime,
-        new_file: Option<Arc<dyn File>>,
         cx: &mut ModelContext<Self>,
     ) {
         self.saved_version = version;
         self.saved_version_fingerprint = fingerprint;
         self.saved_mtime = mtime;
-        if let Some(new_file) = new_file {
-            self.file = Some(new_file);
-            self.file_update_count += 1;
-        }
         cx.emit(Event::Saved);
         cx.notify();
     }
@@ -661,36 +620,35 @@ impl Buffer {
         new_file: Arc<dyn File>,
         cx: &mut ModelContext<Self>,
     ) -> Task<()> {
-        let old_file = if let Some(file) = self.file.as_ref() {
-            file
-        } else {
-            return Task::ready(());
-        };
         let mut file_changed = false;
         let mut task = Task::ready(());
 
-        if new_file.path() != old_file.path() {
-            file_changed = true;
-        }
-
-        if new_file.is_deleted() {
-            if !old_file.is_deleted() {
+        if let Some(old_file) = self.file.as_ref() {
+            if new_file.path() != old_file.path() {
                 file_changed = true;
-                if !self.is_dirty() {
-                    cx.emit(Event::DirtyChanged);
-                }
             }
-        } else {
-            let new_mtime = new_file.mtime();
-            if new_mtime != old_file.mtime() {
-                file_changed = true;
 
-                if !self.is_dirty() {
-                    let reload = self.reload(cx).log_err().map(drop);
-                    task = cx.foreground().spawn(reload);
+            if new_file.is_deleted() {
+                if !old_file.is_deleted() {
+                    file_changed = true;
+                    if !self.is_dirty() {
+                        cx.emit(Event::DirtyChanged);
+                    }
+                }
+            } else {
+                let new_mtime = new_file.mtime();
+                if new_mtime != old_file.mtime() {
+                    file_changed = true;
+
+                    if !self.is_dirty() {
+                        let reload = self.reload(cx).log_err().map(drop);
+                        task = cx.foreground().spawn(reload);
+                    }
                 }
             }
-        }
+        } else {
+            file_changed = true;
+        };
 
         if file_changed {
             self.file_update_count += 1;

crates/project/src/project.rs 🔗

@@ -12,7 +12,7 @@ use clock::ReplicaId;
 use collections::{hash_map, BTreeMap, HashMap, HashSet};
 use futures::{
     channel::{mpsc, oneshot},
-    future::Shared,
+    future::{try_join_all, Shared},
     AsyncWriteExt, Future, FutureExt, StreamExt, TryFutureExt,
 };
 use gpui::{
@@ -28,8 +28,8 @@ use language::{
     range_from_lsp, range_to_lsp, Anchor, Bias, Buffer, CachedLspAdapter, CharKind, CodeAction,
     CodeLabel, Completion, Diagnostic, DiagnosticEntry, DiagnosticSet, Event as BufferEvent,
     File as _, Language, LanguageRegistry, LanguageServerName, LocalFile, OffsetRangeExt,
-    Operation, Patch, PointUtf16, TextBufferSnapshot, ToOffset, ToPointUtf16, Transaction,
-    Unclipped,
+    Operation, Patch, PointUtf16, RopeFingerprint, TextBufferSnapshot, ToOffset, ToPointUtf16,
+    Transaction, Unclipped,
 };
 use lsp::{
     DiagnosticSeverity, DiagnosticTag, DocumentHighlightKind, LanguageServer, LanguageString,
@@ -59,7 +59,7 @@ use std::{
         atomic::{AtomicUsize, Ordering::SeqCst},
         Arc,
     },
-    time::{Duration, Instant},
+    time::{Duration, Instant, SystemTime},
 };
 use terminal::{Terminal, TerminalBuilder};
 use util::{debug_panic, defer, post_inc, ResultExt, TryFutureExt as _};
@@ -554,11 +554,13 @@ impl Project {
             });
         }
 
-        let languages = Arc::new(LanguageRegistry::test());
+        let mut languages = LanguageRegistry::test();
+        languages.set_executor(cx.background());
         let http_client = client::test::FakeHttpClient::with_404_response();
         let client = cx.update(|cx| client::Client::new(http_client.clone(), cx));
         let user_store = cx.add_model(|cx| UserStore::new(client.clone(), http_client, cx));
-        let project = cx.update(|cx| Project::local(client, user_store, languages, fs, cx));
+        let project =
+            cx.update(|cx| Project::local(client, user_store, Arc::new(languages), fs, cx));
         for path in root_paths {
             let (tree, _) = project
                 .update(cx, |project, cx| {
@@ -1426,11 +1428,41 @@ impl Project {
         }
     }
 
+    pub fn save_buffers(
+        &self,
+        buffers: HashSet<ModelHandle<Buffer>>,
+        cx: &mut ModelContext<Self>,
+    ) -> Task<Result<()>> {
+        cx.spawn(|this, mut cx| async move {
+            let save_tasks = buffers
+                .into_iter()
+                .map(|buffer| this.update(&mut cx, |this, cx| this.save_buffer(buffer, cx)));
+            try_join_all(save_tasks).await?;
+            Ok(())
+        })
+    }
+
+    pub fn save_buffer(
+        &self,
+        buffer: ModelHandle<Buffer>,
+        cx: &mut ModelContext<Self>,
+    ) -> Task<Result<(clock::Global, RopeFingerprint, SystemTime)>> {
+        let Some(file) = File::from_dyn(buffer.read(cx).file()) else {
+            return Task::ready(Err(anyhow!("buffer doesn't have a file")));
+        };
+        let worktree = file.worktree.clone();
+        let path = file.path.clone();
+        worktree.update(cx, |worktree, cx| match worktree {
+            Worktree::Local(worktree) => worktree.save_buffer(buffer, path, false, cx),
+            Worktree::Remote(worktree) => worktree.save_buffer(buffer, cx),
+        })
+    }
+
     pub fn save_buffer_as(
         &mut self,
         buffer: ModelHandle<Buffer>,
         abs_path: PathBuf,
-        cx: &mut ModelContext<Project>,
+        cx: &mut ModelContext<Self>,
     ) -> Task<Result<()>> {
         let worktree_task = self.find_or_create_local_worktree(&abs_path, true, cx);
         let old_path =
@@ -1443,11 +1475,11 @@ impl Project {
             }
             let (worktree, path) = worktree_task.await?;
             worktree
-                .update(&mut cx, |worktree, cx| {
-                    worktree
-                        .as_local_mut()
-                        .unwrap()
-                        .save_buffer_as(buffer.clone(), path, cx)
+                .update(&mut cx, |worktree, cx| match worktree {
+                    Worktree::Local(worktree) => {
+                        worktree.save_buffer(buffer.clone(), path.into(), true, cx)
+                    }
+                    Worktree::Remote(_) => panic!("cannot remote buffers as new files"),
                 })
                 .await?;
             this.update(&mut cx, |this, cx| {
@@ -1789,20 +1821,22 @@ impl Project {
             while let Some(()) = subscription.next().await {
                 if let Some(project) = project.upgrade(&cx) {
                     project.update(&mut cx, |project, cx| {
-                        let mut buffers_without_language = Vec::new();
+                        let mut plain_text_buffers = Vec::new();
                         let mut buffers_with_unknown_injections = Vec::new();
                         for buffer in project.opened_buffers.values() {
                             if let Some(handle) = buffer.upgrade(cx) {
                                 let buffer = &handle.read(cx);
-                                if buffer.language().is_none() {
-                                    buffers_without_language.push(handle);
+                                if buffer.language().is_none()
+                                    || buffer.language() == Some(&*language::PLAIN_TEXT)
+                                {
+                                    plain_text_buffers.push(handle);
                                 } else if buffer.contains_unknown_injections() {
                                     buffers_with_unknown_injections.push(handle);
                                 }
                             }
                         }
 
-                        for buffer in buffers_without_language {
+                        for buffer in plain_text_buffers {
                             project.assign_language_to_buffer(&buffer, cx);
                             project.register_buffer_with_language_server(&buffer, cx);
                         }
@@ -2818,126 +2852,126 @@ impl Project {
         trigger: FormatTrigger,
         cx: &mut ModelContext<Project>,
     ) -> Task<Result<ProjectTransaction>> {
-        let mut local_buffers = Vec::new();
-        let mut remote_buffers = None;
-        for buffer_handle in buffers {
-            let buffer = buffer_handle.read(cx);
-            if let Some(file) = File::from_dyn(buffer.file()) {
-                if let Some(buffer_abs_path) = file.as_local().map(|f| f.abs_path(cx)) {
-                    if let Some((_, server)) = self.language_server_for_buffer(buffer, cx) {
-                        local_buffers.push((buffer_handle, buffer_abs_path, server.clone()));
-                    }
-                } else {
-                    remote_buffers.get_or_insert(Vec::new()).push(buffer_handle);
-                }
-            } else {
-                return Task::ready(Ok(Default::default()));
-            }
-        }
-
-        let remote_buffers = self.remote_id().zip(remote_buffers);
-        let client = self.client.clone();
-
-        cx.spawn(|this, mut cx| async move {
-            let mut project_transaction = ProjectTransaction::default();
-
-            if let Some((project_id, remote_buffers)) = remote_buffers {
-                let response = client
-                    .request(proto::FormatBuffers {
-                        project_id,
-                        trigger: trigger as i32,
-                        buffer_ids: remote_buffers
-                            .iter()
-                            .map(|buffer| buffer.read_with(&cx, |buffer, _| buffer.remote_id()))
-                            .collect(),
-                    })
-                    .await?
-                    .transaction
-                    .ok_or_else(|| anyhow!("missing transaction"))?;
-                project_transaction = this
-                    .update(&mut cx, |this, cx| {
-                        this.deserialize_project_transaction(response, push_to_history, cx)
-                    })
-                    .await?;
-            }
+        if self.is_local() {
+            let mut buffers_with_paths_and_servers = buffers
+                .into_iter()
+                .filter_map(|buffer_handle| {
+                    let buffer = buffer_handle.read(cx);
+                    let file = File::from_dyn(buffer.file())?;
+                    let buffer_abs_path = file.as_local()?.abs_path(cx);
+                    let (_, server) = self.language_server_for_buffer(buffer, cx)?;
+                    Some((buffer_handle, buffer_abs_path, server.clone()))
+                })
+                .collect::<Vec<_>>();
 
-            // Do not allow multiple concurrent formatting requests for the
-            // same buffer.
-            this.update(&mut cx, |this, _| {
-                local_buffers
-                    .retain(|(buffer, _, _)| this.buffers_being_formatted.insert(buffer.id()));
-            });
-            let _cleanup = defer({
-                let this = this.clone();
-                let mut cx = cx.clone();
-                let local_buffers = &local_buffers;
-                move || {
-                    this.update(&mut cx, |this, _| {
-                        for (buffer, _, _) in local_buffers {
-                            this.buffers_being_formatted.remove(&buffer.id());
-                        }
-                    });
-                }
-            });
+            cx.spawn(|this, mut cx| async move {
+                // Do not allow multiple concurrent formatting requests for the
+                // same buffer.
+                this.update(&mut cx, |this, _| {
+                    buffers_with_paths_and_servers
+                        .retain(|(buffer, _, _)| this.buffers_being_formatted.insert(buffer.id()));
+                });
 
-            for (buffer, buffer_abs_path, language_server) in &local_buffers {
-                let (format_on_save, formatter, tab_size) = buffer.read_with(&cx, |buffer, cx| {
-                    let settings = cx.global::<Settings>();
-                    let language_name = buffer.language().map(|language| language.name());
-                    (
-                        settings.format_on_save(language_name.as_deref()),
-                        settings.formatter(language_name.as_deref()),
-                        settings.tab_size(language_name.as_deref()),
-                    )
+                let _cleanup = defer({
+                    let this = this.clone();
+                    let mut cx = cx.clone();
+                    let local_buffers = &buffers_with_paths_and_servers;
+                    move || {
+                        this.update(&mut cx, |this, _| {
+                            for (buffer, _, _) in local_buffers {
+                                this.buffers_being_formatted.remove(&buffer.id());
+                            }
+                        });
+                    }
                 });
 
-                let transaction = match (formatter, format_on_save) {
-                    (_, FormatOnSave::Off) if trigger == FormatTrigger::Save => continue,
-
-                    (Formatter::LanguageServer, FormatOnSave::On | FormatOnSave::Off)
-                    | (_, FormatOnSave::LanguageServer) => Self::format_via_lsp(
-                        &this,
-                        &buffer,
-                        &buffer_abs_path,
-                        &language_server,
-                        tab_size,
-                        &mut cx,
-                    )
-                    .await
-                    .context("failed to format via language server")?,
+                let mut project_transaction = ProjectTransaction::default();
+                for (buffer, buffer_abs_path, language_server) in &buffers_with_paths_and_servers {
+                    let (format_on_save, formatter, tab_size) =
+                        buffer.read_with(&cx, |buffer, cx| {
+                            let settings = cx.global::<Settings>();
+                            let language_name = buffer.language().map(|language| language.name());
+                            (
+                                settings.format_on_save(language_name.as_deref()),
+                                settings.formatter(language_name.as_deref()),
+                                settings.tab_size(language_name.as_deref()),
+                            )
+                        });
 
-                    (
-                        Formatter::External { command, arguments },
-                        FormatOnSave::On | FormatOnSave::Off,
-                    )
-                    | (_, FormatOnSave::External { command, arguments }) => {
-                        Self::format_via_external_command(
+                    let transaction = match (formatter, format_on_save) {
+                        (_, FormatOnSave::Off) if trigger == FormatTrigger::Save => continue,
+
+                        (Formatter::LanguageServer, FormatOnSave::On | FormatOnSave::Off)
+                        | (_, FormatOnSave::LanguageServer) => Self::format_via_lsp(
+                            &this,
                             &buffer,
                             &buffer_abs_path,
-                            &command,
-                            &arguments,
+                            &language_server,
+                            tab_size,
                             &mut cx,
                         )
                         .await
-                        .context(format!(
-                            "failed to format via external command {:?}",
-                            command
-                        ))?
-                    }
-                };
+                        .context("failed to format via language server")?,
 
-                if let Some(transaction) = transaction {
-                    if !push_to_history {
-                        buffer.update(&mut cx, |buffer, _| {
-                            buffer.forget_transaction(transaction.id)
-                        });
+                        (
+                            Formatter::External { command, arguments },
+                            FormatOnSave::On | FormatOnSave::Off,
+                        )
+                        | (_, FormatOnSave::External { command, arguments }) => {
+                            Self::format_via_external_command(
+                                &buffer,
+                                &buffer_abs_path,
+                                &command,
+                                &arguments,
+                                &mut cx,
+                            )
+                            .await
+                            .context(format!(
+                                "failed to format via external command {:?}",
+                                command
+                            ))?
+                        }
+                    };
+
+                    if let Some(transaction) = transaction {
+                        if !push_to_history {
+                            buffer.update(&mut cx, |buffer, _| {
+                                buffer.forget_transaction(transaction.id)
+                            });
+                        }
+                        project_transaction.0.insert(buffer.clone(), transaction);
                     }
-                    project_transaction.0.insert(buffer.clone(), transaction);
                 }
-            }
 
-            Ok(project_transaction)
-        })
+                Ok(project_transaction)
+            })
+        } else {
+            let remote_id = self.remote_id();
+            let client = self.client.clone();
+            cx.spawn(|this, mut cx| async move {
+                let mut project_transaction = ProjectTransaction::default();
+                if let Some(project_id) = remote_id {
+                    let response = client
+                        .request(proto::FormatBuffers {
+                            project_id,
+                            trigger: trigger as i32,
+                            buffer_ids: buffers
+                                .iter()
+                                .map(|buffer| buffer.read_with(&cx, |buffer, _| buffer.remote_id()))
+                                .collect(),
+                        })
+                        .await?
+                        .transaction
+                        .ok_or_else(|| anyhow!("missing transaction"))?;
+                    project_transaction = this
+                        .update(&mut cx, |this, cx| {
+                            this.deserialize_project_transaction(response, push_to_history, cx)
+                        })
+                        .await?;
+                }
+                Ok(project_transaction)
+            })
+        }
     }
 
     async fn format_via_lsp(
@@ -4429,16 +4463,19 @@ impl Project {
                             renamed_buffers.push((cx.handle(), old_path));
                         }
 
-                        if let Some(project_id) = self.remote_id() {
-                            self.client
-                                .send(proto::UpdateBufferFile {
-                                    project_id,
-                                    buffer_id: *buffer_id as u64,
-                                    file: Some(new_file.to_proto()),
-                                })
-                                .log_err();
+                        if new_file != *old_file {
+                            if let Some(project_id) = self.remote_id() {
+                                self.client
+                                    .send(proto::UpdateBufferFile {
+                                        project_id,
+                                        buffer_id: *buffer_id as u64,
+                                        file: Some(new_file.to_proto()),
+                                    })
+                                    .log_err();
+                            }
+
+                            buffer.file_updated(Arc::new(new_file), cx).detach();
                         }
-                        buffer.file_updated(Arc::new(new_file), cx).detach();
                     }
                 });
             } else {
@@ -5151,8 +5188,9 @@ impl Project {
             })
             .await;
 
-        let (saved_version, fingerprint, mtime) =
-            buffer.update(&mut cx, |buffer, cx| buffer.save(cx)).await?;
+        let (saved_version, fingerprint, mtime) = this
+            .update(&mut cx, |this, cx| this.save_buffer(buffer, cx))
+            .await?;
         Ok(proto::BufferSaved {
             project_id,
             buffer_id,
@@ -6022,7 +6060,7 @@ impl Project {
                 .and_then(|buffer| buffer.upgrade(cx));
             if let Some(buffer) = buffer {
                 buffer.update(cx, |buffer, cx| {
-                    buffer.did_save(version, fingerprint, mtime, None, cx);
+                    buffer.did_save(version, fingerprint, mtime, cx);
                 });
             }
             Ok(())

crates/project/src/project_tests.rs 🔗

@@ -243,8 +243,8 @@ async fn test_managing_language_servers(
     );
 
     // Save notifications are reported to all servers.
-    toml_buffer
-        .update(cx, |buffer, cx| buffer.save(cx))
+    project
+        .update(cx, |project, cx| project.save_buffer(toml_buffer, cx))
         .await
         .unwrap();
     assert_eq!(
@@ -2083,12 +2083,13 @@ async fn test_save_file(cx: &mut gpui::TestAppContext) {
         .update(cx, |p, cx| p.open_local_buffer("/dir/file1", cx))
         .await
         .unwrap();
-    buffer
-        .update(cx, |buffer, cx| {
-            assert_eq!(buffer.text(), "the old contents");
-            buffer.edit([(0..0, "a line of text.\n".repeat(10 * 1024))], None, cx);
-            buffer.save(cx)
-        })
+    buffer.update(cx, |buffer, cx| {
+        assert_eq!(buffer.text(), "the old contents");
+        buffer.edit([(0..0, "a line of text.\n".repeat(10 * 1024))], None, cx);
+    });
+
+    project
+        .update(cx, |project, cx| project.save_buffer(buffer.clone(), cx))
         .await
         .unwrap();
 
@@ -2112,11 +2113,12 @@ async fn test_save_in_single_file_worktree(cx: &mut gpui::TestAppContext) {
         .update(cx, |p, cx| p.open_local_buffer("/dir/file1", cx))
         .await
         .unwrap();
-    buffer
-        .update(cx, |buffer, cx| {
-            buffer.edit([(0..0, "a line of text.\n".repeat(10 * 1024))], None, cx);
-            buffer.save(cx)
-        })
+    buffer.update(cx, |buffer, cx| {
+        buffer.edit([(0..0, "a line of text.\n".repeat(10 * 1024))], None, cx);
+    });
+
+    project
+        .update(cx, |project, cx| project.save_buffer(buffer.clone(), cx))
         .await
         .unwrap();
 
@@ -2130,6 +2132,20 @@ async fn test_save_as(cx: &mut gpui::TestAppContext) {
     fs.insert_tree("/dir", json!({})).await;
 
     let project = Project::test(fs.clone(), ["/dir".as_ref()], cx).await;
+
+    let languages = project.read_with(cx, |project, _| project.languages().clone());
+    languages.register(
+        "/some/path",
+        LanguageConfig {
+            name: "Rust".into(),
+            path_suffixes: vec!["rs".into()],
+            ..Default::default()
+        },
+        tree_sitter_rust::language(),
+        None,
+        |_| Default::default(),
+    );
+
     let buffer = project.update(cx, |project, cx| {
         project.create_buffer("", None, cx).unwrap()
     });
@@ -2137,23 +2153,30 @@ async fn test_save_as(cx: &mut gpui::TestAppContext) {
         buffer.edit([(0..0, "abc")], None, cx);
         assert!(buffer.is_dirty());
         assert!(!buffer.has_conflict());
+        assert_eq!(buffer.language().unwrap().name().as_ref(), "Plain Text");
     });
     project
         .update(cx, |project, cx| {
-            project.save_buffer_as(buffer.clone(), "/dir/file1".into(), cx)
+            project.save_buffer_as(buffer.clone(), "/dir/file1.rs".into(), cx)
         })
         .await
         .unwrap();
-    assert_eq!(fs.load(Path::new("/dir/file1")).await.unwrap(), "abc");
+    assert_eq!(fs.load(Path::new("/dir/file1.rs")).await.unwrap(), "abc");
+
+    cx.foreground().run_until_parked();
     buffer.read_with(cx, |buffer, cx| {
-        assert_eq!(buffer.file().unwrap().full_path(cx), Path::new("dir/file1"));
+        assert_eq!(
+            buffer.file().unwrap().full_path(cx),
+            Path::new("dir/file1.rs")
+        );
         assert!(!buffer.is_dirty());
         assert!(!buffer.has_conflict());
+        assert_eq!(buffer.language().unwrap().name().as_ref(), "Rust");
     });
 
     let opened_buffer = project
         .update(cx, |project, cx| {
-            project.open_local_buffer("/dir/file1", cx)
+            project.open_local_buffer("/dir/file1.rs", cx)
         })
         .await
         .unwrap();
@@ -2462,7 +2485,6 @@ async fn test_buffer_is_dirty(cx: &mut gpui::TestAppContext) {
             buffer.version(),
             buffer.as_rope().fingerprint(),
             buffer.file().unwrap().mtime(),
-            None,
             cx,
         );
     });
@@ -2682,11 +2704,11 @@ async fn test_buffer_line_endings(cx: &mut gpui::TestAppContext) {
     });
 
     // Save a file with windows line endings. The file is written correctly.
-    buffer2
-        .update(cx, |buffer, cx| {
-            buffer.set_text("one\ntwo\nthree\nfour\n", cx);
-            buffer.save(cx)
-        })
+    buffer2.update(cx, |buffer, cx| {
+        buffer.set_text("one\ntwo\nthree\nfour\n", cx);
+    });
+    project
+        .update(cx, |project, cx| project.save_buffer(buffer2, cx))
         .await
         .unwrap();
     assert_eq!(

crates/project/src/worktree.rs 🔗

@@ -20,6 +20,7 @@ use gpui::{
     executor, AppContext, AsyncAppContext, Entity, ModelContext, ModelHandle, MutableAppContext,
     Task,
 };
+use language::File as _;
 use language::{
     proto::{
         deserialize_fingerprint, deserialize_version, serialize_fingerprint, serialize_line_ending,
@@ -724,34 +725,69 @@ impl LocalWorktree {
         })
     }
 
-    pub fn save_buffer_as(
+    pub fn save_buffer(
         &self,
         buffer_handle: ModelHandle<Buffer>,
-        path: impl Into<Arc<Path>>,
+        path: Arc<Path>,
+        has_changed_file: bool,
         cx: &mut ModelContext<Worktree>,
-    ) -> Task<Result<()>> {
+    ) -> Task<Result<(clock::Global, RopeFingerprint, SystemTime)>> {
+        let handle = cx.handle();
         let buffer = buffer_handle.read(cx);
+
+        let rpc = self.client.clone();
+        let buffer_id = buffer.remote_id();
+        let project_id = self.share.as_ref().map(|share| share.project_id);
+
         let text = buffer.as_rope().clone();
         let fingerprint = text.fingerprint();
         let version = buffer.version();
         let save = self.write_file(path, text, buffer.line_ending(), cx);
-        let handle = cx.handle();
+
         cx.as_mut().spawn(|mut cx| async move {
             let entry = save.await?;
-            let file = File {
-                entry_id: entry.id,
-                worktree: handle,
-                path: entry.path,
-                mtime: entry.mtime,
-                is_local: true,
-                is_deleted: false,
-            };
+
+            if has_changed_file {
+                let new_file = Arc::new(File {
+                    entry_id: entry.id,
+                    worktree: handle,
+                    path: entry.path,
+                    mtime: entry.mtime,
+                    is_local: true,
+                    is_deleted: false,
+                });
+
+                if let Some(project_id) = project_id {
+                    rpc.send(proto::UpdateBufferFile {
+                        project_id,
+                        buffer_id,
+                        file: Some(new_file.to_proto()),
+                    })
+                    .log_err();
+                }
+
+                buffer_handle.update(&mut cx, |buffer, cx| {
+                    if has_changed_file {
+                        buffer.file_updated(new_file, cx).detach();
+                    }
+                });
+            }
+
+            if let Some(project_id) = project_id {
+                rpc.send(proto::BufferSaved {
+                    project_id,
+                    buffer_id,
+                    version: serialize_version(&version),
+                    mtime: Some(entry.mtime.into()),
+                    fingerprint: serialize_fingerprint(fingerprint),
+                })?;
+            }
 
             buffer_handle.update(&mut cx, |buffer, cx| {
-                buffer.did_save(version, fingerprint, file.mtime, Some(Arc::new(file)), cx);
+                buffer.did_save(version.clone(), fingerprint, entry.mtime, cx);
             });
 
-            Ok(())
+            Ok((version, fingerprint, entry.mtime))
         })
     }
 
@@ -1085,6 +1121,39 @@ impl RemoteWorktree {
         self.disconnected = true;
     }
 
+    pub fn save_buffer(
+        &self,
+        buffer_handle: ModelHandle<Buffer>,
+        cx: &mut ModelContext<Worktree>,
+    ) -> Task<Result<(clock::Global, RopeFingerprint, SystemTime)>> {
+        let buffer = buffer_handle.read(cx);
+        let buffer_id = buffer.remote_id();
+        let version = buffer.version();
+        let rpc = self.client.clone();
+        let project_id = self.project_id;
+        cx.as_mut().spawn(|mut cx| async move {
+            let response = rpc
+                .request(proto::SaveBuffer {
+                    project_id,
+                    buffer_id,
+                    version: serialize_version(&version),
+                })
+                .await?;
+            let version = deserialize_version(response.version);
+            let fingerprint = deserialize_fingerprint(&response.fingerprint)?;
+            let mtime = response
+                .mtime
+                .ok_or_else(|| anyhow!("missing mtime"))?
+                .into();
+
+            buffer_handle.update(&mut cx, |buffer, cx| {
+                buffer.did_save(version.clone(), fingerprint, mtime, cx);
+            });
+
+            Ok((version, fingerprint, mtime))
+        })
+    }
+
     pub fn update_from_remote(&mut self, update: proto::UpdateWorktree) {
         if let Some(updates_tx) = &self.updates_tx {
             updates_tx
@@ -1859,57 +1928,6 @@ impl language::File for File {
         self.is_deleted
     }
 
-    fn save(
-        &self,
-        buffer_id: u64,
-        text: Rope,
-        version: clock::Global,
-        line_ending: LineEnding,
-        cx: &mut MutableAppContext,
-    ) -> Task<Result<(clock::Global, RopeFingerprint, SystemTime)>> {
-        self.worktree.update(cx, |worktree, cx| match worktree {
-            Worktree::Local(worktree) => {
-                let rpc = worktree.client.clone();
-                let project_id = worktree.share.as_ref().map(|share| share.project_id);
-                let fingerprint = text.fingerprint();
-                let save = worktree.write_file(self.path.clone(), text, line_ending, cx);
-                cx.background().spawn(async move {
-                    let entry = save.await?;
-                    if let Some(project_id) = project_id {
-                        rpc.send(proto::BufferSaved {
-                            project_id,
-                            buffer_id,
-                            version: serialize_version(&version),
-                            mtime: Some(entry.mtime.into()),
-                            fingerprint: serialize_fingerprint(fingerprint),
-                        })?;
-                    }
-                    Ok((version, fingerprint, entry.mtime))
-                })
-            }
-            Worktree::Remote(worktree) => {
-                let rpc = worktree.client.clone();
-                let project_id = worktree.project_id;
-                cx.foreground().spawn(async move {
-                    let response = rpc
-                        .request(proto::SaveBuffer {
-                            project_id,
-                            buffer_id,
-                            version: serialize_version(&version),
-                        })
-                        .await?;
-                    let version = deserialize_version(response.version);
-                    let fingerprint = deserialize_fingerprint(&response.fingerprint)?;
-                    let mtime = response
-                        .mtime
-                        .ok_or_else(|| anyhow!("missing mtime"))?
-                        .into();
-                    Ok((version, fingerprint, mtime))
-                })
-            }
-        })
-    }
-
     fn as_any(&self) -> &dyn Any {
         self
     }