From 7a9ebd15852eee7b676a671098d63bece679e0f7 Mon Sep 17 00:00:00 2001 From: Liam Murphy Date: Mon, 10 May 2021 16:06:13 +1000 Subject: [PATCH] feat: add deno test --watch (#9160) This commit implements file watching for deno test. When a file is changed, only the test modules which use it as a dependency are rerun. This is accomplished by reworking the file watching infrastructure to pass the paths which have changed to the resolver, and then constructing a module graph for each test module to check if it contains any changed files. --- cli/file_watcher.rs | 263 +++++++++++++++------------------ cli/flags.rs | 7 + cli/main.rs | 258 +++++++++++++++++++++++++------- cli/module_graph.rs | 12 ++ cli/tests/integration_tests.rs | 200 +++++++++++++++++++++++-- cli/tools/fmt.rs | 44 ++++-- cli/tools/test_runner.rs | 34 ++--- 7 files changed, 581 insertions(+), 237 deletions(-) diff --git a/cli/file_watcher.rs b/cli/file_watcher.rs index 651a3ee314..7251dbe8c7 100644 --- a/cli/file_watcher.rs +++ b/cli/file_watcher.rs @@ -2,7 +2,6 @@ use crate::colors; use deno_core::error::AnyError; -use deno_core::futures::ready; use deno_core::futures::stream::{Stream, StreamExt}; use deno_core::futures::Future; use log::info; @@ -14,10 +13,11 @@ use notify::RecommendedWatcher; use notify::RecursiveMode; use notify::Watcher; use pin_project::pin_project; +use std::collections::HashSet; use std::path::PathBuf; use std::pin::Pin; -use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::Arc; +use std::sync::Mutex; use std::task::Context; use std::task::Poll; use std::time::Duration; @@ -29,26 +29,24 @@ use tokio::time::Sleep; const DEBOUNCE_INTERVAL: Duration = Duration::from_millis(200); -type FileWatcherFuture = Pin>>; - #[pin_project(project = DebounceProjection)] struct Debounce { #[pin] timer: Sleep, - event_detected: Arc, + changed_paths: Arc>>, } impl Debounce { fn new() -> Self { Self { timer: sleep(DEBOUNCE_INTERVAL), - event_detected: Arc::new(AtomicBool::new(false)), + changed_paths: Arc::new(Mutex::new(HashSet::new())), } } } impl Stream for Debounce { - type Item = (); + type Item = Vec; /// Note that this never returns `Poll::Ready(None)`, which means that the /// file watcher will be alive until the Deno process is terminated. @@ -56,18 +54,24 @@ impl Stream for Debounce { self: Pin<&mut Self>, cx: &mut Context, ) -> Poll> { - if self.event_detected.swap(false, Ordering::Relaxed) { - Poll::Ready(Some(())) + let mut changed_paths = self.changed_paths.lock().unwrap(); + if changed_paths.len() > 0 { + Poll::Ready(Some(changed_paths.drain().collect())) } else { + drop(changed_paths); let mut timer = self.project().timer; - ready!(timer.as_mut().poll(cx)); - timer.reset(Instant::now() + DEBOUNCE_INTERVAL); + if timer.as_mut().poll(cx).is_ready() { + timer.reset(Instant::now() + DEBOUNCE_INTERVAL); + } Poll::Pending } } } -async fn error_handler(watch_future: FileWatcherFuture>) { +async fn error_handler(watch_future: F) +where + F: Future>, +{ let result = watch_future.await; if let Err(err) = result { let msg = format!("{}: {}", colors::red_bold("error"), err.to_string(),); @@ -75,172 +79,139 @@ async fn error_handler(watch_future: FileWatcherFuture>) { } } -/// This function adds watcher functionality to subcommands like `fmt` or `lint`. -/// The difference from [`watch_func_with_module_resolution`] is that this doesn't depend on -/// [`ModuleGraph`]. -/// -/// - `target_resolver` is used for resolving file paths to be watched at every restarting of the watcher. The -/// return value of this closure will then be passed to `operation` as an argument. -/// -/// - `operation` is the actual operation we want to run every time the watcher detects file -/// changes. For example, in the case where we would like to apply `fmt`, then `operation` would -/// have the logic for it like calling `format_source_files`. -/// -/// - `job_name` is just used for printing watcher status to terminal. -/// -/// Note that the watcher will stop working if `target_resolver` fails at some point. -/// -/// [`ModuleGraph`]: crate::module_graph::Graph -pub async fn watch_func( - target_resolver: F, - operation: G, - job_name: &str, -) -> Result<(), AnyError> -where - F: Fn() -> Result, AnyError>, - G: Fn(Vec) -> FileWatcherFuture>, -{ - let debounce = Debounce::new(); - pin!(debounce); +pub enum ResolutionResult { + Restart { + paths_to_watch: Vec, + result: Result, + }, + Ignore, +} +async fn next_restart( + resolver: &mut R, + debounce: &mut Pin<&mut Debounce>, +) -> (Vec, Result) +where + R: FnMut(Option>) -> F, + F: Future>, +{ loop { - let paths = target_resolver()?; - let _watcher = new_watcher(&paths, &debounce)?; - let func = error_handler(operation(paths)); - let mut is_file_changed = false; - select! { - _ = debounce.next() => { - is_file_changed = true; + let changed = debounce.next().await; + match resolver(changed).await { + ResolutionResult::Ignore => { + log::debug!("File change ignored") + } + ResolutionResult::Restart { + paths_to_watch, + result, + } => { info!( "{} File change detected! Restarting!", colors::intense_blue("Watcher"), ); - }, - _ = func => {}, - }; - - if !is_file_changed { - info!( - "{} {} finished! Restarting on file change...", - colors::intense_blue("Watcher"), - job_name, - ); - debounce.next().await; - info!( - "{} File change detected! Restarting!", - colors::intense_blue("Watcher"), - ); + return (paths_to_watch, result); + } } } } -pub enum ModuleResolutionResult { - Success { - paths_to_watch: Vec, - module_info: T, - }, - Fail { - source_path: PathBuf, - error: AnyError, - }, -} - -/// This function adds watcher functionality to subcommands like `run` or `bundle`. -/// The difference from [`watch_func`] is that this does depend on [`ModuleGraph`]. +/// Creates a file watcher, which will call `resolver` with every file change. /// -/// - `module_resolver` is used for both resolving file paths to be watched at every restarting -/// of the watcher and building [`ModuleGraph`] or [`ModuleSpecifier`] which will then be passed -/// to `operation`. +/// - `resolver` is used for resolving file paths to be watched at every restarting +/// of the watcher, and can also return a value to be passed to `operation`. +/// It returns a [`ResolutionResult`], which can either instruct the watcher to restart or ignore the change. +/// This always contains paths to watch; /// /// - `operation` is the actual operation we want to run every time the watcher detects file /// changes. For example, in the case where we would like to bundle, then `operation` would -/// have the logic for it like doing bundle with the help of [`ModuleGraph`]. +/// have the logic for it like bundling the code. /// /// - `job_name` is just used for printing watcher status to terminal. -/// -/// Note that the watcher will try to continue watching files using the previously resolved -/// data if `module_resolver` fails at some point, which means the watcher won't work at all -/// if `module_resolver` fails at the first attempt. -/// -/// [`ModuleGraph`]: crate::module_graph::Graph -/// [`ModuleSpecifier`]: deno_core::ModuleSpecifier -pub async fn watch_func_with_module_resolution( - module_resolver: F, - operation: G, +pub async fn watch_func( + mut resolver: R, + mut operation: O, job_name: &str, ) -> Result<(), AnyError> where - F: Fn() -> FileWatcherFuture>, - G: Fn(T) -> FileWatcherFuture>, - T: Clone, + R: FnMut(Option>) -> F1, + O: FnMut(T) -> F2, + F1: Future>, + F2: Future>, { let debounce = Debounce::new(); pin!(debounce); // Store previous data. If module resolution fails at some point, the watcher will try to // continue watching files using these data. - let mut paths = Vec::new(); - let mut module = None; + let mut paths_to_watch; + let mut resolution_result; + + match resolver(None).await { + ResolutionResult::Ignore => { + // The only situation where it makes sense to ignore the initial 'change' + // is if the command isn't supposed to do anything until something changes, + // e.g. a variant of `deno test` which doesn't run the entire test suite to start with, + // but instead does nothing until you make a change. + // + // In that case, this is probably the correct output. + info!( + "{} Waiting for file changes...", + colors::intense_blue("Watcher"), + ); + + let (paths, result) = next_restart(&mut resolver, &mut debounce).await; + paths_to_watch = paths; + resolution_result = result; + } + ResolutionResult::Restart { + paths_to_watch: paths, + result, + } => { + paths_to_watch = paths; + resolution_result = result; + } + }; loop { - match module_resolver().await { - ModuleResolutionResult::Success { - paths_to_watch, - module_info, - } => { - paths = paths_to_watch; - module = Some(module_info); - } - ModuleResolutionResult::Fail { source_path, error } => { - if paths.is_empty() { - paths = vec![source_path]; - } + let watcher = new_watcher(&paths_to_watch, &debounce)?; - if module.is_none() { - eprintln!("{}: {}", colors::red_bold("error"), error); - } - } - } - let _watcher = new_watcher(&paths, &debounce)?; + match resolution_result { + Ok(operation_arg) => { + let fut = error_handler(operation(operation_arg)); + select! { + (paths, result) = next_restart(&mut resolver, &mut debounce) => { + if result.is_ok() { + paths_to_watch = paths; + } + resolution_result = result; + continue; + }, + _ = fut => {}, + }; - if let Some(module) = &module { - let func = error_handler(operation(module.clone())); - let mut is_file_changed = false; - select! { - _ = debounce.next() => { - is_file_changed = true; - info!( - "{} File change detected! Restarting!", - colors::intense_blue("Watcher"), - ); - }, - _ = func => {}, - }; - - if !is_file_changed { info!( - "{} {} finished! Restarting on file change...", + "{} {} finished. Restarting on file change...", colors::intense_blue("Watcher"), job_name, ); - debounce.next().await; + } + Err(error) => { + eprintln!("{}: {}", colors::red_bold("error"), error); info!( - "{} File change detected! Restarting!", + "{} {} failed. Restarting on file change...", colors::intense_blue("Watcher"), + job_name, ); } - } else { - info!( - "{} {} failed! Restarting on file change...", - colors::intense_blue("Watcher"), - job_name, - ); - debounce.next().await; - info!( - "{} File change detected! Restarting!", - colors::intense_blue("Watcher"), - ); } + + let (paths, result) = next_restart(&mut resolver, &mut debounce).await; + if result.is_ok() { + paths_to_watch = paths; + } + resolution_result = result; + + drop(watcher); } } @@ -248,7 +219,7 @@ fn new_watcher( paths: &[PathBuf], debounce: &Debounce, ) -> Result { - let event_detected = Arc::clone(&debounce.event_detected); + let changed_paths = Arc::clone(&debounce.changed_paths); let mut watcher: RecommendedWatcher = Watcher::new_immediate(move |res: Result| { @@ -257,16 +228,22 @@ fn new_watcher( event.kind, EventKind::Create(_) | EventKind::Modify(_) | EventKind::Remove(_) ) { - event_detected.store(true, Ordering::Relaxed); + let paths = event + .paths + .iter() + .filter_map(|path| path.canonicalize().ok()); + let mut changed_paths = changed_paths.lock().unwrap(); + changed_paths.extend(paths); } } })?; watcher.configure(Config::PreciseEvents(true)).unwrap(); + log::debug!("Watching paths: {:?}", paths); for path in paths { // Ignore any error e.g. `PathNotFound` - let _ = watcher.watch(path, RecursiveMode::NonRecursive); + let _ = watcher.watch(path, RecursiveMode::Recursive); } Ok(watcher) diff --git a/cli/flags.rs b/cli/flags.rs index 0c916ecb8f..6f23afba16 100644 --- a/cli/flags.rs +++ b/cli/flags.rs @@ -1031,6 +1031,11 @@ fn test_subcommand<'a, 'b>() -> App<'a, 'b> { .takes_value(true) .multiple(true), ) + .arg( + watch_arg() + .conflicts_with("no-run") + .conflicts_with("coverage"), + ) .arg(script_arg().last(true)) .about("Run tests") .long_about( @@ -1667,6 +1672,8 @@ fn test_parse(flags: &mut Flags, matches: &clap::ArgMatches) { let quiet = matches.is_present("quiet"); let filter = matches.value_of("filter").map(String::from); + flags.watch = matches.is_present("watch"); + if matches.is_present("script_arg") { let script_arg: Vec = matches .values_of("script_arg") diff --git a/cli/main.rs b/cli/main.rs index ba3abc43d2..d20462c96f 100644 --- a/cli/main.rs +++ b/cli/main.rs @@ -38,11 +38,13 @@ mod unix_util; mod version; use crate::file_fetcher::File; -use crate::file_watcher::ModuleResolutionResult; +use crate::file_watcher::ResolutionResult; use crate::flags::DenoSubcommand; use crate::flags::Flags; use crate::fmt_errors::PrettyJsError; use crate::media_type::MediaType; +use crate::module_graph::GraphBuilder; +use crate::module_graph::Module; use crate::module_loader::CliModuleLoader; use crate::program_state::ProgramState; use crate::source_maps::apply_source_map; @@ -67,6 +69,7 @@ use log::debug; use log::info; use log::Level; use log::LevelFilter; +use std::collections::HashSet; use std::env; use std::io::Read; use std::io::Write; @@ -76,6 +79,7 @@ use std::pin::Pin; use std::rc::Rc; use std::sync::Arc; use std::sync::Mutex; +use tools::test_runner; fn create_web_worker_callback( program_state: Arc, @@ -595,7 +599,7 @@ async fn bundle_command( ) -> Result<(), AnyError> { let debug = flags.log_level == Some(log::Level::Debug); - let module_resolver = || { + let resolver = |_| { let flags = flags.clone(); let source_file1 = source_file.clone(); let source_file2 = source_file.clone(); @@ -605,12 +609,6 @@ async fn bundle_command( debug!(">>>>> bundle START"); let program_state = ProgramState::build(flags.clone()).await?; - info!( - "{} {}", - colors::green("Bundle"), - module_specifier.to_string() - ); - let module_graph = create_module_graph_and_maybe_check( module_specifier, program_state.clone(), @@ -632,22 +630,23 @@ async fn bundle_command( Ok((paths_to_watch, module_graph)) } .map(move |result| match result { - Ok((paths_to_watch, module_graph)) => ModuleResolutionResult::Success { + Ok((paths_to_watch, module_graph)) => ResolutionResult::Restart { paths_to_watch, - module_info: module_graph, + result: Ok(module_graph), }, - Err(e) => ModuleResolutionResult::Fail { - source_path: PathBuf::from(source_file2), - error: e, + Err(e) => ResolutionResult::Restart { + paths_to_watch: vec![PathBuf::from(source_file2)], + result: Err(e), }, }) - .boxed_local() }; let operation = |module_graph: module_graph::Graph| { let flags = flags.clone(); let out_file = out_file.clone(); async move { + info!("{} {}", colors::green("Bundle"), module_graph.info()?.root); + let output = bundle_module_graph(module_graph, flags, debug)?; debug!(">>>>> bundle END"); @@ -668,21 +667,17 @@ async fn bundle_command( Ok(()) } - .boxed_local() }; if flags.watch { - file_watcher::watch_func_with_module_resolution( - module_resolver, - operation, - "Bundle", - ) - .await?; + file_watcher::watch_func(resolver, operation, "Bundle").await?; } else { - let module_graph = match module_resolver().await { - ModuleResolutionResult::Fail { error, .. } => return Err(error), - ModuleResolutionResult::Success { module_info, .. } => module_info, - }; + let module_graph = + if let ResolutionResult::Restart { result, .. } = resolver(None).await { + result? + } else { + unreachable!(); + }; operation(module_graph).await?; } @@ -759,7 +754,7 @@ async fn run_from_stdin(flags: Flags) -> Result<(), AnyError> { } async fn run_with_watch(flags: Flags, script: String) -> Result<(), AnyError> { - let module_resolver = || { + let resolver = |_| { let script1 = script.clone(); let script2 = script.clone(); let flags = flags.clone(); @@ -793,16 +788,15 @@ async fn run_with_watch(flags: Flags, script: String) -> Result<(), AnyError> { Ok((paths_to_watch, main_module)) } .map(move |result| match result { - Ok((paths_to_watch, module_info)) => ModuleResolutionResult::Success { + Ok((paths_to_watch, module_info)) => ResolutionResult::Restart { paths_to_watch, - module_info, + result: Ok(module_info), }, - Err(e) => ModuleResolutionResult::Fail { - source_path: PathBuf::from(script2), - error: e, + Err(e) => ResolutionResult::Restart { + paths_to_watch: vec![PathBuf::from(script2)], + result: Err(e), }, }) - .boxed_local() }; let operation = |main_module: ModuleSpecifier| { @@ -824,15 +818,9 @@ async fn run_with_watch(flags: Flags, script: String) -> Result<(), AnyError> { worker.execute("window.dispatchEvent(new Event('unload'))")?; Ok(()) } - .boxed_local() }; - file_watcher::watch_func_with_module_resolution( - module_resolver, - operation, - "Process", - ) - .await + file_watcher::watch_func(resolver, operation, "Process").await } async fn run_command(flags: Flags, script: String) -> Result<(), AnyError> { @@ -917,17 +905,187 @@ async fn test_command( env::set_var("DENO_UNSTABLE_COVERAGE_DIR", coverage_dir); } - tools::test_runner::run_tests( - flags, - include, - no_run, - fail_fast, - quiet, - allow_none, - filter, - concurrent_jobs, - ) - .await?; + let program_state = ProgramState::build(flags.clone()).await?; + + let include = include.unwrap_or_else(|| vec![".".to_string()]); + let cwd = std::env::current_dir().expect("No current directory"); + + let permissions = Permissions::from_options(&flags.clone().into()); + let lib = if flags.unstable { + module_graph::TypeLib::UnstableDenoWindow + } else { + module_graph::TypeLib::DenoWindow + }; + + if flags.watch { + let handler = Arc::new(Mutex::new(FetchHandler::new( + &program_state, + Permissions::allow_all(), + )?)); + + let paths_to_watch: Vec<_> = include.iter().map(PathBuf::from).collect(); + + let resolver = |changed: Option>| { + let test_modules_result = + test_runner::collect_test_module_specifiers(include.clone(), &cwd); + let paths_to_watch = paths_to_watch.clone(); + let paths_to_watch_clone = paths_to_watch.clone(); + + let handler = handler.clone(); + let program_state = program_state.clone(); + let files_changed = changed.is_some(); + async move { + let test_modules = test_modules_result?; + + let mut paths_to_watch = paths_to_watch_clone; + let mut modules_to_reload = if files_changed { + Vec::new() + } else { + test_modules + .iter() + .filter_map(|url| deno_core::resolve_url(url.as_str()).ok()) + .collect() + }; + + let mut builder = GraphBuilder::new( + handler, + program_state.maybe_import_map.clone(), + program_state.lockfile.clone(), + ); + for specifier in test_modules.iter() { + builder.add(specifier, false).await?; + } + let graph = builder.get_graph(); + + for specifier in test_modules { + fn get_dependencies<'a>( + graph: &'a module_graph::Graph, + module: &'a Module, + // This needs to be accessible to skip getting dependencies if they're already there, + // otherwise this will cause a stack overflow with circular dependencies + output: &mut HashSet<&'a ModuleSpecifier>, + ) -> Result<(), AnyError> { + for dep in module.dependencies.values() { + if let Some(specifier) = &dep.maybe_code { + if !output.contains(specifier) { + output.insert(specifier); + + get_dependencies( + &graph, + graph.get_specifier(specifier)?, + output, + )?; + } + } + if let Some(specifier) = &dep.maybe_type { + if !output.contains(specifier) { + output.insert(specifier); + + get_dependencies( + &graph, + graph.get_specifier(specifier)?, + output, + )?; + } + } + } + + Ok(()) + } + + // This test module and all it's dependencies + let mut modules = HashSet::new(); + modules.insert(&specifier); + get_dependencies( + &graph, + graph.get_specifier(&specifier)?, + &mut modules, + )?; + + paths_to_watch.extend( + modules + .iter() + .filter_map(|specifier| specifier.to_file_path().ok()), + ); + + if let Some(changed) = &changed { + for path in changed.iter().filter_map(|path| { + deno_core::resolve_url_or_path(&path.to_string_lossy()).ok() + }) { + if modules.contains(&&path) { + modules_to_reload.push(specifier); + break; + } + } + } + } + + Ok((paths_to_watch, modules_to_reload)) + } + .map(move |result| { + if files_changed + && matches!(result, Ok((_, ref modules)) if modules.is_empty()) + { + ResolutionResult::Ignore + } else { + match result { + Ok((paths_to_watch, modules_to_reload)) => { + ResolutionResult::Restart { + paths_to_watch, + result: Ok(modules_to_reload), + } + } + Err(e) => ResolutionResult::Restart { + paths_to_watch, + result: Err(e), + }, + } + } + }) + }; + + file_watcher::watch_func( + resolver, + |modules_to_reload| { + test_runner::run_tests( + program_state.clone(), + permissions.clone(), + lib.clone(), + modules_to_reload, + no_run, + fail_fast, + quiet, + true, + filter.clone(), + concurrent_jobs, + ) + .map(|res| res.map(|_| ())) + }, + "Test", + ) + .await?; + } else { + let test_modules = + test_runner::collect_test_module_specifiers(include, &cwd)?; + + let failed = test_runner::run_tests( + program_state.clone(), + permissions, + lib, + test_modules, + no_run, + fail_fast, + quiet, + allow_none, + filter, + concurrent_jobs, + ) + .await?; + + if failed { + std::process::exit(1); + } + } Ok(()) } diff --git a/cli/module_graph.rs b/cli/module_graph.rs index 93930c8a5b..2300e89d69 100644 --- a/cli/module_graph.rs +++ b/cli/module_graph.rs @@ -1248,6 +1248,18 @@ impl Graph { self.modules.get_mut(s) } + pub fn get_specifier( + &self, + specifier: &ModuleSpecifier, + ) -> Result<&Module, AnyError> { + let s = self.resolve_specifier(specifier); + match self.get_module(s) { + ModuleSlot::Module(m) => Ok(m.as_ref()), + ModuleSlot::Err(e) => Err(anyhow!(e.to_string())), + _ => Err(GraphError::MissingSpecifier(specifier.clone()).into()), + } + } + /// Consume graph and return list of all module specifiers contained in the /// graph. pub fn get_modules(&self) -> Vec { diff --git a/cli/tests/integration_tests.rs b/cli/tests/integration_tests.rs index f19966fdff..674f42f870 100644 --- a/cli/tests/integration_tests.rs +++ b/cli/tests/integration_tests.rs @@ -617,6 +617,21 @@ mod integration { } } + /// Helper function to skip watcher output that doesn't contain + /// "{job_name} failed" phrase. + fn wait_for_process_failed( + job_name: &str, + stderr_lines: &mut impl Iterator, + ) { + let phrase = format!("{} failed", job_name); + loop { + let msg = stderr_lines.next().unwrap(); + if msg.contains(&phrase) { + break; + } + } + } + #[test] fn fmt_watch_test() { let t = TempDir::new().expect("tempdir fail"); @@ -685,6 +700,7 @@ mod integration { .arg(&bundle) .arg("--watch") .arg("--unstable") + .env("NO_COLOR", "1") .stdout(std::process::Stdio::piped()) .stderr(std::process::Stdio::piped()) .spawn() @@ -722,11 +738,8 @@ mod integration { .next() .unwrap() .contains("File change detected!")); - assert!(stderr_lines.next().unwrap().contains("file_to_watch.js")); - assert!(stderr_lines.next().unwrap().contains("mod6.bundle.js")); - let file = PathBuf::from(&bundle); - assert!(file.is_file()); - wait_for_process_finished("Bundle", &mut stderr_lines); + assert!(stderr_lines.next().unwrap().contains("error: ")); + wait_for_process_failed("Bundle", &mut stderr_lines); // the watcher process is still alive assert!(deno.try_wait().unwrap().is_none()); @@ -762,9 +775,8 @@ mod integration { std::io::BufReader::new(stderr).lines().map(|r| r.unwrap()); std::thread::sleep(std::time::Duration::from_secs(1)); - assert!(stderr_lines.next().unwrap().contains("file_to_watch.js")); assert!(stderr_lines.next().unwrap().contains("error:")); - assert!(stderr_lines.next().unwrap().contains("Bundle failed!")); + assert!(stderr_lines.next().unwrap().contains("Bundle failed")); // the target file hasn't been created yet assert!(!target_file.is_file()); @@ -858,7 +870,7 @@ mod integration { std::thread::sleep(std::time::Duration::from_secs(1)); assert!(stderr_lines.next().unwrap().contains("Restarting")); assert!(stderr_lines.next().unwrap().contains("error:")); - wait_for_process_finished("Process", &mut stderr_lines); + wait_for_process_failed("Process", &mut stderr_lines); // Then restore the file std::fs::write( @@ -877,7 +889,7 @@ mod integration { std::thread::sleep(std::time::Duration::from_secs(1)); assert!(stderr_lines.next().unwrap().contains("Restarting")); assert!(stderr_lines.next().unwrap().contains("error:")); - wait_for_process_finished("Process", &mut stderr_lines); + wait_for_process_failed("Process", &mut stderr_lines); // Modify the imported file and make sure that restarting occurs std::fs::write(&another_file, "export const foo = 'modified!';") @@ -923,7 +935,7 @@ mod integration { std::thread::sleep(std::time::Duration::from_secs(1)); assert!(stderr_lines.next().unwrap().contains("error:")); - assert!(stderr_lines.next().unwrap().contains("Process failed!")); + assert!(stderr_lines.next().unwrap().contains("Process failed")); // Make sure the watcher actually restarts and works fine with the proper syntax std::fs::write(&file_to_watch, "console.log(42);") @@ -999,6 +1011,174 @@ mod integration { drop(import_map_path); temp_directory.close().unwrap(); } + + #[test] + fn test_watch() { + macro_rules! assert_contains { + ($string:expr, $($test:expr),+) => { + let string = $string; // This might be a function call or something + if !($(string.contains($test))||+) { + panic!("{:?} does not contain any of {:?}", string, [$($test),+]); + } + } + } + + let t = TempDir::new().expect("tempdir fail"); + + let mut child = util::deno_cmd() + .current_dir(util::root_path()) + .arg("test") + .arg("--watch") + .arg("--unstable") + .arg("--no-check") + .arg(&t.path()) + .env("NO_COLOR", "1") + .stdout(std::process::Stdio::piped()) + .stderr(std::process::Stdio::piped()) + .spawn() + .expect("failed to spawn script"); + + let stdout = child.stdout.as_mut().unwrap(); + let mut stdout_lines = + std::io::BufReader::new(stdout).lines().map(|r| r.unwrap()); + let stderr = child.stderr.as_mut().unwrap(); + let mut stderr_lines = + std::io::BufReader::new(stderr).lines().map(|r| r.unwrap()); + + assert_contains!( + stdout_lines.next().unwrap(), + "No matching test modules found" + ); + wait_for_process_finished("Test", &mut stderr_lines); + + let foo_file = t.path().join("foo.js"); + let bar_file = t.path().join("bar.js"); + let foo_test = t.path().join("foo_test.js"); + let bar_test = t.path().join("bar_test.js"); + std::fs::write(&foo_file, "export default function foo() { 1 + 1 }") + .expect("error writing file"); + std::fs::write(&bar_file, "export default function bar() { 2 + 2 }") + .expect("error writing file"); + std::fs::write( + &foo_test, + "import foo from './foo.js'; Deno.test('foo', foo);", + ) + .expect("error writing file"); + std::fs::write( + &bar_test, + "import bar from './bar.js'; Deno.test('bar', bar);", + ) + .expect("error writing file"); + + assert_contains!(stdout_lines.next().unwrap(), "running 1 test"); + assert_contains!(stdout_lines.next().unwrap(), "foo", "bar"); + assert_contains!(stdout_lines.next().unwrap(), "running 1 test"); + assert_contains!(stdout_lines.next().unwrap(), "foo", "bar"); + stdout_lines.next(); + stdout_lines.next(); + stdout_lines.next(); + wait_for_process_finished("Test", &mut stderr_lines); + + // Change content of the file + std::fs::write( + &foo_test, + "import foo from './foo.js'; Deno.test('foobar', foo);", + ) + .expect("error writing file"); + + assert_contains!(stderr_lines.next().unwrap(), "Restarting"); + assert_contains!(stdout_lines.next().unwrap(), "running 1 test"); + assert_contains!(stdout_lines.next().unwrap(), "foobar"); + stdout_lines.next(); + stdout_lines.next(); + stdout_lines.next(); + wait_for_process_finished("Test", &mut stderr_lines); + + // Add test + let another_test = t.path().join("new_test.js"); + std::fs::write(&another_test, "Deno.test('another one', () => 3 + 3)") + .expect("error writing file"); + assert_contains!(stderr_lines.next().unwrap(), "Restarting"); + assert_contains!(stdout_lines.next().unwrap(), "running 1 test"); + assert_contains!(stdout_lines.next().unwrap(), "another one"); + stdout_lines.next(); + stdout_lines.next(); + stdout_lines.next(); + wait_for_process_finished("Test", &mut stderr_lines); + + // Confirm that restarting occurs when a new file is updated + std::fs::write(&another_test, "Deno.test('another one', () => 3 + 3); Deno.test('another another one', () => 4 + 4)") + .expect("error writing file"); + assert_contains!(stderr_lines.next().unwrap(), "Restarting"); + assert_contains!(stdout_lines.next().unwrap(), "running 2 tests"); + assert_contains!(stdout_lines.next().unwrap(), "another one"); + assert_contains!(stdout_lines.next().unwrap(), "another another one"); + stdout_lines.next(); + stdout_lines.next(); + stdout_lines.next(); + wait_for_process_finished("Test", &mut stderr_lines); + + // Confirm that the watcher keeps on working even if the file is updated and has invalid syntax + std::fs::write(&another_test, "syntax error ^^") + .expect("error writing file"); + assert_contains!(stderr_lines.next().unwrap(), "Restarting"); + assert_contains!(stderr_lines.next().unwrap(), "error:"); + assert_contains!(stderr_lines.next().unwrap(), "Test failed"); + + // Then restore the file + std::fs::write(&another_test, "Deno.test('another one', () => 3 + 3)") + .expect("error writing file"); + assert_contains!(stderr_lines.next().unwrap(), "Restarting"); + assert_contains!(stdout_lines.next().unwrap(), "running 1 test"); + assert_contains!(stdout_lines.next().unwrap(), "another one"); + stdout_lines.next(); + stdout_lines.next(); + stdout_lines.next(); + wait_for_process_finished("Test", &mut stderr_lines); + + // Confirm that the watcher keeps on working even if the file is updated and the test fails + // This also confirms that it restarts when dependencies change + std::fs::write( + &foo_file, + "export default function foo() { throw new Error('Whoops!'); }", + ) + .expect("error writing file"); + assert_contains!(stderr_lines.next().unwrap(), "Restarting"); + assert_contains!(stdout_lines.next().unwrap(), "running 1 test"); + assert_contains!(stdout_lines.next().unwrap(), "FAILED"); + while !stdout_lines.next().unwrap().contains("test result") {} + stdout_lines.next(); + wait_for_process_finished("Test", &mut stderr_lines); + + // Then restore the file + std::fs::write(&foo_file, "export default function foo() { 1 + 1 }") + .expect("error writing file"); + assert_contains!(stderr_lines.next().unwrap(), "Restarting"); + assert_contains!(stdout_lines.next().unwrap(), "running 1 test"); + assert_contains!(stdout_lines.next().unwrap(), "foo"); + stdout_lines.next(); + stdout_lines.next(); + stdout_lines.next(); + wait_for_process_finished("Test", &mut stderr_lines); + + // Test that circular dependencies work fine + std::fs::write( + &foo_file, + "import './bar.js'; export default function foo() { 1 + 1 }", + ) + .expect("error writing file"); + std::fs::write( + &bar_file, + "import './foo.js'; export default function bar() { 2 + 2 }", + ) + .expect("error writing file"); + + // the watcher process is still alive + assert!(child.try_wait().unwrap().is_none()); + + child.kill().unwrap(); + drop(t); + } } #[test] diff --git a/cli/tools/fmt.rs b/cli/tools/fmt.rs index 91d730c448..9a16afecab 100644 --- a/cli/tools/fmt.rs +++ b/cli/tools/fmt.rs @@ -10,12 +10,12 @@ use crate::colors; use crate::diff::diff; use crate::file_watcher; +use crate::file_watcher::ResolutionResult; use crate::fs_util::{collect_files, get_extension, is_supported_ext_fmt}; use crate::text_encoding; use deno_core::error::generic_error; use deno_core::error::AnyError; use deno_core::futures; -use deno_core::futures::FutureExt; use log::debug; use log::info; use std::fs; @@ -37,15 +37,32 @@ pub async fn format( check: bool, watch: bool, ) -> Result<(), AnyError> { - let target_file_resolver = || { - // collect the files that are to be formatted - collect_files(&args, &ignore, is_supported_ext_fmt).and_then(|files| { - if files.is_empty() { - Err(generic_error("No target files found.")) + let resolver = |changed: Option>| { + let files_changed = changed.is_some(); + let result = + collect_files(&args, &ignore, is_supported_ext_fmt).map(|files| { + if let Some(paths) = changed { + files + .into_iter() + .filter(|path| paths.contains(path)) + .collect::>() + } else { + files + } + }); + let paths_to_watch = args.clone(); + async move { + if (files_changed || !watch) + && matches!(result, Ok(ref files) if files.is_empty()) + { + ResolutionResult::Ignore } else { - Ok(files) + ResolutionResult::Restart { + paths_to_watch, + result, + } } - }) + } }; let operation = |paths: Vec| { let config = get_typescript_config(); @@ -57,13 +74,18 @@ pub async fn format( } Ok(()) } - .boxed_local() }; if watch { - file_watcher::watch_func(target_file_resolver, operation, "Fmt").await?; + file_watcher::watch_func(resolver, operation, "Fmt").await?; } else { - operation(target_file_resolver()?).await?; + let files = + if let ResolutionResult::Restart { result, .. } = resolver(None).await { + result? + } else { + return Err(generic_error("No target files found.")); + }; + operation(files).await?; } Ok(()) diff --git a/cli/tools/test_runner.rs b/cli/tools/test_runner.rs index 7f3a139a4b..e24d8b4587 100644 --- a/cli/tools/test_runner.rs +++ b/cli/tools/test_runner.rs @@ -3,7 +3,6 @@ use crate::colors; use crate::create_main_worker; use crate::file_fetcher::File; -use crate::flags::Flags; use crate::fs_util; use crate::media_type::MediaType; use crate::module_graph; @@ -304,37 +303,30 @@ pub async fn run_test_file( Ok(()) } +/// Runs tests. +/// +/// Returns a boolean indicating whether the tests failed. #[allow(clippy::too_many_arguments)] pub async fn run_tests( - flags: Flags, - include: Option>, + program_state: Arc, + permissions: Permissions, + lib: module_graph::TypeLib, + test_modules: Vec, no_run: bool, fail_fast: bool, quiet: bool, allow_none: bool, filter: Option, concurrent_jobs: usize, -) -> Result<(), AnyError> { - let program_state = ProgramState::build(flags.clone()).await?; - let permissions = Permissions::from_options(&flags.clone().into()); - let cwd = std::env::current_dir().expect("No current directory"); - let include = include.unwrap_or_else(|| vec![".".to_string()]); - let test_modules = collect_test_module_specifiers(include, &cwd)?; - +) -> Result { if test_modules.is_empty() { println!("No matching test modules found"); if !allow_none { std::process::exit(1); } - return Ok(()); + return Ok(false); } - let lib = if flags.unstable { - module_graph::TypeLib::UnstableDenoWindow - } else { - module_graph::TypeLib::DenoWindow - }; - program_state .prepare_module_graph( test_modules.clone(), @@ -345,7 +337,7 @@ pub async fn run_tests( .await?; if no_run { - return Ok(()); + return Ok(false); } // Because scripts, and therefore worker.execute cannot detect unresolved promises at the moment @@ -475,11 +467,7 @@ pub async fn run_tests( if let Some(e) = join_errors.next() { Err(e) } else { - if result.unwrap_or(false) { - std::process::exit(1); - } - - Ok(()) + Ok(result.unwrap_or(false)) } }