Detailed changes
@@ -13297,11 +13297,14 @@ dependencies = [
"dap",
"db",
"editor",
+ "extension",
"extension_host",
"file_finder",
+ "fs",
"futures 0.3.31",
"fuzzy",
"gpui",
+ "http_client",
"indoc",
"language",
"log",
@@ -13314,6 +13317,7 @@ dependencies = [
"project",
"release_channel",
"remote",
+ "remote_server",
"semver",
"serde",
"serde_json",
@@ -62,7 +62,7 @@ async fn test_sharing_an_ssh_remote_project(
.await;
// Set up project on remote FS
- let (opts, server_ssh) = RemoteClient::fake_server(cx_a, server_cx);
+ let (opts, server_ssh, _) = RemoteClient::fake_server(cx_a, server_cx);
let remote_fs = FakeFs::new(server_cx.executor());
remote_fs
.insert_tree(
@@ -104,7 +104,7 @@ async fn test_sharing_an_ssh_remote_project(
)
});
- let client_ssh = RemoteClient::fake_client(opts, cx_a).await;
+ let client_ssh = RemoteClient::connect_mock(opts, cx_a).await;
let (project_a, worktree_id) = client_a
.build_ssh_project(path!("/code/project1"), client_ssh, false, cx_a)
.await;
@@ -232,7 +232,7 @@ async fn test_ssh_collaboration_git_branches(
.await;
// Set up project on remote FS
- let (opts, server_ssh) = RemoteClient::fake_server(cx_a, server_cx);
+ let (opts, server_ssh, _) = RemoteClient::fake_server(cx_a, server_cx);
let remote_fs = FakeFs::new(server_cx.executor());
remote_fs
.insert_tree("/project", serde_json::json!({ ".git":{} }))
@@ -265,7 +265,7 @@ async fn test_ssh_collaboration_git_branches(
)
});
- let client_ssh = RemoteClient::fake_client(opts, cx_a).await;
+ let client_ssh = RemoteClient::connect_mock(opts, cx_a).await;
let (project_a, _) = client_a
.build_ssh_project("/project", client_ssh, false, cx_a)
.await;
@@ -417,7 +417,7 @@ async fn test_ssh_collaboration_formatting_with_prettier(
.create_room(&mut [(&client_a, cx_a), (&client_b, cx_b)])
.await;
- let (opts, server_ssh) = RemoteClient::fake_server(cx_a, server_cx);
+ let (opts, server_ssh, _) = RemoteClient::fake_server(cx_a, server_cx);
let remote_fs = FakeFs::new(server_cx.executor());
let buffer_text = "let one = \"two\"";
let prettier_format_suffix = project::TEST_PRETTIER_FORMAT_SUFFIX;
@@ -470,7 +470,7 @@ async fn test_ssh_collaboration_formatting_with_prettier(
)
});
- let client_ssh = RemoteClient::fake_client(opts, cx_a).await;
+ let client_ssh = RemoteClient::connect_mock(opts, cx_a).await;
let (project_a, worktree_id) = client_a
.build_ssh_project(path!("/project"), client_ssh, false, cx_a)
.await;
@@ -601,7 +601,7 @@ async fn test_remote_server_debugger(
release_channel::init(semver::Version::new(0, 0, 0), cx);
dap_adapters::init(cx);
});
- let (opts, server_ssh) = RemoteClient::fake_server(cx_a, server_cx);
+ let (opts, server_ssh, _) = RemoteClient::fake_server(cx_a, server_cx);
let remote_fs = FakeFs::new(server_cx.executor());
remote_fs
.insert_tree(
@@ -632,7 +632,7 @@ async fn test_remote_server_debugger(
)
});
- let client_ssh = RemoteClient::fake_client(opts, cx_a).await;
+ let client_ssh = RemoteClient::connect_mock(opts, cx_a).await;
let mut server = TestServer::start(server_cx.executor()).await;
let client_a = server.create_client(cx_a, "user_a").await;
cx_a.update(|cx| {
@@ -710,7 +710,7 @@ async fn test_slow_adapter_startup_retries(
release_channel::init(semver::Version::new(0, 0, 0), cx);
dap_adapters::init(cx);
});
- let (opts, server_ssh) = RemoteClient::fake_server(cx_a, server_cx);
+ let (opts, server_ssh, _) = RemoteClient::fake_server(cx_a, server_cx);
let remote_fs = FakeFs::new(server_cx.executor());
remote_fs
.insert_tree(
@@ -741,7 +741,7 @@ async fn test_slow_adapter_startup_retries(
)
});
- let client_ssh = RemoteClient::fake_client(opts, cx_a).await;
+ let client_ssh = RemoteClient::connect_mock(opts, cx_a).await;
let mut server = TestServer::start(server_cx.executor()).await;
let client_a = server.create_client(cx_a, "user_a").await;
cx_a.update(|cx| {
@@ -870,7 +870,7 @@ async fn test_ssh_remote_worktree_trust(cx_a: &mut TestAppContext, server_cx: &m
let server_name = "override-rust-analyzer";
let lsp_inlay_hint_request_count = Arc::new(AtomicUsize::new(0));
- let (opts, server_ssh) = RemoteClient::fake_server(cx_a, server_cx);
+ let (opts, server_ssh, _) = RemoteClient::fake_server(cx_a, server_cx);
let remote_fs = FakeFs::new(server_cx.executor());
remote_fs
.insert_tree(
@@ -944,7 +944,7 @@ async fn test_ssh_remote_worktree_trust(cx_a: &mut TestAppContext, server_cx: &m
)
});
- let client_ssh = RemoteClient::fake_client(opts, cx_a).await;
+ let client_ssh = RemoteClient::connect_mock(opts, cx_a).await;
let (project_a, worktree_id_a) = client_a
.build_ssh_project(path!("/projects/project_a"), client_ssh.clone(), true, cx_a)
.await;
@@ -59,9 +59,15 @@ windows-registry = "0.6.0"
[dev-dependencies]
dap.workspace = true
editor = { workspace = true, features = ["test-support"] }
+extension.workspace = true
+fs.workspace = true
+gpui = { workspace = true, features = ["test-support"] }
+http_client.workspace = true
language = { workspace = true, features = ["test-support"] }
project = { workspace = true, features = ["test-support"] }
+release_channel.workspace = true
+remote = { workspace = true, features = ["test-support"] }
+remote_server.workspace = true
serde_json.workspace = true
settings = { workspace = true, features = ["test-support"] }
workspace = { workspace = true, features = ["test-support"] }
-remote = { workspace = true, features = ["test-support"] }
@@ -848,8 +848,10 @@ pub async fn open_remote_project(
window
.update(cx, |workspace, _, cx| {
if let Some(client) = workspace.project().read(cx).remote_client() {
- ExtensionStore::global(cx)
- .update(cx, |store, cx| store.register_remote_client(client, cx));
+ if let Some(extension_store) = ExtensionStore::try_global(cx) {
+ extension_store
+ .update(cx, |store, cx| store.register_remote_client(client, cx));
+ }
}
})
.ok();
@@ -901,3 +903,102 @@ async fn path_exists(connection: &Arc<dyn RemoteConnection>, path: &Path) -> boo
};
child.status().await.is_ok_and(|status| status.success())
}
+
+#[cfg(test)]
+mod tests {
+ use super::*;
+ use extension::ExtensionHostProxy;
+ use fs::FakeFs;
+ use gpui::TestAppContext;
+ use http_client::BlockedHttpClient;
+ use node_runtime::NodeRuntime;
+ use remote::RemoteClient;
+ use remote_server::{HeadlessAppState, HeadlessProject};
+ use serde_json::json;
+ use util::path;
+
+ #[gpui::test]
+ async fn test_open_remote_project_with_mock_connection(
+ cx: &mut TestAppContext,
+ server_cx: &mut TestAppContext,
+ ) {
+ let app_state = init_test(cx);
+ let executor = cx.executor();
+
+ cx.update(|cx| {
+ release_channel::init(semver::Version::new(0, 0, 0), cx);
+ });
+ server_cx.update(|cx| {
+ release_channel::init(semver::Version::new(0, 0, 0), cx);
+ });
+
+ let (opts, server_session, connect_guard) = RemoteClient::fake_server(cx, server_cx);
+
+ let remote_fs = FakeFs::new(server_cx.executor());
+ remote_fs
+ .insert_tree(
+ path!("/project"),
+ json!({
+ "src": {
+ "main.rs": "fn main() {}",
+ },
+ "README.md": "# Test Project",
+ }),
+ )
+ .await;
+
+ server_cx.update(HeadlessProject::init);
+ let http_client = Arc::new(BlockedHttpClient);
+ let node_runtime = NodeRuntime::unavailable();
+ let languages = Arc::new(language::LanguageRegistry::new(server_cx.executor()));
+ let proxy = Arc::new(ExtensionHostProxy::new());
+
+ let _headless = server_cx.new(|cx| {
+ HeadlessProject::new(
+ HeadlessAppState {
+ session: server_session,
+ fs: remote_fs.clone(),
+ http_client,
+ node_runtime,
+ languages,
+ extension_host_proxy: proxy,
+ },
+ false,
+ cx,
+ )
+ });
+
+ drop(connect_guard);
+
+ let paths = vec![PathBuf::from(path!("/project"))];
+ let open_options = workspace::OpenOptions::default();
+
+ let mut async_cx = cx.to_async();
+ let result = open_remote_project(opts, paths, app_state, open_options, &mut async_cx).await;
+
+ executor.run_until_parked();
+
+ assert!(result.is_ok(), "open_remote_project should succeed");
+
+ let windows = cx.update(|cx| cx.windows().len());
+ assert_eq!(windows, 1, "Should have opened a window");
+
+ let workspace_handle = cx.update(|cx| cx.windows()[0].downcast::<Workspace>().unwrap());
+
+ workspace_handle
+ .update(cx, |workspace, _, cx| {
+ let project = workspace.project().read(cx);
+ assert!(project.is_remote(), "Project should be a remote project");
+ })
+ .unwrap();
+ }
+
+ fn init_test(cx: &mut TestAppContext) -> Arc<AppState> {
+ cx.update(|cx| {
+ let state = AppState::test(cx);
+ crate::init(cx);
+ editor::init(cx);
+ state
+ })
+ }
+}
@@ -1,3 +1,5 @@
+#[cfg(any(test, feature = "test-support"))]
+use crate::transport::mock::ConnectGuard;
use crate::{
SshConnectionOptions,
protocol::MessageId,
@@ -977,23 +979,26 @@ impl RemoteClient {
/// Creates a mock connection pair for testing.
///
/// This is the recommended way to create mock remote connections for tests.
- /// It returns both the `MockConnectionOptions` (which can be passed to create
- /// a `HeadlessProject`) and an `AnyProtoClient` for the server side.
+ /// It returns the `MockConnectionOptions` (which can be passed to create a
+ /// `HeadlessProject`), an `AnyProtoClient` for the server side and a
+ /// `ConnectGuard` for the client side which blocks the connection from
+ /// being established until dropped.
///
/// # Example
/// ```ignore
- /// let (opts, server_session) = RemoteClient::fake_server(cx, server_cx);
+ /// let (opts, server_session, connect_guard) = RemoteClient::fake_server(cx, server_cx);
/// // Set up HeadlessProject with server_session...
+ /// drop(connect_guard);
/// let client = RemoteClient::fake_client(opts, cx).await;
/// ```
#[cfg(any(test, feature = "test-support"))]
pub fn fake_server(
client_cx: &mut gpui::TestAppContext,
server_cx: &mut gpui::TestAppContext,
- ) -> (RemoteConnectionOptions, AnyProtoClient) {
+ ) -> (RemoteConnectionOptions, AnyProtoClient, ConnectGuard) {
use crate::transport::mock::MockConnection;
- let (opts, server_client) = MockConnection::new(client_cx, server_cx);
- (opts.into(), server_client)
+ let (opts, server_client, connect_guard) = MockConnection::new(client_cx, server_cx);
+ (opts.into(), server_client, connect_guard)
}
/// Creates a `RemoteClient` connected to a mock server.
@@ -1002,10 +1007,11 @@ impl RemoteClient {
/// `HeadlessProject` with the server session, then call this method
/// to create the client.
#[cfg(any(test, feature = "test-support"))]
- pub async fn fake_client(
+ pub async fn connect_mock(
opts: RemoteConnectionOptions,
client_cx: &mut gpui::TestAppContext,
) -> Entity<Self> {
+ assert!(matches!(opts, RemoteConnectionOptions::Mock(..)));
use crate::transport::mock::MockDelegate;
let (_tx, rx) = oneshot::channel();
let mut cx = client_cx.to_async();
@@ -1047,7 +1053,7 @@ struct ConnectionPool {
impl Global for ConnectionPool {}
impl ConnectionPool {
- pub fn connect(
+ fn connect(
&mut self,
opts: RemoteConnectionOptions,
delegate: Arc<dyn RemoteClientDelegate>,
@@ -1095,16 +1101,15 @@ impl ConnectionPool {
.map(|connection| Arc::new(connection) as Arc<dyn RemoteConnection>)
}
#[cfg(any(test, feature = "test-support"))]
- RemoteConnectionOptions::Mock(opts) => {
- cx.update(|cx| {
- cx.default_global::<crate::transport::mock::MockConnectionRegistry>()
- .take(&opts)
- .ok_or_else(|| anyhow!(
- "Mock connection not found. Call MockConnection::new() first."
- ))
- .map(|connection| connection as Arc<dyn RemoteConnection>)
- })
- }
+ RemoteConnectionOptions::Mock(opts) => match cx.update(|cx| {
+ cx.default_global::<crate::transport::mock::MockConnectionRegistry>()
+ .take(&opts)
+ }) {
+ Some(connection) => Ok(connection.await as Arc<dyn RemoteConnection>),
+ None => Err(anyhow!(
+ "Mock connection not found. Call MockConnection::new() first."
+ )),
+ },
};
cx.update_global(|pool: &mut Self, _| {
@@ -37,7 +37,10 @@ use async_trait::async_trait;
use collections::HashMap;
use futures::{
FutureExt, SinkExt, StreamExt,
- channel::mpsc::{self, Sender},
+ channel::{
+ mpsc::{self, Sender},
+ oneshot,
+ },
select_biased,
};
use gpui::{App, AppContext as _, AsyncApp, Global, Task, TestAppContext};
@@ -94,21 +97,30 @@ unsafe impl Sync for SendableCx {}
/// it retrieves the connection from this registry.
#[derive(Default)]
pub struct MockConnectionRegistry {
- pending: HashMap<MockConnectionOptions, Arc<MockRemoteConnection>>,
+ pending: HashMap<MockConnectionOptions, (oneshot::Receiver<()>, Arc<MockRemoteConnection>)>,
}
impl Global for MockConnectionRegistry {}
impl MockConnectionRegistry {
/// Called by `ConnectionPool::connect` to retrieve a pre-registered mock connection.
- pub fn take(&mut self, opts: &MockConnectionOptions) -> Option<Arc<MockRemoteConnection>> {
- self.pending.remove(opts)
+ pub fn take(
+ &mut self,
+ opts: &MockConnectionOptions,
+ ) -> Option<impl Future<Output = Arc<MockRemoteConnection>> + use<>> {
+ let (guard, con) = self.pending.remove(opts)?;
+ Some(async move {
+ _ = guard.await;
+ con
+ })
}
}
/// Helper for creating mock connection pairs in tests.
pub struct MockConnection;
+pub type ConnectGuard = oneshot::Sender<()>;
+
impl MockConnection {
/// Creates a new mock connection pair for testing.
///
@@ -125,10 +137,10 @@ impl MockConnection {
/// # Arguments
/// - `client_cx`: The test context for the client side
/// - `server_cx`: The test context for the server/headless side
- pub fn new(
+ pub(crate) fn new(
client_cx: &mut TestAppContext,
server_cx: &mut TestAppContext,
- ) -> (MockConnectionOptions, AnyProtoClient) {
+ ) -> (MockConnectionOptions, AnyProtoClient, ConnectGuard) {
static NEXT_ID: AtomicU64 = AtomicU64::new(0);
let id = NEXT_ID.fetch_add(1, Ordering::SeqCst);
let opts = MockConnectionOptions { id };
@@ -144,13 +156,15 @@ impl MockConnection {
server_cx: SendableCx::new(server_cx),
});
+ let (tx, rx) = oneshot::channel();
+
client_cx.update(|cx| {
cx.default_global::<MockConnectionRegistry>()
.pending
- .insert(opts.clone(), connection);
+ .insert(opts.clone(), (rx, connection));
});
- (opts, server_client.into())
+ (opts, server_client.into(), tx)
}
}
@@ -1920,7 +1920,7 @@ pub async fn init_test(
});
init_logger();
- let (opts, ssh_server_client) = RemoteClient::fake_server(cx, server_cx);
+ let (opts, ssh_server_client, _) = RemoteClient::fake_server(cx, server_cx);
let http_client = Arc::new(BlockedHttpClient);
let node_runtime = NodeRuntime::unavailable();
let languages = Arc::new(LanguageRegistry::new(cx.executor()));
@@ -1941,7 +1941,7 @@ pub async fn init_test(
)
});
- let ssh = RemoteClient::fake_client(opts, cx).await;
+ let ssh = RemoteClient::connect_mock(opts, cx).await;
let project = build_project(ssh, cx);
project
.update(cx, {