summaryrefslogtreecommitdiff
path: root/cli
diff options
context:
space:
mode:
Diffstat (limited to 'cli')
-rw-r--r--cli/file_watcher.rs261
-rw-r--r--cli/flags.rs7
-rw-r--r--cli/main.rs258
-rw-r--r--cli/module_graph.rs12
-rw-r--r--cli/tests/integration_tests.rs200
-rw-r--r--cli/tools/fmt.rs44
-rw-r--r--cli/tools/test_runner.rs34
7 files changed, 580 insertions, 236 deletions
diff --git a/cli/file_watcher.rs b/cli/file_watcher.rs
index 651a3ee31..7251dbe8c 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<T> = Pin<Box<dyn Future<Output = T>>>;
-
#[pin_project(project = DebounceProjection)]
struct Debounce {
#[pin]
timer: Sleep,
- event_detected: Arc<AtomicBool>,
+ changed_paths: Arc<Mutex<HashSet<PathBuf>>>,
}
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<PathBuf>;
/// 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<Option<Self::Item>> {
- 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<Result<(), AnyError>>) {
+async fn error_handler<F>(watch_future: F)
+where
+ F: Future<Output = Result<(), AnyError>>,
+{
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<Result<(), AnyError>>) {
}
}
-/// 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<F, G>(
- target_resolver: F,
- operation: G,
- job_name: &str,
-) -> Result<(), AnyError>
+pub enum ResolutionResult<T> {
+ Restart {
+ paths_to_watch: Vec<PathBuf>,
+ result: Result<T, AnyError>,
+ },
+ Ignore,
+}
+
+async fn next_restart<R, T, F>(
+ resolver: &mut R,
+ debounce: &mut Pin<&mut Debounce>,
+) -> (Vec<PathBuf>, Result<T, AnyError>)
where
- F: Fn() -> Result<Vec<PathBuf>, AnyError>,
- G: Fn(Vec<PathBuf>) -> FileWatcherFuture<Result<(), AnyError>>,
+ R: FnMut(Option<Vec<PathBuf>>) -> F,
+ F: Future<Output = ResolutionResult<T>>,
{
- let debounce = Debounce::new();
- pin!(debounce);
-
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<T> {
- Success {
- paths_to_watch: Vec<PathBuf>,
- 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<F, G, T>(
- module_resolver: F,
- operation: G,
+pub async fn watch_func<R, O, T, F1, F2>(
+ mut resolver: R,
+ mut operation: O,
job_name: &str,
) -> Result<(), AnyError>
where
- F: Fn() -> FileWatcherFuture<ModuleResolutionResult<T>>,
- G: Fn(T) -> FileWatcherFuture<Result<(), AnyError>>,
- T: Clone,
+ R: FnMut(Option<Vec<PathBuf>>) -> F1,
+ O: FnMut(T) -> F2,
+ F1: Future<Output = ResolutionResult<T>>,
+ F2: Future<Output = Result<(), AnyError>>,
{
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;
-
- 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 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"),
+ );
- if module.is_none() {
- eprintln!("{}: {}", colors::red_bold("error"), error);
- }
- }
+ 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;
}
- let _watcher = new_watcher(&paths, &debounce)?;
+ };
- 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 => {},
- };
+ loop {
+ let watcher = new_watcher(&paths_to_watch, &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 !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<RecommendedWatcher, AnyError> {
- 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<NotifyEvent, NotifyError>| {
@@ -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 0c916ecb8..6f23afba1 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<String> = matches
.values_of("script_arg")
diff --git a/cli/main.rs b/cli/main.rs
index ba3abc43d..d20462c96 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<ProgramState>,
@@ -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<Vec<PathBuf>>| {
+ 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 93930c8a5..2300e89d6 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<ModuleSpecifier> {
diff --git a/cli/tests/integration_tests.rs b/cli/tests/integration_tests.rs
index f19966fdf..674f42f87 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<Item = String>,
+ ) {
+ 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 91d730c44..9a16afeca 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<Vec<PathBuf>>| {
+ 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::<Vec<_>>()
+ } 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<PathBuf>| {
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 7f3a139a4..e24d8b458 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<Vec<String>>,
+ program_state: Arc<ProgramState>,
+ permissions: Permissions,
+ lib: module_graph::TypeLib,
+ test_modules: Vec<ModuleSpecifier>,
no_run: bool,
fail_fast: bool,
quiet: bool,
allow_none: bool,
filter: Option<String>,
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<bool, AnyError> {
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))
}
}