Split worktree updates when a peer joins an already-shared project

This commit is contained in:
Antonio Scandurra 2022-06-30 16:49:56 +02:00
parent 845c79ee05
commit 484af8c7c4
6 changed files with 137 additions and 184 deletions

View file

@ -1472,6 +1472,7 @@ async fn test_collaborating_with_diagnostics(
// Join project as client C and observe the diagnostics. // Join project as client C and observe the diagnostics.
let project_c = client_c.build_remote_project(&project_a, cx_a, cx_c).await; let project_c = client_c.build_remote_project(&project_a, cx_a, cx_c).await;
deterministic.run_until_parked();
project_c.read_with(cx_c, |project, cx| { project_c.read_with(cx_c, |project, cx| {
assert_eq!( assert_eq!(
project.diagnostic_summaries(cx).collect::<Vec<_>>(), project.diagnostic_summaries(cx).collect::<Vec<_>>(),

View file

@ -791,21 +791,10 @@ impl Server {
let worktrees = project let worktrees = project
.worktrees .worktrees
.iter() .iter()
.filter_map(|(id, shared_worktree)| { .map(|(id, worktree)| proto::WorktreeMetadata {
let worktree = project.worktrees.get(&id)?; id: *id,
Some(proto::Worktree { root_name: worktree.root_name.clone(),
id: *id, visible: worktree.visible,
root_name: worktree.root_name.clone(),
entries: shared_worktree.entries.values().cloned().collect(),
diagnostic_summaries: shared_worktree
.diagnostic_summaries
.values()
.cloned()
.collect(),
visible: worktree.visible,
scan_id: shared_worktree.scan_id,
is_complete: worktree.is_complete,
})
}) })
.collect::<Vec<_>>(); .collect::<Vec<_>>();
@ -841,14 +830,15 @@ impl Server {
} }
} }
for (receipt, replica_id) in receipts_with_replica_ids { // First, we send the metadata associated with each worktree.
for (receipt, replica_id) in &receipts_with_replica_ids {
self.peer.respond( self.peer.respond(
receipt, receipt.clone(),
proto::JoinProjectResponse { proto::JoinProjectResponse {
variant: Some(proto::join_project_response::Variant::Accept( variant: Some(proto::join_project_response::Variant::Accept(
proto::join_project_response::Accept { proto::join_project_response::Accept {
worktrees: worktrees.clone(), worktrees: worktrees.clone(),
replica_id: replica_id as u32, replica_id: *replica_id as u32,
collaborators: collaborators.clone(), collaborators: collaborators.clone(),
language_servers: project.language_servers.clone(), language_servers: project.language_servers.clone(),
}, },
@ -856,6 +846,43 @@ impl Server {
}, },
)?; )?;
} }
for (worktree_id, worktree) in &project.worktrees {
#[cfg(any(test, feature = "test-support"))]
const MAX_CHUNK_SIZE: usize = 2;
#[cfg(not(any(test, feature = "test-support")))]
const MAX_CHUNK_SIZE: usize = 256;
// Stream this worktree's entries.
let message = proto::UpdateWorktree {
project_id: project_id.to_proto(),
worktree_id: *worktree_id,
root_name: worktree.root_name.clone(),
updated_entries: worktree.entries.values().cloned().collect(),
removed_entries: Default::default(),
scan_id: worktree.scan_id,
is_last_update: worktree.is_complete,
};
for update in proto::split_worktree_update(message, MAX_CHUNK_SIZE) {
for (receipt, _) in &receipts_with_replica_ids {
self.peer.send(receipt.sender_id, update.clone())?;
}
}
// Stream this worktree's diagnostics.
for summary in worktree.diagnostic_summaries.values() {
for (receipt, _) in &receipts_with_replica_ids {
self.peer.send(
receipt.sender_id,
proto::UpdateDiagnosticSummary {
project_id: project_id.to_proto(),
worktree_id: *worktree_id,
summary: Some(summary.clone()),
},
)?;
}
}
}
} }
self.update_user_contacts(host_user_id).await?; self.update_user_contacts(host_user_id).await?;

View file

@ -487,10 +487,9 @@ impl Project {
let mut worktrees = Vec::new(); let mut worktrees = Vec::new();
for worktree in response.worktrees { for worktree in response.worktrees {
let (worktree, load_task) = cx let worktree = cx
.update(|cx| Worktree::remote(remote_id, replica_id, worktree, client.clone(), cx)); .update(|cx| Worktree::remote(remote_id, replica_id, worktree, client.clone(), cx));
worktrees.push(worktree); worktrees.push(worktree);
load_task.detach();
} }
let (opened_buffer_tx, opened_buffer_rx) = watch::channel(); let (opened_buffer_tx, opened_buffer_rx) = watch::channel();
@ -4441,19 +4440,9 @@ impl Project {
{ {
this.worktrees.push(WorktreeHandle::Strong(old_worktree)); this.worktrees.push(WorktreeHandle::Strong(old_worktree));
} else { } else {
let worktree = proto::Worktree { let worktree =
id: worktree.id,
root_name: worktree.root_name,
entries: Default::default(),
diagnostic_summaries: Default::default(),
visible: worktree.visible,
scan_id: 0,
is_complete: false,
};
let (worktree, load_task) =
Worktree::remote(remote_id, replica_id, worktree, client.clone(), cx); Worktree::remote(remote_id, replica_id, worktree, client.clone(), cx);
this.add_worktree(&worktree, cx); this.add_worktree(&worktree, cx);
load_task.detach();
} }
} }
@ -4477,8 +4466,8 @@ impl Project {
if let Some(worktree) = this.worktree_for_id(worktree_id, cx) { if let Some(worktree) = this.worktree_for_id(worktree_id, cx) {
worktree.update(cx, |worktree, _| { worktree.update(cx, |worktree, _| {
let worktree = worktree.as_remote_mut().unwrap(); let worktree = worktree.as_remote_mut().unwrap();
worktree.update_from_remote(envelope) worktree.update_from_remote(envelope.payload);
})?; });
} }
Ok(()) Ok(())
}) })
@ -7996,7 +7985,10 @@ mod tests {
} }
#[gpui::test(retries = 5)] #[gpui::test(retries = 5)]
async fn test_rescan_and_remote_updates(cx: &mut gpui::TestAppContext) { async fn test_rescan_and_remote_updates(
deterministic: Arc<Deterministic>,
cx: &mut gpui::TestAppContext,
) {
let dir = temp_tree(json!({ let dir = temp_tree(json!({
"a": { "a": {
"file1": "", "file1": "",
@ -8040,17 +8032,24 @@ mod tests {
// Create a remote copy of this worktree. // Create a remote copy of this worktree.
let tree = project.read_with(cx, |project, cx| project.worktrees(cx).next().unwrap()); let tree = project.read_with(cx, |project, cx| project.worktrees(cx).next().unwrap());
let initial_snapshot = tree.read_with(cx, |tree, _| tree.as_local().unwrap().snapshot()); let initial_snapshot = tree.read_with(cx, |tree, _| tree.as_local().unwrap().snapshot());
let (remote, load_task) = cx.update(|cx| { let remote = cx.update(|cx| {
Worktree::remote( Worktree::remote(
1, 1,
1, 1,
initial_snapshot.to_proto(&Default::default(), true), proto::WorktreeMetadata {
id: initial_snapshot.id().to_proto(),
root_name: initial_snapshot.root_name().into(),
visible: true,
},
rpc.clone(), rpc.clone(),
cx, cx,
) )
}); });
// tree remote.update(cx, |remote, _| {
load_task.await; let update = initial_snapshot.build_initial_update(1);
remote.as_remote_mut().unwrap().update_from_remote(update);
});
deterministic.run_until_parked();
cx.read(|cx| { cx.read(|cx| {
assert!(!buffer2.read(cx).is_dirty()); assert!(!buffer2.read(cx).is_dirty());
@ -8116,19 +8115,16 @@ mod tests {
// Update the remote worktree. Check that it becomes consistent with the // Update the remote worktree. Check that it becomes consistent with the
// local worktree. // local worktree.
remote.update(cx, |remote, cx| { remote.update(cx, |remote, cx| {
let update_message = tree.read(cx).as_local().unwrap().snapshot().build_update( let update = tree.read(cx).as_local().unwrap().snapshot().build_update(
&initial_snapshot, &initial_snapshot,
1, 1,
1, 1,
true, true,
); );
remote remote.as_remote_mut().unwrap().update_from_remote(update);
.as_remote_mut() });
.unwrap() deterministic.run_until_parked();
.snapshot remote.read_with(cx, |remote, _| {
.apply_remote_update(update_message)
.unwrap();
assert_eq!( assert_eq!(
remote remote
.paths() .paths()

View file

@ -7,7 +7,7 @@ use super::{
}; };
use ::ignore::gitignore::{Gitignore, GitignoreBuilder}; use ::ignore::gitignore::{Gitignore, GitignoreBuilder};
use anyhow::{anyhow, Context, Result}; use anyhow::{anyhow, Context, Result};
use client::{proto, Client, TypedEnvelope}; use client::{proto, Client};
use clock::ReplicaId; use clock::ReplicaId;
use collections::HashMap; use collections::HashMap;
use futures::{ use futures::{
@ -40,7 +40,6 @@ use std::{
ffi::{OsStr, OsString}, ffi::{OsStr, OsString},
fmt, fmt,
future::Future, future::Future,
mem,
ops::{Deref, DerefMut}, ops::{Deref, DerefMut},
os::unix::prelude::{OsStrExt, OsStringExt}, os::unix::prelude::{OsStrExt, OsStringExt},
path::{Path, PathBuf}, path::{Path, PathBuf},
@ -173,10 +172,10 @@ impl Worktree {
pub fn remote( pub fn remote(
project_remote_id: u64, project_remote_id: u64,
replica_id: ReplicaId, replica_id: ReplicaId,
worktree: proto::Worktree, worktree: proto::WorktreeMetadata,
client: Arc<Client>, client: Arc<Client>,
cx: &mut MutableAppContext, cx: &mut MutableAppContext,
) -> (ModelHandle<Self>, Task<()>) { ) -> ModelHandle<Self> {
let remote_id = worktree.id; let remote_id = worktree.id;
let root_char_bag: CharBag = worktree let root_char_bag: CharBag = worktree
.root_name .root_name
@ -191,8 +190,8 @@ impl Worktree {
root_char_bag, root_char_bag,
entries_by_path: Default::default(), entries_by_path: Default::default(),
entries_by_id: Default::default(), entries_by_id: Default::default(),
scan_id: worktree.scan_id as usize, scan_id: 0,
is_complete: worktree.is_complete, is_complete: false,
}; };
let (updates_tx, mut updates_rx) = mpsc::unbounded(); let (updates_tx, mut updates_rx) = mpsc::unbounded();
@ -207,90 +206,37 @@ impl Worktree {
updates_tx: Some(updates_tx), updates_tx: Some(updates_tx),
snapshot_updated_rx: snapshot_updated_rx.clone(), snapshot_updated_rx: snapshot_updated_rx.clone(),
client: client.clone(), client: client.clone(),
diagnostic_summaries: TreeMap::from_ordered_entries( diagnostic_summaries: Default::default(),
worktree.diagnostic_summaries.into_iter().map(|summary| {
(
PathKey(PathBuf::from(summary.path).into()),
DiagnosticSummary {
language_server_id: summary.language_server_id as usize,
error_count: summary.error_count as usize,
warning_count: summary.warning_count as usize,
},
)
}),
),
visible, visible,
}) })
}); });
let deserialize_task = cx.spawn({ cx.background()
let worktree_handle = worktree_handle.clone(); .spawn(async move {
|cx| async move { while let Some(update) = updates_rx.next().await {
let (entries_by_path, entries_by_id) = cx if let Err(error) = background_snapshot.lock().apply_remote_update(update) {
.background() log::error!("error applying worktree update: {}", error);
.spawn(async move { }
let mut entries_by_path_edits = Vec::new();
let mut entries_by_id_edits = Vec::new();
for entry in worktree.entries {
match Entry::try_from((&root_char_bag, entry)) {
Ok(entry) => {
entries_by_id_edits.push(Edit::Insert(PathEntry {
id: entry.id,
path: entry.path.clone(),
is_ignored: entry.is_ignored,
scan_id: 0,
}));
entries_by_path_edits.push(Edit::Insert(entry));
}
Err(err) => log::warn!("error for remote worktree entry {:?}", err),
}
}
let mut entries_by_path = SumTree::new();
let mut entries_by_id = SumTree::new();
entries_by_path.edit(entries_by_path_edits, &());
entries_by_id.edit(entries_by_id_edits, &());
(entries_by_path, entries_by_id)
})
.await;
{
let mut snapshot = background_snapshot.lock();
snapshot.entries_by_path = entries_by_path;
snapshot.entries_by_id = entries_by_id;
snapshot_updated_tx.send(()).await.ok(); snapshot_updated_tx.send(()).await.ok();
} }
})
.detach();
cx.background() cx.spawn(|mut cx| {
.spawn(async move { let this = worktree_handle.downgrade();
while let Some(update) = updates_rx.next().await { async move {
if let Err(error) = while let Some(_) = snapshot_updated_rx.recv().await {
background_snapshot.lock().apply_remote_update(update) if let Some(this) = this.upgrade(&cx) {
{ this.update(&mut cx, |this, cx| this.poll_snapshot(cx));
log::error!("error applying worktree update: {}", error); } else {
} break;
snapshot_updated_tx.send(()).await.ok();
}
})
.detach();
cx.spawn(|mut cx| {
let this = worktree_handle.downgrade();
async move {
while let Some(_) = snapshot_updated_rx.recv().await {
if let Some(this) = this.upgrade(&cx) {
this.update(&mut cx, |this, cx| this.poll_snapshot(cx));
} else {
break;
}
}
} }
}) }
.detach();
} }
}); })
(worktree_handle, deserialize_task) .detach();
worktree_handle
} }
pub fn as_local(&self) -> Option<&LocalWorktree> { pub fn as_local(&self) -> Option<&LocalWorktree> {
@ -1015,16 +961,12 @@ impl RemoteWorktree {
self.updates_tx.take(); self.updates_tx.take();
} }
pub fn update_from_remote( pub fn update_from_remote(&mut self, update: proto::UpdateWorktree) {
&mut self,
envelope: TypedEnvelope<proto::UpdateWorktree>,
) -> Result<()> {
if let Some(updates_tx) = &self.updates_tx { if let Some(updates_tx) = &self.updates_tx {
updates_tx updates_tx
.unbounded_send(envelope.payload) .unbounded_send(update)
.expect("consumer runs to completion"); .expect("consumer runs to completion");
} }
Ok(())
} }
fn wait_for_snapshot( fn wait_for_snapshot(
@ -1162,7 +1104,7 @@ impl Snapshot {
for entry_id in update.removed_entries { for entry_id in update.removed_entries {
let entry = self let entry = self
.entry_for_id(ProjectEntryId::from_proto(entry_id)) .entry_for_id(ProjectEntryId::from_proto(entry_id))
.ok_or_else(|| anyhow!("unknown entry"))?; .ok_or_else(|| anyhow!("unknown entry {}", entry_id))?;
entries_by_path_edits.push(Edit::Remove(PathKey(entry.path.clone()))); entries_by_path_edits.push(Edit::Remove(PathKey(entry.path.clone())));
entries_by_id_edits.push(Edit::Remove(entry.id)); entries_by_id_edits.push(Edit::Remove(entry.id));
} }
@ -1306,28 +1248,16 @@ impl LocalSnapshot {
} }
#[cfg(test)] #[cfg(test)]
pub(crate) fn to_proto( pub(crate) fn build_initial_update(&self, project_id: u64) -> proto::UpdateWorktree {
&self,
diagnostic_summaries: &TreeMap<PathKey, DiagnosticSummary>,
visible: bool,
) -> proto::Worktree {
let root_name = self.root_name.clone(); let root_name = self.root_name.clone();
proto::Worktree { proto::UpdateWorktree {
id: self.id.0 as u64, project_id,
worktree_id: self.id().to_proto(),
root_name, root_name,
entries: self updated_entries: self.entries_by_path.iter().map(Into::into).collect(),
.entries_by_path removed_entries: Default::default(),
.iter()
.filter(|e| !e.is_ignored)
.map(Into::into)
.collect(),
diagnostic_summaries: diagnostic_summaries
.iter()
.map(|(path, summary)| summary.to_proto(&path.0))
.collect(),
visible,
scan_id: self.scan_id as u64, scan_id: self.scan_id as u64,
is_complete: true, is_last_update: true,
} }
} }
@ -2709,31 +2639,14 @@ impl<'a> TryFrom<(&'a CharBag, proto::Entry)> for Entry {
} }
} }
async fn send_worktree_update( async fn send_worktree_update(client: &Arc<Client>, update: proto::UpdateWorktree) -> Result<()> {
client: &Arc<Client>,
mut update: proto::UpdateWorktree,
) -> Result<()> {
#[cfg(any(test, feature = "test-support"))] #[cfg(any(test, feature = "test-support"))]
const MAX_CHUNK_SIZE: usize = 2; const MAX_CHUNK_SIZE: usize = 2;
#[cfg(not(any(test, feature = "test-support")))] #[cfg(not(any(test, feature = "test-support")))]
const MAX_CHUNK_SIZE: usize = 256; const MAX_CHUNK_SIZE: usize = 256;
let mut is_last_update = false; for update in proto::split_worktree_update(update, MAX_CHUNK_SIZE) {
while !is_last_update { client.request(update).await?;
let chunk_size = cmp::min(update.updated_entries.len(), MAX_CHUNK_SIZE);
let updated_entries = update.updated_entries.drain(..chunk_size).collect();
is_last_update = update.updated_entries.is_empty();
client
.request(proto::UpdateWorktree {
project_id: update.project_id,
worktree_id: update.worktree_id,
root_name: update.root_name.clone(),
updated_entries,
removed_entries: mem::take(&mut update.removed_entries),
scan_id: update.scan_id,
is_last_update,
})
.await?;
} }
Ok(()) Ok(())

View file

@ -168,7 +168,7 @@ message JoinProjectResponse {
message Accept { message Accept {
uint32 replica_id = 1; uint32 replica_id = 1;
repeated Worktree worktrees = 2; repeated WorktreeMetadata worktrees = 2;
repeated Collaborator collaborators = 3; repeated Collaborator collaborators = 3;
repeated LanguageServer language_servers = 4; repeated LanguageServer language_servers = 4;
} }
@ -766,16 +766,6 @@ message User {
string avatar_url = 3; string avatar_url = 3;
} }
message Worktree {
uint64 id = 1;
string root_name = 2;
repeated Entry entries = 3;
repeated DiagnosticSummary diagnostic_summaries = 4;
bool visible = 5;
uint64 scan_id = 6;
bool is_complete = 7;
}
message File { message File {
uint64 worktree_id = 1; uint64 worktree_id = 1;
optional uint64 entry_id = 2; optional uint64 entry_id = 2;

View file

@ -5,6 +5,7 @@ use futures::{SinkExt as _, StreamExt as _};
use prost::Message as _; use prost::Message as _;
use serde::Serialize; use serde::Serialize;
use std::any::{Any, TypeId}; use std::any::{Any, TypeId};
use std::{cmp, iter, mem};
use std::{ use std::{
fmt::Debug, fmt::Debug,
io, io,
@ -390,6 +391,31 @@ impl From<Nonce> for u128 {
} }
} }
pub fn split_worktree_update(
mut message: UpdateWorktree,
max_chunk_size: usize,
) -> impl Iterator<Item = UpdateWorktree> {
let mut done = false;
iter::from_fn(move || {
if done {
return None;
}
let chunk_size = cmp::min(message.updated_entries.len(), max_chunk_size);
let updated_entries = message.updated_entries.drain(..chunk_size).collect();
done = message.updated_entries.is_empty();
Some(UpdateWorktree {
project_id: message.project_id,
worktree_id: message.worktree_id,
root_name: message.root_name.clone(),
updated_entries,
removed_entries: mem::take(&mut message.removed_entries),
scan_id: message.scan_id,
is_last_update: done && message.is_last_update,
})
})
}
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use super::*; use super::*;