mirror of
https://github.com/denoland/deno.git
synced 2024-11-25 15:29:32 -05:00
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.
This commit is contained in:
parent
84733d90c7
commit
7a9ebd1585
7 changed files with 581 additions and 237 deletions
|
@ -2,7 +2,6 @@
|
||||||
|
|
||||||
use crate::colors;
|
use crate::colors;
|
||||||
use deno_core::error::AnyError;
|
use deno_core::error::AnyError;
|
||||||
use deno_core::futures::ready;
|
|
||||||
use deno_core::futures::stream::{Stream, StreamExt};
|
use deno_core::futures::stream::{Stream, StreamExt};
|
||||||
use deno_core::futures::Future;
|
use deno_core::futures::Future;
|
||||||
use log::info;
|
use log::info;
|
||||||
|
@ -14,10 +13,11 @@ use notify::RecommendedWatcher;
|
||||||
use notify::RecursiveMode;
|
use notify::RecursiveMode;
|
||||||
use notify::Watcher;
|
use notify::Watcher;
|
||||||
use pin_project::pin_project;
|
use pin_project::pin_project;
|
||||||
|
use std::collections::HashSet;
|
||||||
use std::path::PathBuf;
|
use std::path::PathBuf;
|
||||||
use std::pin::Pin;
|
use std::pin::Pin;
|
||||||
use std::sync::atomic::{AtomicBool, Ordering};
|
|
||||||
use std::sync::Arc;
|
use std::sync::Arc;
|
||||||
|
use std::sync::Mutex;
|
||||||
use std::task::Context;
|
use std::task::Context;
|
||||||
use std::task::Poll;
|
use std::task::Poll;
|
||||||
use std::time::Duration;
|
use std::time::Duration;
|
||||||
|
@ -29,26 +29,24 @@ use tokio::time::Sleep;
|
||||||
|
|
||||||
const DEBOUNCE_INTERVAL: Duration = Duration::from_millis(200);
|
const DEBOUNCE_INTERVAL: Duration = Duration::from_millis(200);
|
||||||
|
|
||||||
type FileWatcherFuture<T> = Pin<Box<dyn Future<Output = T>>>;
|
|
||||||
|
|
||||||
#[pin_project(project = DebounceProjection)]
|
#[pin_project(project = DebounceProjection)]
|
||||||
struct Debounce {
|
struct Debounce {
|
||||||
#[pin]
|
#[pin]
|
||||||
timer: Sleep,
|
timer: Sleep,
|
||||||
event_detected: Arc<AtomicBool>,
|
changed_paths: Arc<Mutex<HashSet<PathBuf>>>,
|
||||||
}
|
}
|
||||||
|
|
||||||
impl Debounce {
|
impl Debounce {
|
||||||
fn new() -> Self {
|
fn new() -> Self {
|
||||||
Self {
|
Self {
|
||||||
timer: sleep(DEBOUNCE_INTERVAL),
|
timer: sleep(DEBOUNCE_INTERVAL),
|
||||||
event_detected: Arc::new(AtomicBool::new(false)),
|
changed_paths: Arc::new(Mutex::new(HashSet::new())),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
impl Stream for Debounce {
|
impl Stream for Debounce {
|
||||||
type Item = ();
|
type Item = Vec<PathBuf>;
|
||||||
|
|
||||||
/// Note that this never returns `Poll::Ready(None)`, which means that the
|
/// Note that this never returns `Poll::Ready(None)`, which means that the
|
||||||
/// file watcher will be alive until the Deno process is terminated.
|
/// file watcher will be alive until the Deno process is terminated.
|
||||||
|
@ -56,18 +54,24 @@ impl Stream for Debounce {
|
||||||
self: Pin<&mut Self>,
|
self: Pin<&mut Self>,
|
||||||
cx: &mut Context,
|
cx: &mut Context,
|
||||||
) -> Poll<Option<Self::Item>> {
|
) -> Poll<Option<Self::Item>> {
|
||||||
if self.event_detected.swap(false, Ordering::Relaxed) {
|
let mut changed_paths = self.changed_paths.lock().unwrap();
|
||||||
Poll::Ready(Some(()))
|
if changed_paths.len() > 0 {
|
||||||
|
Poll::Ready(Some(changed_paths.drain().collect()))
|
||||||
} else {
|
} else {
|
||||||
|
drop(changed_paths);
|
||||||
let mut timer = self.project().timer;
|
let mut timer = self.project().timer;
|
||||||
ready!(timer.as_mut().poll(cx));
|
if timer.as_mut().poll(cx).is_ready() {
|
||||||
timer.reset(Instant::now() + DEBOUNCE_INTERVAL);
|
timer.reset(Instant::now() + DEBOUNCE_INTERVAL);
|
||||||
|
}
|
||||||
Poll::Pending
|
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;
|
let result = watch_future.await;
|
||||||
if let Err(err) = result {
|
if let Err(err) = result {
|
||||||
let msg = format!("{}: {}", colors::red_bold("error"), err.to_string(),);
|
let msg = format!("{}: {}", colors::red_bold("error"), err.to_string(),);
|
||||||
|
@ -75,180 +79,147 @@ async fn error_handler(watch_future: FileWatcherFuture<Result<(), AnyError>>) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/// This function adds watcher functionality to subcommands like `fmt` or `lint`.
|
pub enum ResolutionResult<T> {
|
||||||
/// The difference from [`watch_func_with_module_resolution`] is that this doesn't depend on
|
Restart {
|
||||||
/// [`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>
|
|
||||||
where
|
|
||||||
F: Fn() -> Result<Vec<PathBuf>, AnyError>,
|
|
||||||
G: Fn(Vec<PathBuf>) -> FileWatcherFuture<Result<(), AnyError>>,
|
|
||||||
{
|
|
||||||
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;
|
|
||||||
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"),
|
|
||||||
);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
pub enum ModuleResolutionResult<T> {
|
|
||||||
Success {
|
|
||||||
paths_to_watch: Vec<PathBuf>,
|
paths_to_watch: Vec<PathBuf>,
|
||||||
module_info: T,
|
result: Result<T, AnyError>,
|
||||||
},
|
|
||||||
Fail {
|
|
||||||
source_path: PathBuf,
|
|
||||||
error: AnyError,
|
|
||||||
},
|
},
|
||||||
|
Ignore,
|
||||||
}
|
}
|
||||||
|
|
||||||
/// This function adds watcher functionality to subcommands like `run` or `bundle`.
|
async fn next_restart<R, T, F>(
|
||||||
/// The difference from [`watch_func`] is that this does depend on [`ModuleGraph`].
|
resolver: &mut R,
|
||||||
|
debounce: &mut Pin<&mut Debounce>,
|
||||||
|
) -> (Vec<PathBuf>, Result<T, AnyError>)
|
||||||
|
where
|
||||||
|
R: FnMut(Option<Vec<PathBuf>>) -> F,
|
||||||
|
F: Future<Output = ResolutionResult<T>>,
|
||||||
|
{
|
||||||
|
loop {
|
||||||
|
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"),
|
||||||
|
);
|
||||||
|
return (paths_to_watch, result);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/// 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
|
/// - `resolver` is used for resolving file paths to be watched at every restarting
|
||||||
/// of the watcher and building [`ModuleGraph`] or [`ModuleSpecifier`] which will then be passed
|
/// of the watcher, and can also return a value to be passed to `operation`.
|
||||||
/// 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
|
/// - `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
|
/// 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.
|
/// - `job_name` is just used for printing watcher status to terminal.
|
||||||
///
|
pub async fn watch_func<R, O, T, F1, F2>(
|
||||||
/// Note that the watcher will try to continue watching files using the previously resolved
|
mut resolver: R,
|
||||||
/// data if `module_resolver` fails at some point, which means the watcher won't work at all
|
mut operation: O,
|
||||||
/// 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,
|
|
||||||
job_name: &str,
|
job_name: &str,
|
||||||
) -> Result<(), AnyError>
|
) -> Result<(), AnyError>
|
||||||
where
|
where
|
||||||
F: Fn() -> FileWatcherFuture<ModuleResolutionResult<T>>,
|
R: FnMut(Option<Vec<PathBuf>>) -> F1,
|
||||||
G: Fn(T) -> FileWatcherFuture<Result<(), AnyError>>,
|
O: FnMut(T) -> F2,
|
||||||
T: Clone,
|
F1: Future<Output = ResolutionResult<T>>,
|
||||||
|
F2: Future<Output = Result<(), AnyError>>,
|
||||||
{
|
{
|
||||||
let debounce = Debounce::new();
|
let debounce = Debounce::new();
|
||||||
pin!(debounce);
|
pin!(debounce);
|
||||||
|
|
||||||
// Store previous data. If module resolution fails at some point, the watcher will try to
|
// Store previous data. If module resolution fails at some point, the watcher will try to
|
||||||
// continue watching files using these data.
|
// continue watching files using these data.
|
||||||
let mut paths = Vec::new();
|
let mut paths_to_watch;
|
||||||
let mut module = None;
|
let mut resolution_result;
|
||||||
|
|
||||||
loop {
|
match resolver(None).await {
|
||||||
match module_resolver().await {
|
ResolutionResult::Ignore => {
|
||||||
ModuleResolutionResult::Success {
|
// The only situation where it makes sense to ignore the initial 'change'
|
||||||
paths_to_watch,
|
// is if the command isn't supposed to do anything until something changes,
|
||||||
module_info,
|
// 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.
|
||||||
paths = paths_to_watch;
|
//
|
||||||
module = Some(module_info);
|
// In that case, this is probably the correct output.
|
||||||
}
|
|
||||||
ModuleResolutionResult::Fail { source_path, error } => {
|
|
||||||
if paths.is_empty() {
|
|
||||||
paths = vec![source_path];
|
|
||||||
}
|
|
||||||
|
|
||||||
if module.is_none() {
|
|
||||||
eprintln!("{}: {}", colors::red_bold("error"), error);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
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!(
|
info!(
|
||||||
"{} File change detected! Restarting!",
|
"{} Waiting for file changes...",
|
||||||
colors::intense_blue("Watcher"),
|
colors::intense_blue("Watcher"),
|
||||||
);
|
);
|
||||||
},
|
|
||||||
_ = func => {},
|
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 {
|
||||||
|
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!(
|
info!(
|
||||||
"{} {} finished! Restarting on file change...",
|
"{} {} finished. Restarting on file change...",
|
||||||
colors::intense_blue("Watcher"),
|
colors::intense_blue("Watcher"),
|
||||||
job_name,
|
job_name,
|
||||||
);
|
);
|
||||||
debounce.next().await;
|
|
||||||
info!(
|
|
||||||
"{} File change detected! Restarting!",
|
|
||||||
colors::intense_blue("Watcher"),
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
} else {
|
Err(error) => {
|
||||||
|
eprintln!("{}: {}", colors::red_bold("error"), error);
|
||||||
info!(
|
info!(
|
||||||
"{} {} failed! Restarting on file change...",
|
"{} {} failed. Restarting on file change...",
|
||||||
colors::intense_blue("Watcher"),
|
colors::intense_blue("Watcher"),
|
||||||
job_name,
|
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);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
fn new_watcher(
|
fn new_watcher(
|
||||||
paths: &[PathBuf],
|
paths: &[PathBuf],
|
||||||
debounce: &Debounce,
|
debounce: &Debounce,
|
||||||
) -> Result<RecommendedWatcher, AnyError> {
|
) -> Result<RecommendedWatcher, AnyError> {
|
||||||
let event_detected = Arc::clone(&debounce.event_detected);
|
let changed_paths = Arc::clone(&debounce.changed_paths);
|
||||||
|
|
||||||
let mut watcher: RecommendedWatcher =
|
let mut watcher: RecommendedWatcher =
|
||||||
Watcher::new_immediate(move |res: Result<NotifyEvent, NotifyError>| {
|
Watcher::new_immediate(move |res: Result<NotifyEvent, NotifyError>| {
|
||||||
|
@ -257,16 +228,22 @@ fn new_watcher(
|
||||||
event.kind,
|
event.kind,
|
||||||
EventKind::Create(_) | EventKind::Modify(_) | EventKind::Remove(_)
|
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();
|
watcher.configure(Config::PreciseEvents(true)).unwrap();
|
||||||
|
|
||||||
|
log::debug!("Watching paths: {:?}", paths);
|
||||||
for path in paths {
|
for path in paths {
|
||||||
// Ignore any error e.g. `PathNotFound`
|
// Ignore any error e.g. `PathNotFound`
|
||||||
let _ = watcher.watch(path, RecursiveMode::NonRecursive);
|
let _ = watcher.watch(path, RecursiveMode::Recursive);
|
||||||
}
|
}
|
||||||
|
|
||||||
Ok(watcher)
|
Ok(watcher)
|
||||||
|
|
|
@ -1031,6 +1031,11 @@ fn test_subcommand<'a, 'b>() -> App<'a, 'b> {
|
||||||
.takes_value(true)
|
.takes_value(true)
|
||||||
.multiple(true),
|
.multiple(true),
|
||||||
)
|
)
|
||||||
|
.arg(
|
||||||
|
watch_arg()
|
||||||
|
.conflicts_with("no-run")
|
||||||
|
.conflicts_with("coverage"),
|
||||||
|
)
|
||||||
.arg(script_arg().last(true))
|
.arg(script_arg().last(true))
|
||||||
.about("Run tests")
|
.about("Run tests")
|
||||||
.long_about(
|
.long_about(
|
||||||
|
@ -1667,6 +1672,8 @@ fn test_parse(flags: &mut Flags, matches: &clap::ArgMatches) {
|
||||||
let quiet = matches.is_present("quiet");
|
let quiet = matches.is_present("quiet");
|
||||||
let filter = matches.value_of("filter").map(String::from);
|
let filter = matches.value_of("filter").map(String::from);
|
||||||
|
|
||||||
|
flags.watch = matches.is_present("watch");
|
||||||
|
|
||||||
if matches.is_present("script_arg") {
|
if matches.is_present("script_arg") {
|
||||||
let script_arg: Vec<String> = matches
|
let script_arg: Vec<String> = matches
|
||||||
.values_of("script_arg")
|
.values_of("script_arg")
|
||||||
|
|
240
cli/main.rs
240
cli/main.rs
|
@ -38,11 +38,13 @@ mod unix_util;
|
||||||
mod version;
|
mod version;
|
||||||
|
|
||||||
use crate::file_fetcher::File;
|
use crate::file_fetcher::File;
|
||||||
use crate::file_watcher::ModuleResolutionResult;
|
use crate::file_watcher::ResolutionResult;
|
||||||
use crate::flags::DenoSubcommand;
|
use crate::flags::DenoSubcommand;
|
||||||
use crate::flags::Flags;
|
use crate::flags::Flags;
|
||||||
use crate::fmt_errors::PrettyJsError;
|
use crate::fmt_errors::PrettyJsError;
|
||||||
use crate::media_type::MediaType;
|
use crate::media_type::MediaType;
|
||||||
|
use crate::module_graph::GraphBuilder;
|
||||||
|
use crate::module_graph::Module;
|
||||||
use crate::module_loader::CliModuleLoader;
|
use crate::module_loader::CliModuleLoader;
|
||||||
use crate::program_state::ProgramState;
|
use crate::program_state::ProgramState;
|
||||||
use crate::source_maps::apply_source_map;
|
use crate::source_maps::apply_source_map;
|
||||||
|
@ -67,6 +69,7 @@ use log::debug;
|
||||||
use log::info;
|
use log::info;
|
||||||
use log::Level;
|
use log::Level;
|
||||||
use log::LevelFilter;
|
use log::LevelFilter;
|
||||||
|
use std::collections::HashSet;
|
||||||
use std::env;
|
use std::env;
|
||||||
use std::io::Read;
|
use std::io::Read;
|
||||||
use std::io::Write;
|
use std::io::Write;
|
||||||
|
@ -76,6 +79,7 @@ use std::pin::Pin;
|
||||||
use std::rc::Rc;
|
use std::rc::Rc;
|
||||||
use std::sync::Arc;
|
use std::sync::Arc;
|
||||||
use std::sync::Mutex;
|
use std::sync::Mutex;
|
||||||
|
use tools::test_runner;
|
||||||
|
|
||||||
fn create_web_worker_callback(
|
fn create_web_worker_callback(
|
||||||
program_state: Arc<ProgramState>,
|
program_state: Arc<ProgramState>,
|
||||||
|
@ -595,7 +599,7 @@ async fn bundle_command(
|
||||||
) -> Result<(), AnyError> {
|
) -> Result<(), AnyError> {
|
||||||
let debug = flags.log_level == Some(log::Level::Debug);
|
let debug = flags.log_level == Some(log::Level::Debug);
|
||||||
|
|
||||||
let module_resolver = || {
|
let resolver = |_| {
|
||||||
let flags = flags.clone();
|
let flags = flags.clone();
|
||||||
let source_file1 = source_file.clone();
|
let source_file1 = source_file.clone();
|
||||||
let source_file2 = source_file.clone();
|
let source_file2 = source_file.clone();
|
||||||
|
@ -605,12 +609,6 @@ async fn bundle_command(
|
||||||
debug!(">>>>> bundle START");
|
debug!(">>>>> bundle START");
|
||||||
let program_state = ProgramState::build(flags.clone()).await?;
|
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(
|
let module_graph = create_module_graph_and_maybe_check(
|
||||||
module_specifier,
|
module_specifier,
|
||||||
program_state.clone(),
|
program_state.clone(),
|
||||||
|
@ -632,22 +630,23 @@ async fn bundle_command(
|
||||||
Ok((paths_to_watch, module_graph))
|
Ok((paths_to_watch, module_graph))
|
||||||
}
|
}
|
||||||
.map(move |result| match result {
|
.map(move |result| match result {
|
||||||
Ok((paths_to_watch, module_graph)) => ModuleResolutionResult::Success {
|
Ok((paths_to_watch, module_graph)) => ResolutionResult::Restart {
|
||||||
paths_to_watch,
|
paths_to_watch,
|
||||||
module_info: module_graph,
|
result: Ok(module_graph),
|
||||||
},
|
},
|
||||||
Err(e) => ModuleResolutionResult::Fail {
|
Err(e) => ResolutionResult::Restart {
|
||||||
source_path: PathBuf::from(source_file2),
|
paths_to_watch: vec![PathBuf::from(source_file2)],
|
||||||
error: e,
|
result: Err(e),
|
||||||
},
|
},
|
||||||
})
|
})
|
||||||
.boxed_local()
|
|
||||||
};
|
};
|
||||||
|
|
||||||
let operation = |module_graph: module_graph::Graph| {
|
let operation = |module_graph: module_graph::Graph| {
|
||||||
let flags = flags.clone();
|
let flags = flags.clone();
|
||||||
let out_file = out_file.clone();
|
let out_file = out_file.clone();
|
||||||
async move {
|
async move {
|
||||||
|
info!("{} {}", colors::green("Bundle"), module_graph.info()?.root);
|
||||||
|
|
||||||
let output = bundle_module_graph(module_graph, flags, debug)?;
|
let output = bundle_module_graph(module_graph, flags, debug)?;
|
||||||
|
|
||||||
debug!(">>>>> bundle END");
|
debug!(">>>>> bundle END");
|
||||||
|
@ -668,20 +667,16 @@ async fn bundle_command(
|
||||||
|
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
.boxed_local()
|
|
||||||
};
|
};
|
||||||
|
|
||||||
if flags.watch {
|
if flags.watch {
|
||||||
file_watcher::watch_func_with_module_resolution(
|
file_watcher::watch_func(resolver, operation, "Bundle").await?;
|
||||||
module_resolver,
|
|
||||||
operation,
|
|
||||||
"Bundle",
|
|
||||||
)
|
|
||||||
.await?;
|
|
||||||
} else {
|
} else {
|
||||||
let module_graph = match module_resolver().await {
|
let module_graph =
|
||||||
ModuleResolutionResult::Fail { error, .. } => return Err(error),
|
if let ResolutionResult::Restart { result, .. } = resolver(None).await {
|
||||||
ModuleResolutionResult::Success { module_info, .. } => module_info,
|
result?
|
||||||
|
} else {
|
||||||
|
unreachable!();
|
||||||
};
|
};
|
||||||
operation(module_graph).await?;
|
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> {
|
async fn run_with_watch(flags: Flags, script: String) -> Result<(), AnyError> {
|
||||||
let module_resolver = || {
|
let resolver = |_| {
|
||||||
let script1 = script.clone();
|
let script1 = script.clone();
|
||||||
let script2 = script.clone();
|
let script2 = script.clone();
|
||||||
let flags = flags.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))
|
Ok((paths_to_watch, main_module))
|
||||||
}
|
}
|
||||||
.map(move |result| match result {
|
.map(move |result| match result {
|
||||||
Ok((paths_to_watch, module_info)) => ModuleResolutionResult::Success {
|
Ok((paths_to_watch, module_info)) => ResolutionResult::Restart {
|
||||||
paths_to_watch,
|
paths_to_watch,
|
||||||
module_info,
|
result: Ok(module_info),
|
||||||
},
|
},
|
||||||
Err(e) => ModuleResolutionResult::Fail {
|
Err(e) => ResolutionResult::Restart {
|
||||||
source_path: PathBuf::from(script2),
|
paths_to_watch: vec![PathBuf::from(script2)],
|
||||||
error: e,
|
result: Err(e),
|
||||||
},
|
},
|
||||||
})
|
})
|
||||||
.boxed_local()
|
|
||||||
};
|
};
|
||||||
|
|
||||||
let operation = |main_module: ModuleSpecifier| {
|
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'))")?;
|
worker.execute("window.dispatchEvent(new Event('unload'))")?;
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
.boxed_local()
|
|
||||||
};
|
};
|
||||||
|
|
||||||
file_watcher::watch_func_with_module_resolution(
|
file_watcher::watch_func(resolver, operation, "Process").await
|
||||||
module_resolver,
|
|
||||||
operation,
|
|
||||||
"Process",
|
|
||||||
)
|
|
||||||
.await
|
|
||||||
}
|
}
|
||||||
|
|
||||||
async fn run_command(flags: Flags, script: String) -> Result<(), AnyError> {
|
async fn run_command(flags: Flags, script: String) -> Result<(), AnyError> {
|
||||||
|
@ -917,9 +905,174 @@ async fn test_command(
|
||||||
env::set_var("DENO_UNSTABLE_COVERAGE_DIR", coverage_dir);
|
env::set_var("DENO_UNSTABLE_COVERAGE_DIR", coverage_dir);
|
||||||
}
|
}
|
||||||
|
|
||||||
tools::test_runner::run_tests(
|
let program_state = ProgramState::build(flags.clone()).await?;
|
||||||
flags,
|
|
||||||
include,
|
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,
|
no_run,
|
||||||
fail_fast,
|
fail_fast,
|
||||||
quiet,
|
quiet,
|
||||||
|
@ -929,6 +1082,11 @@ async fn test_command(
|
||||||
)
|
)
|
||||||
.await?;
|
.await?;
|
||||||
|
|
||||||
|
if failed {
|
||||||
|
std::process::exit(1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -1248,6 +1248,18 @@ impl Graph {
|
||||||
self.modules.get_mut(s)
|
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
|
/// Consume graph and return list of all module specifiers contained in the
|
||||||
/// graph.
|
/// graph.
|
||||||
pub fn get_modules(&self) -> Vec<ModuleSpecifier> {
|
pub fn get_modules(&self) -> Vec<ModuleSpecifier> {
|
||||||
|
|
|
@ -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]
|
#[test]
|
||||||
fn fmt_watch_test() {
|
fn fmt_watch_test() {
|
||||||
let t = TempDir::new().expect("tempdir fail");
|
let t = TempDir::new().expect("tempdir fail");
|
||||||
|
@ -685,6 +700,7 @@ mod integration {
|
||||||
.arg(&bundle)
|
.arg(&bundle)
|
||||||
.arg("--watch")
|
.arg("--watch")
|
||||||
.arg("--unstable")
|
.arg("--unstable")
|
||||||
|
.env("NO_COLOR", "1")
|
||||||
.stdout(std::process::Stdio::piped())
|
.stdout(std::process::Stdio::piped())
|
||||||
.stderr(std::process::Stdio::piped())
|
.stderr(std::process::Stdio::piped())
|
||||||
.spawn()
|
.spawn()
|
||||||
|
@ -722,11 +738,8 @@ mod integration {
|
||||||
.next()
|
.next()
|
||||||
.unwrap()
|
.unwrap()
|
||||||
.contains("File change detected!"));
|
.contains("File change detected!"));
|
||||||
assert!(stderr_lines.next().unwrap().contains("file_to_watch.js"));
|
assert!(stderr_lines.next().unwrap().contains("error: "));
|
||||||
assert!(stderr_lines.next().unwrap().contains("mod6.bundle.js"));
|
wait_for_process_failed("Bundle", &mut stderr_lines);
|
||||||
let file = PathBuf::from(&bundle);
|
|
||||||
assert!(file.is_file());
|
|
||||||
wait_for_process_finished("Bundle", &mut stderr_lines);
|
|
||||||
|
|
||||||
// the watcher process is still alive
|
// the watcher process is still alive
|
||||||
assert!(deno.try_wait().unwrap().is_none());
|
assert!(deno.try_wait().unwrap().is_none());
|
||||||
|
@ -762,9 +775,8 @@ mod integration {
|
||||||
std::io::BufReader::new(stderr).lines().map(|r| r.unwrap());
|
std::io::BufReader::new(stderr).lines().map(|r| r.unwrap());
|
||||||
|
|
||||||
std::thread::sleep(std::time::Duration::from_secs(1));
|
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("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
|
// the target file hasn't been created yet
|
||||||
assert!(!target_file.is_file());
|
assert!(!target_file.is_file());
|
||||||
|
|
||||||
|
@ -858,7 +870,7 @@ mod integration {
|
||||||
std::thread::sleep(std::time::Duration::from_secs(1));
|
std::thread::sleep(std::time::Duration::from_secs(1));
|
||||||
assert!(stderr_lines.next().unwrap().contains("Restarting"));
|
assert!(stderr_lines.next().unwrap().contains("Restarting"));
|
||||||
assert!(stderr_lines.next().unwrap().contains("error:"));
|
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
|
// Then restore the file
|
||||||
std::fs::write(
|
std::fs::write(
|
||||||
|
@ -877,7 +889,7 @@ mod integration {
|
||||||
std::thread::sleep(std::time::Duration::from_secs(1));
|
std::thread::sleep(std::time::Duration::from_secs(1));
|
||||||
assert!(stderr_lines.next().unwrap().contains("Restarting"));
|
assert!(stderr_lines.next().unwrap().contains("Restarting"));
|
||||||
assert!(stderr_lines.next().unwrap().contains("error:"));
|
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
|
// Modify the imported file and make sure that restarting occurs
|
||||||
std::fs::write(&another_file, "export const foo = 'modified!';")
|
std::fs::write(&another_file, "export const foo = 'modified!';")
|
||||||
|
@ -923,7 +935,7 @@ mod integration {
|
||||||
|
|
||||||
std::thread::sleep(std::time::Duration::from_secs(1));
|
std::thread::sleep(std::time::Duration::from_secs(1));
|
||||||
assert!(stderr_lines.next().unwrap().contains("error:"));
|
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
|
// Make sure the watcher actually restarts and works fine with the proper syntax
|
||||||
std::fs::write(&file_to_watch, "console.log(42);")
|
std::fs::write(&file_to_watch, "console.log(42);")
|
||||||
|
@ -999,6 +1011,174 @@ mod integration {
|
||||||
drop(import_map_path);
|
drop(import_map_path);
|
||||||
temp_directory.close().unwrap();
|
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]
|
#[test]
|
||||||
|
|
|
@ -10,12 +10,12 @@
|
||||||
use crate::colors;
|
use crate::colors;
|
||||||
use crate::diff::diff;
|
use crate::diff::diff;
|
||||||
use crate::file_watcher;
|
use crate::file_watcher;
|
||||||
|
use crate::file_watcher::ResolutionResult;
|
||||||
use crate::fs_util::{collect_files, get_extension, is_supported_ext_fmt};
|
use crate::fs_util::{collect_files, get_extension, is_supported_ext_fmt};
|
||||||
use crate::text_encoding;
|
use crate::text_encoding;
|
||||||
use deno_core::error::generic_error;
|
use deno_core::error::generic_error;
|
||||||
use deno_core::error::AnyError;
|
use deno_core::error::AnyError;
|
||||||
use deno_core::futures;
|
use deno_core::futures;
|
||||||
use deno_core::futures::FutureExt;
|
|
||||||
use log::debug;
|
use log::debug;
|
||||||
use log::info;
|
use log::info;
|
||||||
use std::fs;
|
use std::fs;
|
||||||
|
@ -37,15 +37,32 @@ pub async fn format(
|
||||||
check: bool,
|
check: bool,
|
||||||
watch: bool,
|
watch: bool,
|
||||||
) -> Result<(), AnyError> {
|
) -> Result<(), AnyError> {
|
||||||
let target_file_resolver = || {
|
let resolver = |changed: Option<Vec<PathBuf>>| {
|
||||||
// collect the files that are to be formatted
|
let files_changed = changed.is_some();
|
||||||
collect_files(&args, &ignore, is_supported_ext_fmt).and_then(|files| {
|
let result =
|
||||||
if files.is_empty() {
|
collect_files(&args, &ignore, is_supported_ext_fmt).map(|files| {
|
||||||
Err(generic_error("No target files found."))
|
if let Some(paths) = changed {
|
||||||
|
files
|
||||||
|
.into_iter()
|
||||||
|
.filter(|path| paths.contains(path))
|
||||||
|
.collect::<Vec<_>>()
|
||||||
} else {
|
} else {
|
||||||
Ok(files)
|
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 {
|
||||||
|
ResolutionResult::Restart {
|
||||||
|
paths_to_watch,
|
||||||
|
result,
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
})
|
|
||||||
};
|
};
|
||||||
let operation = |paths: Vec<PathBuf>| {
|
let operation = |paths: Vec<PathBuf>| {
|
||||||
let config = get_typescript_config();
|
let config = get_typescript_config();
|
||||||
|
@ -57,13 +74,18 @@ pub async fn format(
|
||||||
}
|
}
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
.boxed_local()
|
|
||||||
};
|
};
|
||||||
|
|
||||||
if watch {
|
if watch {
|
||||||
file_watcher::watch_func(target_file_resolver, operation, "Fmt").await?;
|
file_watcher::watch_func(resolver, operation, "Fmt").await?;
|
||||||
} else {
|
} 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(())
|
Ok(())
|
||||||
|
|
|
@ -3,7 +3,6 @@
|
||||||
use crate::colors;
|
use crate::colors;
|
||||||
use crate::create_main_worker;
|
use crate::create_main_worker;
|
||||||
use crate::file_fetcher::File;
|
use crate::file_fetcher::File;
|
||||||
use crate::flags::Flags;
|
|
||||||
use crate::fs_util;
|
use crate::fs_util;
|
||||||
use crate::media_type::MediaType;
|
use crate::media_type::MediaType;
|
||||||
use crate::module_graph;
|
use crate::module_graph;
|
||||||
|
@ -304,37 +303,30 @@ pub async fn run_test_file(
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Runs tests.
|
||||||
|
///
|
||||||
|
/// Returns a boolean indicating whether the tests failed.
|
||||||
#[allow(clippy::too_many_arguments)]
|
#[allow(clippy::too_many_arguments)]
|
||||||
pub async fn run_tests(
|
pub async fn run_tests(
|
||||||
flags: Flags,
|
program_state: Arc<ProgramState>,
|
||||||
include: Option<Vec<String>>,
|
permissions: Permissions,
|
||||||
|
lib: module_graph::TypeLib,
|
||||||
|
test_modules: Vec<ModuleSpecifier>,
|
||||||
no_run: bool,
|
no_run: bool,
|
||||||
fail_fast: bool,
|
fail_fast: bool,
|
||||||
quiet: bool,
|
quiet: bool,
|
||||||
allow_none: bool,
|
allow_none: bool,
|
||||||
filter: Option<String>,
|
filter: Option<String>,
|
||||||
concurrent_jobs: usize,
|
concurrent_jobs: usize,
|
||||||
) -> Result<(), AnyError> {
|
) -> Result<bool, 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)?;
|
|
||||||
|
|
||||||
if test_modules.is_empty() {
|
if test_modules.is_empty() {
|
||||||
println!("No matching test modules found");
|
println!("No matching test modules found");
|
||||||
if !allow_none {
|
if !allow_none {
|
||||||
std::process::exit(1);
|
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
|
program_state
|
||||||
.prepare_module_graph(
|
.prepare_module_graph(
|
||||||
test_modules.clone(),
|
test_modules.clone(),
|
||||||
|
@ -345,7 +337,7 @@ pub async fn run_tests(
|
||||||
.await?;
|
.await?;
|
||||||
|
|
||||||
if no_run {
|
if no_run {
|
||||||
return Ok(());
|
return Ok(false);
|
||||||
}
|
}
|
||||||
|
|
||||||
// Because scripts, and therefore worker.execute cannot detect unresolved promises at the moment
|
// 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() {
|
if let Some(e) = join_errors.next() {
|
||||||
Err(e)
|
Err(e)
|
||||||
} else {
|
} else {
|
||||||
if result.unwrap_or(false) {
|
Ok(result.unwrap_or(false))
|
||||||
std::process::exit(1);
|
|
||||||
}
|
|
||||||
|
|
||||||
Ok(())
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in a new issue