summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorRyan Dahl <ry@tinyclouds.org>2019-05-20 12:06:57 -0400
committerRyan Dahl <ry@tinyclouds.org>2019-05-29 07:53:39 -0400
commit856c44213b7faf507d4b481cfc170b2fd08f971a (patch)
treeb2971883b0aeb43437a9be0076b4ffacde55d5b8
parent64d2b7bc90cf6fdba661d6d3fe243fe332c076ee (diff)
TS compiler refactor
* Compiler no longer has its own Tokio runtime. Compiler handles one message and then exits. * Uses the simpler ts.CompilerHost interface instead of ts.LanguageServiceHost. * avoids recompiling the same module by introducing a hacky but simple `hashset<string>` that stores the module names that have been already compiled. * Removes the CompilerConfig op. * Removes a lot of the mocking stuff in compiler.ts like `this._ts`. It is not useful as we don't even have tests. * Turns off checkJs because it causes fmt_test to die with OOM.
-rw-r--r--cli/compiler.rs283
-rw-r--r--cli/msg.fbs32
-rw-r--r--cli/ops.rs106
-rw-r--r--cli/state.rs18
-rw-r--r--cli/worker.rs88
-rw-r--r--core/shared_queue.js2
-rw-r--r--js/compiler.ts758
-rw-r--r--js/os.ts45
-rw-r--r--rollup.config.js4
-rw-r--r--tests/config.ts.out3
-rw-r--r--tests/error_004_missing_module.ts.out4
-rw-r--r--tests/error_005_missing_dynamic_import.ts.out5
-rw-r--r--tests/error_006_import_ext_failure.ts.out5
13 files changed, 476 insertions, 877 deletions
diff --git a/cli/compiler.rs b/cli/compiler.rs
index d4913a4e2..e1bb56130 100644
--- a/cli/compiler.rs
+++ b/cli/compiler.rs
@@ -1,10 +1,6 @@
// Copyright 2018-2019 the Deno authors. All rights reserved. MIT license.
-use crate::js_errors;
-use crate::js_errors::JSErrorColor;
use crate::msg;
-use crate::ops::op_selector_compiler;
use crate::resources;
-use crate::resources::ResourceId;
use crate::startup_data;
use crate::state::*;
use crate::tokio_util;
@@ -12,31 +8,10 @@ use crate::worker::Worker;
use deno::js_check;
use deno::Buf;
use deno::JSError;
-use futures::future::*;
-use futures::sync::oneshot;
use futures::Future;
use futures::Stream;
-use serde_json;
-use std::collections::HashMap;
use std::str;
-use std::sync::atomic::AtomicUsize;
use std::sync::atomic::Ordering;
-use std::sync::Mutex;
-use tokio::runtime::Runtime;
-
-type CmdId = u32;
-type ResponseSenderTable = HashMap<CmdId, oneshot::Sender<Buf>>;
-
-lazy_static! {
- static ref C_NEXT_CMD_ID: AtomicUsize = AtomicUsize::new(1);
- // Map of response senders
- static ref C_RES_SENDER_TABLE: Mutex<ResponseSenderTable> = Mutex::new(ResponseSenderTable::new());
- // Shared worker resources so we can spawn
- static ref C_RID: Mutex<Option<ResourceId>> = Mutex::new(None);
- // tokio runtime specifically for spawning logic that is dependent on
- // completetion of the compiler worker future
- static ref C_RUNTIME: Mutex<Runtime> = Mutex::new(tokio_util::create_threadpool_runtime());
-}
// This corresponds to JS ModuleMetaData.
// TODO Rename one or the other so they correspond.
@@ -72,91 +47,22 @@ impl ModuleMetaData {
}
}
-fn new_cmd_id() -> CmdId {
- let next_rid = C_NEXT_CMD_ID.fetch_add(1, Ordering::SeqCst);
- next_rid as CmdId
-}
-
-fn parse_cmd_id(res_json: &str) -> CmdId {
- match serde_json::from_str::<serde_json::Value>(res_json) {
- Ok(serde_json::Value::Object(map)) => match map["cmdId"].as_u64() {
- Some(cmd_id) => cmd_id as CmdId,
- _ => panic!("Error decoding compiler response: expected cmdId"),
- },
- _ => panic!("Error decoding compiler response"),
- }
-}
-
-fn lazy_start(parent_state: ThreadSafeState) -> ResourceId {
- let mut cell = C_RID.lock().unwrap();
- cell
- .get_or_insert_with(|| {
- let child_state = ThreadSafeState::new(
- parent_state.flags.clone(),
- parent_state.argv.clone(),
- op_selector_compiler,
- parent_state.progress.clone(),
- );
- let rid = child_state.resource.rid;
- let resource = child_state.resource.clone();
-
- let mut worker = Worker::new(
- "TS".to_string(),
- startup_data::compiler_isolate_init(),
- child_state,
- );
-
- js_check(worker.execute("denoMain()"));
- js_check(worker.execute("workerMain()"));
- js_check(worker.execute("compilerMain()"));
+type CompilerConfig = Option<(String, Vec<u8>)>;
- let mut runtime = C_RUNTIME.lock().unwrap();
- runtime.spawn(lazy(move || {
- worker.then(move |result| -> Result<(), ()> {
- // Close resource so the future created by
- // handle_worker_message_stream exits
- resource.close();
- debug!("Compiler worker exited!");
- if let Err(e) = result {
- eprintln!("{}", JSErrorColor(&e).to_string());
- }
- std::process::exit(1);
- })
- }));
- runtime.spawn(lazy(move || {
- debug!("Start worker stream handler!");
- let worker_stream = resources::get_message_stream_from_worker(rid);
- worker_stream
- .for_each(|msg: Buf| {
- // All worker responses are handled here first before being sent via
- // their respective sender. This system can be compared to the
- // promise system used on the js side. This provides a way to
- // resolve many futures via the same channel.
- let res_json = std::str::from_utf8(&msg).unwrap();
- debug!("Got message from worker: {}", res_json);
- // Get the intended receiver's cmd_id from the message.
- let cmd_id = parse_cmd_id(res_json);
- let mut table = C_RES_SENDER_TABLE.lock().unwrap();
- debug!("Cmd id for get message handler: {}", cmd_id);
- // Get the corresponding response sender from the table and
- // send a response.
- let response_sender = table.remove(&(cmd_id as CmdId)).unwrap();
- response_sender.send(msg).unwrap();
- Ok(())
- }).map_err(|_| ())
- }));
- rid
- }).to_owned()
-}
-
-fn req(specifier: &str, referrer: &str, cmd_id: u32) -> Buf {
- json!({
- "specifier": specifier,
- "referrer": referrer,
- "cmdId": cmd_id,
- }).to_string()
- .into_boxed_str()
- .into_boxed_bytes()
+/// Creates the JSON message send to compiler.ts's onmessage.
+fn req(root_names: Vec<String>, compiler_config: CompilerConfig) -> Buf {
+ let j = if let Some((config_path, config_data)) = compiler_config {
+ json!({
+ "rootNames": root_names,
+ "configPath": config_path,
+ "config": str::from_utf8(&config_data).unwrap(),
+ })
+ } else {
+ json!({
+ "rootNames": root_names,
+ })
+ };
+ j.to_string().into_boxed_str().into_boxed_bytes()
}
/// Returns an optional tuple which represents the state of the compiler
@@ -165,7 +71,7 @@ fn req(specifier: &str, referrer: &str, cmd_id: u32) -> Buf {
pub fn get_compiler_config(
parent_state: &ThreadSafeState,
_compiler_type: &str,
-) -> Option<(String, Vec<u8>)> {
+) -> CompilerConfig {
// The compiler type is being passed to make it easier to implement custom
// compilers in the future.
match (&parent_state.config_path, &parent_state.config) {
@@ -177,7 +83,7 @@ pub fn get_compiler_config(
}
pub fn compile_async(
- parent_state: ThreadSafeState,
+ state: ThreadSafeState,
specifier: &str,
referrer: &str,
module_meta_data: &ModuleMetaData,
@@ -186,100 +92,86 @@ pub fn compile_async(
"Running rust part of compile_sync. specifier: {}, referrer: {}",
&specifier, &referrer
);
- let cmd_id = new_cmd_id();
- let req_msg = req(&specifier, &referrer, cmd_id);
+ let root_names = vec![module_meta_data.module_name.clone()];
+ let compiler_config = get_compiler_config(&state, "typescript");
+ let req_msg = req(root_names, compiler_config);
+
let module_meta_data_ = module_meta_data.clone();
- let compiler_rid = lazy_start(parent_state.clone());
+ // Count how many times we start the compiler worker.
+ state.metrics.compiler_starts.fetch_add(1, Ordering::SeqCst);
+
+ let mut worker = Worker::new(
+ "TS".to_string(),
+ startup_data::compiler_isolate_init(),
+ // TODO(ry) Maybe we should use a separate state for the compiler.
+ // as was done previously.
+ state.clone(),
+ );
+ js_check(worker.execute("denoMain()"));
+ js_check(worker.execute("workerMain()"));
+ js_check(worker.execute("compilerMain()"));
- let compiling_job = parent_state
+ let compiling_job = state
.progress
.add(format!("Compiling {}", module_meta_data_.module_name));
- let (local_sender, local_receiver) =
- oneshot::channel::<Result<ModuleMetaData, Option<JSError>>>();
-
- let (response_sender, response_receiver) = oneshot::channel::<Buf>();
-
- // Scoping to auto dispose of locks when done using them
- {
- let mut table = C_RES_SENDER_TABLE.lock().unwrap();
- debug!("Cmd id for response sender insert: {}", cmd_id);
- // Place our response sender in the table so we can find it later.
- table.insert(cmd_id, response_sender);
-
- let mut runtime = C_RUNTIME.lock().unwrap();
- runtime.spawn(lazy(move || {
- resources::post_message_to_worker(compiler_rid, req_msg)
- .then(move |_| {
- debug!("Sent message to worker");
- response_receiver.map_err(|_| None)
- }).and_then(move |res_msg| {
- debug!("Received message from worker");
- let res_json = std::str::from_utf8(res_msg.as_ref()).unwrap();
- let res = serde_json::from_str::<serde_json::Value>(res_json)
- .expect("Error decoding compiler response");
- let res_data = res["data"].as_object().expect(
- "Error decoding compiler response: expected object field 'data'",
- );
+ let resource = worker.state.resource.clone();
+ let compiler_rid = resource.rid;
+ let first_msg_fut = resources::post_message_to_worker(compiler_rid, req_msg)
+ .then(move |_| worker)
+ .then(move |result| {
+ if let Err(err) = result {
+ // TODO(ry) Need to forward the error instead of exiting.
+ eprintln!("{}", err.to_string());
+ std::process::exit(1);
+ }
+ debug!("Sent message to worker");
+ let stream_future =
+ resources::get_message_stream_from_worker(compiler_rid).into_future();
+ stream_future.map(|(f, _rest)| f).map_err(|(f, _rest)| f)
+ });
+
+ first_msg_fut
+ .map_err(|_| panic!("not handled"))
+ .and_then(move |maybe_msg: Option<Buf>| {
+ let _res_msg = maybe_msg.unwrap();
+
+ debug!("Received message from worker");
+
+ // TODO res is EmitResult, use serde_derive to parse it. Errors from the
+ // worker or Diagnostics should be somehow forwarded to the caller!
+ // Currently they are handled inside compiler.ts with os.exit(1) and above
+ // with std::process::exit(1). This bad.
+
+ let r = state.dir.fetch_module_meta_data(
+ &module_meta_data_.module_name,
+ ".",
+ true,
+ true,
+ );
+ let module_meta_data_after_compile = r.unwrap();
- // Explicit drop to keep reference alive until future completes.
- drop(compiling_job);
+ // Explicit drop to keep reference alive until future completes.
+ drop(compiling_job);
- match res["success"].as_bool() {
- Some(true) => Ok(ModuleMetaData {
- maybe_output_code: res_data["outputCode"]
- .as_str()
- .map(|s| s.as_bytes().to_owned()),
- maybe_source_map: res_data["sourceMap"]
- .as_str()
- .map(|s| s.as_bytes().to_owned()),
- ..module_meta_data_
- }),
- Some(false) => {
- let js_error = JSError::from_json_value(
- serde_json::Value::Object(res_data.clone()),
- ).expect(
- "Error decoding compiler response: failed to parse error",
- );
- Err(Some(js_errors::apply_source_map(
- &js_error,
- &parent_state.dir,
- )))
- }
- _ => panic!(
- "Error decoding compiler response: expected bool field 'success'"
- ),
- }
- }).then(move |result| {
- local_sender.send(result).expect("Oneshot send() failed");
- Ok(())
- })
- }));
- }
-
- local_receiver
- .map_err(|e| {
- panic!(
- "Local channel canceled before compile request could be completed: {}",
- e
- )
- }).and_then(move |result| match result {
- Ok(v) => futures::future::result(Ok(v)),
- Err(Some(err)) => futures::future::result(Err(err)),
- Err(None) => panic!("Failed to communicate with the compiler worker."),
+ Ok(module_meta_data_after_compile)
+ }).then(move |r| {
+ // TODO(ry) do this in worker's destructor.
+ // resource.close();
+ r
})
}
pub fn compile_sync(
- parent_state: ThreadSafeState,
+ state: ThreadSafeState,
specifier: &str,
referrer: &str,
module_meta_data: &ModuleMetaData,
) -> Result<ModuleMetaData, JSError> {
tokio_util::block_on(compile_async(
- parent_state,
+ state,
specifier,
referrer,
module_meta_data,
@@ -298,9 +190,13 @@ mod tests {
let specifier = "./tests/002_hello.ts";
let referrer = cwd_string + "/";
+ use crate::worker;
+ let module_name = worker::root_specifier_to_url(specifier)
+ .unwrap()
+ .to_string();
let mut out = ModuleMetaData {
- module_name: "xxx".to_owned(),
+ module_name,
module_redirect_source_name: None,
filename: "/tests/002_hello.ts".to_owned(),
media_type: msg::MediaType::TypeScript,
@@ -323,17 +219,6 @@ mod tests {
}
#[test]
- fn test_parse_cmd_id() {
- let cmd_id = new_cmd_id();
-
- let msg = req("Hello", "World", cmd_id);
-
- let res_json = std::str::from_utf8(&msg).unwrap();
-
- assert_eq!(parse_cmd_id(res_json), cmd_id);
- }
-
- #[test]
fn test_get_compiler_config_no_flag() {
let compiler_type = "typescript";
let state = ThreadSafeState::mock();
diff --git a/cli/msg.fbs b/cli/msg.fbs
index 08e58351c..7f2db381f 100644
--- a/cli/msg.fbs
+++ b/cli/msg.fbs
@@ -1,12 +1,13 @@
union Any {
Accept,
+ Cache,
Chdir,
Chmod,
Chown,
Close,
- CompilerConfig,
- CompilerConfigRes,
CopyFile,
+ CreateWorker,
+ CreateWorkerRes,
Cwd,
CwdRes,
Dial,
@@ -23,6 +24,10 @@ union Any {
GlobalTimer,
GlobalTimerRes,
GlobalTimerStop,
+ HostGetMessage,
+ HostGetMessageRes,
+ HostGetWorkerClosed,
+ HostPostMessage,
IsTTY,
IsTTYRes,
Kill,
@@ -70,12 +75,6 @@ union Any {
Symlink,
Truncate,
Utime,
- CreateWorker,
- CreateWorkerRes,
- HostGetWorkerClosed,
- HostGetMessage,
- HostGetMessageRes,
- HostPostMessage,
WorkerGetMessage,
WorkerGetMessageRes,
WorkerPostMessage,
@@ -180,15 +179,6 @@ table StartRes {
xeval_delim: string;
}
-table CompilerConfig {
- compiler_type: string;
-}
-
-table CompilerConfigRes {
- path: string;
- data: [ubyte];
-}
-
table FormatError {
error: string;
}
@@ -246,7 +236,7 @@ table FetchModuleMetaData {
table FetchModuleMetaDataRes {
// If it's a non-http module, moduleName and filename will be the same.
- // For http modules, moduleName is its resolved http URL, and filename
+ // For http modules, module_name is its resolved http URL, and filename
// is the location of the locally downloaded source code.
module_name: string;
filename: string;
@@ -254,6 +244,12 @@ table FetchModuleMetaDataRes {
data: [ubyte];
}
+table Cache {
+ extension: string;
+ module_id: string;
+ contents: string;
+}
+
table Chdir {
directory: string;
}
diff --git a/cli/ops.rs b/cli/ops.rs
index 1f208bbe2..e41c9caa2 100644
--- a/cli/ops.rs
+++ b/cli/ops.rs
@@ -1,7 +1,6 @@
// Copyright 2018-2019 the Deno authors. All rights reserved. MIT license.
use atty;
use crate::ansi;
-use crate::compiler::get_compiler_config;
use crate::deno_dir::resolve_path;
use crate::dispatch_minimal::dispatch_minimal;
use crate::dispatch_minimal::parse_min_record;
@@ -177,37 +176,30 @@ pub fn dispatch_all_legacy(
}
}
-pub fn op_selector_compiler(inner_type: msg::Any) -> Option<OpCreator> {
- match inner_type {
- msg::Any::CompilerConfig => Some(op_compiler_config),
- msg::Any::Cwd => Some(op_cwd),
- msg::Any::FetchModuleMetaData => Some(op_fetch_module_meta_data),
- msg::Any::WorkerGetMessage => Some(op_worker_get_message),
- msg::Any::WorkerPostMessage => Some(op_worker_post_message),
- msg::Any::Exit => Some(op_exit),
- msg::Any::Start => Some(op_start),
- _ => None,
- }
-}
-
/// Standard ops set for most isolates
pub fn op_selector_std(inner_type: msg::Any) -> Option<OpCreator> {
match inner_type {
msg::Any::Accept => Some(op_accept),
+ msg::Any::Cache => Some(op_cache),
msg::Any::Chdir => Some(op_chdir),
msg::Any::Chmod => Some(op_chmod),
msg::Any::Chown => Some(op_chown),
msg::Any::Close => Some(op_close),
msg::Any::CopyFile => Some(op_copy_file),
+ msg::Any::CreateWorker => Some(op_create_worker),
msg::Any::Cwd => Some(op_cwd),
msg::Any::Dial => Some(op_dial),
msg::Any::Environ => Some(op_env),
msg::Any::Exit => Some(op_exit),
msg::Any::Fetch => Some(op_fetch),
+ msg::Any::FetchModuleMetaData => Some(op_fetch_module_meta_data),
msg::Any::FormatError => Some(op_format_error),
msg::Any::GetRandomValues => Some(op_get_random_values),
msg::Any::GlobalTimer => Some(op_global_timer),
msg::Any::GlobalTimerStop => Some(op_global_timer_stop),
+ msg::Any::HostGetMessage => Some(op_host_get_message),
+ msg::Any::HostGetWorkerClosed => Some(op_host_get_worker_closed),
+ msg::Any::HostPostMessage => Some(op_host_post_message),
msg::Any::IsTTY => Some(op_is_tty),
msg::Any::Kill => Some(op_kill),
msg::Any::Link => Some(op_link),
@@ -237,10 +229,6 @@ pub fn op_selector_std(inner_type: msg::Any) -> Option<OpCreator> {
msg::Any::Symlink => Some(op_symlink),
msg::Any::Truncate => Some(op_truncate),
msg::Any::Utime => Some(op_utime),
- msg::Any::CreateWorker => Some(op_create_worker),
- msg::Any::HostGetWorkerClosed => Some(op_host_get_worker_closed),
- msg::Any::HostGetMessage => Some(op_host_get_message),
- msg::Any::HostPostMessage => Some(op_host_post_message),
msg::Any::Write => Some(op_write),
// TODO(ry) split these out so that only the appropriate Workers can access
@@ -446,6 +434,53 @@ pub fn odd_future(err: DenoError) -> Box<OpWithError> {
Box::new(futures::future::err(err))
}
+fn op_cache(
+ state: &ThreadSafeState,
+ base: &msg::Base<'_>,
+ data: Option<PinnedBuf>,
+) -> Box<OpWithError> {
+ assert!(data.is_none());
+ let inner = base.inner_as_cache().unwrap();
+ let extension = inner.extension().unwrap();
+ let module_id = inner.module_id().unwrap();
+ let contents = inner.contents().unwrap();
+
+ state.mark_compiled(&module_id);
+
+ // TODO It shouldn't be necessary to call fetch_module_meta_data() here.
+ // However, we need module_meta_data.source_code in order to calculate the
+ // cache path. In the future, checksums will not be used in the cache
+ // filenames and this requirement can be removed. See
+ // https://github.com/denoland/deno/issues/2057
+ let r = state.dir.fetch_module_meta_data(module_id, ".", true, true);
+ if let Err(err) = r {
+ return odd_future(err);
+ }
+ let module_meta_data = r.unwrap();
+
+ let (js_cache_path, source_map_path) = state
+ .dir
+ .cache_path(&module_meta_data.filename, &module_meta_data.source_code);
+
+ if extension == ".map" {
+ debug!("cache {:?}", source_map_path);
+ let r = fs::write(source_map_path, contents);
+ if let Err(err) = r {
+ return odd_future(err.into());
+ }
+ } else if extension == ".js" {
+ debug!("cache {:?}", js_cache_path);
+ let r = fs::write(js_cache_path, contents);
+ if let Err(err) = r {
+ return odd_future(err.into());
+ }
+ } else {
+ unreachable!();
+ }
+
+ ok_future(empty_buf())
+}
+
// https://github.com/denoland/deno/blob/golang/os.go#L100-L154
fn op_fetch_module_meta_data(
state: &ThreadSafeState,
@@ -498,41 +533,6 @@ fn op_fetch_module_meta_data(
Box::new(futures::future::result(tokio_util::block_on(fut)))
}
-/// Retrieve any relevant compiler configuration.
-fn op_compiler_config(
- state: &ThreadSafeState,
- base: &msg::Base<'_>,
- data: Option<PinnedBuf>,
-) -> Box<OpWithError> {
- assert!(data.is_none());
- let inner = base.inner_as_compiler_config().unwrap();
- let cmd_id = base.cmd_id();
- let compiler_type = inner.compiler_type().unwrap();
-
- Box::new(futures::future::result(|| -> OpResult {
- let builder = &mut FlatBufferBuilder::new();
- let (path, out) = match get_compiler_config(state, compiler_type) {
- Some(val) => val,
- _ => ("".to_owned(), vec![]),
- };
- let data_off = builder.create_vector(&out);
- let msg_args = msg::CompilerConfigResArgs {
- path: Some(builder.create_string(&path)),
- data: Some(data_off),
- };
- let inner = msg::CompilerConfigRes::create(builder, &msg_args);
- Ok(serialize_response(
- cmd_id,
- builder,
- msg::BaseArgs {
- inner: Some(inner.as_union_value()),
- inner_type: msg::Any::CompilerConfigRes,
- ..Default::default()
- },
- ))
- }()))
-}
-
fn op_chdir(
_state: &ThreadSafeState,
base: &msg::Base<'_>,
diff --git a/cli/state.rs b/cli/state.rs
index 2254f8e25..dffd1202f 100644
--- a/cli/state.rs
+++ b/cli/state.rs
@@ -15,6 +15,7 @@ use deno::PinnedBuf;
use futures::future::Shared;
use std;
use std::collections::HashMap;
+use std::collections::HashSet;
use std::env;
use std::fs;
use std::ops::Deref;
@@ -37,6 +38,7 @@ pub struct Metrics {
pub bytes_sent_data: AtomicUsize,
pub bytes_received: AtomicUsize,
pub resolve_count: AtomicUsize,
+ pub compiler_starts: AtomicUsize,
}
/// Isolate cannot be passed between threads but ThreadSafeState can.
@@ -66,6 +68,11 @@ pub struct State {
pub dispatch_selector: ops::OpSelector,
/// Reference to global progress bar.
pub progress: Progress,
+
+ /// Set of all URLs that have been compiled. This is a hacky way to work
+ /// around the fact that --reload will force multiple compilations of the same
+ /// module.
+ compiled: Mutex<HashSet<String>>,
}
impl Clone for ThreadSafeState {
@@ -157,6 +164,7 @@ impl ThreadSafeState {
resource,
dispatch_selector,
progress,
+ compiled: Mutex::new(HashSet::new()),
}))
}
@@ -177,6 +185,16 @@ impl ThreadSafeState {
}
}
+ pub fn mark_compiled(&self, module_id: &str) {
+ let mut c = self.compiled.lock().unwrap();
+ c.insert(module_id.to_string());
+ }
+
+ pub fn has_compiled(&self, module_id: &str) -> bool {
+ let c = self.compiled.lock().unwrap();
+ c.contains(module_id)
+ }
+
#[inline]
pub fn check_read(&self, filename: &str) -> DenoResult<()> {
self.permissions.check_read(filename)
diff --git a/cli/worker.rs b/cli/worker.rs
index 98bea6eb8..59eecda6f 100644
--- a/cli/worker.rs
+++ b/cli/worker.rs
@@ -212,35 +212,39 @@ fn fetch_module_meta_data_and_maybe_compile_async(
specifier: &str,
referrer: &str,
) -> impl Future<Item = ModuleMetaData, Error = DenoError> {
- let use_cache = !state.flags.reload;
- let no_fetch = state.flags.no_fetch;
let state_ = state.clone();
let specifier = specifier.to_string();
let referrer = referrer.to_string();
- state
- .dir
- .fetch_module_meta_data_async(&specifier, &referrer, use_cache, no_fetch)
- .and_then(move |out| {
- if out.media_type == msg::MediaType::TypeScript
- && !out.has_output_code_and_source_map()
- {
- debug!(">>>>> compile_sync START");
- Either::A(
- compile_async(state_.clone(), &specifier, &referrer, &out)
- .map_err(|e| {
- debug!("compiler error exiting!");
- eprintln!("{}", JSErrorColor(&e).to_string());
- std::process::exit(1);
- }).and_then(move |out| {
- debug!(">>>>> compile_sync END");
- state_.dir.code_cache(&out)?;
- Ok(out)
- }),
- )
- } else {
- Either::B(futures::future::ok(out))
- }
- })
+
+ let f = futures::future::result(Worker::resolve(&specifier, &referrer));
+ f.and_then(move |module_id| {
+ let use_cache = !state_.flags.reload || state_.has_compiled(&module_id);
+ let no_fetch = state_.flags.no_fetch;
+
+ state_
+ .dir
+ .fetch_module_meta_data_async(&specifier, &referrer, use_cache, no_fetch)
+ .and_then(move |out| {
+ if out.media_type == msg::MediaType::TypeScript
+ && !out.has_output_code_and_source_map()
+ {
+ debug!(">>>>> compile_sync START");
+ Either::A(
+ compile_async(state_.clone(), &specifier, &referrer, &out)
+ .map_err(|e| {
+ debug!("compiler error exiting!");
+ eprintln!("{}", JSErrorColor(&e).to_string());
+ std::process::exit(1);
+ }).and_then(move |out| {
+ debug!(">>>>> compile_sync END");
+ Ok(out)
+ }),
+ )
+ } else {
+ Either::B(futures::future::ok(out))
+ }
+ })
+ })
}
pub fn fetch_module_meta_data_and_maybe_compile(
@@ -297,6 +301,8 @@ mod tests {
let metrics = &state_.metrics;
assert_eq!(metrics.resolve_count.load(Ordering::SeqCst), 2);
+ // Check that we didn't start the compiler.
+ assert_eq!(metrics.compiler_starts.load(Ordering::SeqCst), 0);
}
#[test]
@@ -327,6 +333,8 @@ mod tests {
let metrics = &state_.metrics;
assert_eq!(metrics.resolve_count.load(Ordering::SeqCst), 2);
+ // Check that we didn't start the compiler.
+ assert_eq!(metrics.compiler_starts.load(Ordering::SeqCst), 0);
}
#[test]
@@ -361,6 +369,8 @@ mod tests {
let metrics = &state_.metrics;
assert_eq!(metrics.resolve_count.load(Ordering::SeqCst), 3);
+ // Check that we've only invoked the compiler once.
+ assert_eq!(metrics.compiler_starts.load(Ordering::SeqCst), 1);
}
fn create_test_worker() -> Worker {
@@ -459,20 +469,24 @@ mod tests {
#[test]
fn execute_mod_resolve_error() {
- // "foo" is not a vailid module specifier so this should return an error.
- let worker = create_test_worker();
- let js_url = root_specifier_to_url("does-not-exist").unwrap();
- let result = worker.execute_mod_async(&js_url, false).wait();
- assert!(result.is_err());
+ tokio_util::init(|| {
+ // "foo" is not a vailid module specifier so this should return an error.
+ let worker = create_test_worker();
+ let js_url = root_specifier_to_url("does-not-exist").unwrap();
+ let result = worker.execute_mod_async(&js_url, false).wait();
+ assert!(result.is_err());
+ })
}
#[test]
fn execute_mod_002_hello() {
- // This assumes cwd is project root (an assumption made throughout the
- // tests).
- let worker = create_test_worker();
- let js_url = root_specifier_to_url("./tests/002_hello.ts").unwrap();
- let result = worker.execute_mod_async(&js_url, false).wait();
- assert!(result.is_ok());
+ tokio_util::init(|| {
+ // This assumes cwd is project root (an assumption made throughout the
+ // tests).
+ let worker = create_test_worker();
+ let js_url = root_specifier_to_url("./tests/002_hello.ts").unwrap();
+ let result = worker.execute_mod_async(&js_url, false).wait();
+ assert!(result.is_ok());
+ })
}
}
diff --git a/core/shared_queue.js b/core/shared_queue.js
index 3a447cedb..75f370ce4 100644
--- a/core/shared_queue.js
+++ b/core/shared_queue.js
@@ -109,7 +109,7 @@ SharedQueue Binary Layout
let end = off + buf.byteLength;
let index = numRecords();
if (end > shared32.byteLength || index >= MAX_RECORDS) {
- console.log("shared_queue.ts push fail");
+ // console.log("shared_queue.js push fail");
return false;
}
setEnd(index, end);
diff --git a/js/compiler.ts b/js/compiler.ts
index d830dd711..928fa09d8 100644
--- a/js/compiler.ts
+++ b/js/compiler.ts
@@ -1,76 +1,40 @@
// Copyright 2018-2019 the Deno authors. All rights reserved. MIT license.
-import * as ts from "typescript";
import * as msg from "gen/cli/msg_generated";
-import { window } from "./window";
-import { assetSourceCode } from "./assets";
-import { bold, cyan, yellow } from "./colors";
-import { Console } from "./console";
import { core } from "./core";
-import { cwd } from "./dir";
-import { sendSync } from "./dispatch";
import * as flatbuffers from "./flatbuffers";
+import { sendSync } from "./dispatch";
+import { TextDecoder } from "./text_encoding";
+import * as ts from "typescript";
import * as os from "./os";
-import { TextDecoder, TextEncoder } from "./text_encoding";
-import { clearTimer, setTimeout } from "./timers";
+import { bold, cyan, yellow } from "./colors";
+import { window } from "./window";
import { postMessage, workerClose, workerMain } from "./workers";
-import { assert, log, notImplemented } from "./util";
-
-const EOL = "\n";
-const ASSETS = "$asset$";
-const LIB_RUNTIME = `${ASSETS}/lib.deno_runtime.d.ts`;
+import { Console } from "./console";
+import { assert, notImplemented } from "./util";
+import * as util from "./util";
+import { cwd } from "./dir";
+import { assetSourceCode } from "./assets";
-// An instance of console
+// Startup boilerplate. This is necessary because the compiler has its own
+// snapshot. (It would be great if we could remove these things or centralize
+// them somewhere else.)
const console = new Console(core.print);
-
-/** The location that a module is being loaded from. This could be a directory,
- * like `.`, or it could be a module specifier like
- * `http://gist.github.com/somefile.ts`
- */
-type ContainingFile = string;
-/** The internal local filename of a compiled module. It will often be something
- * like `/home/ry/.deno/gen/f7b4605dfbc4d3bb356e98fda6ceb1481e4a8df5.js`
- */
-type ModuleFileName = string;
-/** The original resolved resource name.
- * Path to cached module file or URL from which dependency was retrieved
- */
-type ModuleId = string;
-/** The external name of a module - could be a URL or could be a relative path.
- * Examples `http://gist.github.com/somefile.ts` or `./somefile.ts`
- */
-type ModuleSpecifier = string;
-/** The compiled source code which is cached in `.deno/gen/` */
-type OutputCode = string;
-/** The original source code */
-type SourceCode = string;
-/** The output source map */
-type SourceMap = string;
-
-/** The format of the work message payload coming from the privileged side */
-interface CompilerLookup {
- specifier: ModuleSpecifier;
- referrer: ContainingFile;
- cmdId: number;
+window.console = console;
+window.workerMain = workerMain;
+export default function denoMain(): void {
+ os.start("TS");
}
-/** Abstraction of the APIs required from the `os` module so they can be
- * easily mocked.
- */
-interface Os {
- fetchModuleMetaData: typeof os.fetchModuleMetaData;
- exit: typeof os.exit;
- noColor: typeof os.noColor;
-}
+const ASSETS = "$asset$";
+const OUT_DIR = "$deno$";
-/** Abstraction of the APIs required from the `typescript` module so they can
- * be easily mocked.
- */
-interface Ts {
- convertCompilerOptionsFromJson: typeof ts.convertCompilerOptionsFromJson;
- createLanguageService: typeof ts.createLanguageService;
- formatDiagnosticsWithColorAndContext: typeof ts.formatDiagnosticsWithColorAndContext;
- formatDiagnostics: typeof ts.formatDiagnostics;
- parseConfigFileTextToJson: typeof ts.parseConfigFileTextToJson;
+/** The format of the work message payload coming from the privileged side */
+interface CompilerReq {
+ rootNames: string[];
+ // TODO(ry) add compiler config to this interface.
+ // options: ts.CompilerOptions;
+ configPath?: string;
+ config?: string;
}
/** Options that either do nothing in Deno, or would cause undesired behavior
@@ -134,48 +98,68 @@ const ignoredCompilerOptions: ReadonlyArray<string> = [
"watch"
];
-/** A simple object structure for caching resolved modules and their contents.
- *
- * Named `ModuleMetaData` to clarify it is just a representation of meta data of
- * the module, not the actual module instance.
- */
-class ModuleMetaData implements ts.IScriptSnapshot {
- public scriptVersion = "";
-
- constructor(
- public readonly moduleId: ModuleId,
- public readonly fileName: ModuleFileName,
- public readonly mediaType: msg.MediaType,
- public readonly sourceCode: SourceCode = "",
- public outputCode: OutputCode = "",
- public sourceMap: SourceMap = ""
- ) {
- if (outputCode !== "" || fileName.endsWith(".d.ts")) {
- this.scriptVersion = "1";
- }
- }
-
- /** TypeScript IScriptSnapshot Interface */
-
- public getText(start: number, end: number): string {
- return start === 0 && end === this.sourceCode.length
- ? this.sourceCode
- : this.sourceCode.substring(start, end);
- }
+interface ModuleMetaData {
+ moduleName: string | undefined;
+ filename: string | undefined;
+ mediaType: msg.MediaType;
+ sourceCode: string | undefined;
+}
- public getLength(): number {
- return this.sourceCode.length;
- }
+function fetchModuleMetaData(
+ specifier: string,
+ referrer: string
+): ModuleMetaData {
+ util.log("compiler.fetchModuleMetaData", { specifier, referrer });
+ // Send FetchModuleMetaData message
+ const builder = flatbuffers.createBuilder();
+ const specifier_ = builder.createString(specifier);
+ const referrer_ = builder.createString(referrer);
+ const inner = msg.FetchModuleMetaData.createFetchModuleMetaData(
+ builder,
+ specifier_,
+ referrer_
+ );
+ const baseRes = sendSync(builder, msg.Any.FetchModuleMetaData, inner);
+ assert(baseRes != null);
+ assert(
+ msg.Any.FetchModuleMetaDataRes === baseRes!.innerType(),
+ `base.innerType() unexpectedly is ${baseRes!.innerType()}`
+ );
+ const fetchModuleMetaDataRes = new msg.FetchModuleMetaDataRes();
+ assert(baseRes!.inner(fetchModuleMetaDataRes) != null);
+ const dataArray = fetchModuleMetaDataRes.dataArray();
+ const decoder = new TextDecoder();
+ const sourceCode = dataArray ? decoder.decode(dataArray) : undefined;
+ // flatbuffers returns `null` for an empty value, this does not fit well with
+ // idiomatic TypeScript under strict null checks, so converting to `undefined`
+ return {
+ moduleName: fetchModuleMetaDataRes.moduleName() || undefined,
+ filename: fetchModuleMetaDataRes.filename() || undefined,
+ mediaType: fetchModuleMetaDataRes.mediaType(),
+ sourceCode
+ };
+}
- public getChangeRange(): undefined {
- // Required `IScriptSnapshot` API, but not implemented/needed in deno
- return undefined;
- }
+/** For caching source map and compiled js */
+function cache(extension: string, moduleId: string, contents: string): void {
+ util.log("compiler.cache", moduleId);
+ const builder = flatbuffers.createBuilder();
+ const extension_ = builder.createString(extension);
+ const moduleId_ = builder.createString(moduleId);
+ const contents_ = builder.createString(contents);
+ const inner = msg.Cache.createCache(
+ builder,
+ extension_,
+ moduleId_,
+ contents_
+ );
+ const baseRes = sendSync(builder, msg.Any.Cache, inner);
+ assert(baseRes == null);
}
/** Returns the TypeScript Extension enum for a given media type. */
function getExtension(
- fileName: ModuleFileName,
+ fileName: string,
mediaType: msg.MediaType
): ts.Extension {
switch (mediaType) {
@@ -191,294 +175,34 @@ function getExtension(
}
}
-/** Generate output code for a provided JSON string along with its source. */
-function jsonEsmTemplate(
- jsonString: string,
- sourceFileName: string
-): OutputCode {
- return (
- `const _json = JSON.parse(\`${jsonString}\`);\n` +
- `export default _json;\n` +
- `//# sourceURL=${sourceFileName}\n`
- );
-}
-
-/** A singleton class that combines the TypeScript Language Service host API
- * with Deno specific APIs to provide an interface for compiling and running
- * TypeScript and JavaScript modules.
- */
-class Compiler implements ts.LanguageServiceHost, ts.FormatDiagnosticsHost {
- // Modules are usually referenced by their ModuleSpecifier and ContainingFile,
- // and keeping a map of the resolved module file name allows more efficient
- // future resolution
- private readonly _fileNamesMap = new Map<
- ContainingFile,
- Map<ModuleSpecifier, ModuleFileName>
- >();
- // A reference to the log utility, so it can be monkey patched during testing
- private _log = log;
- // A map of module file names to module meta data
- private readonly _moduleMetaDataMap = new Map<
- ModuleFileName,
- ModuleMetaData
- >();
- // TODO ideally this are not static and can be influenced by command line
- // arguments
+class Host implements ts.CompilerHost {
private readonly _options: ts.CompilerOptions = {
allowJs: true,
allowNonTsExtensions: true,
- checkJs: true,
+ checkJs: false,
esModuleInterop: true,
module: ts.ModuleKind.ESNext,
- outDir: "$deno$",
+ outDir: OUT_DIR,
resolveJsonModule: true,
sourceMap: true,
stripComments: true,
target: ts.ScriptTarget.ESNext
};
- // A reference to the `./os.ts` module, so it can be monkey patched during
- // testing
- private _os: Os = os;
- // Used to contain the script file we are currently running
- private _scriptFileNames: string[] = [];
- // A reference to the TypeScript LanguageService instance so it can be
- // monkey patched during testing
- private _service: ts.LanguageService;
- // A reference to `typescript` module so it can be monkey patched during
- // testing
- private _ts: Ts = ts;
-
- private readonly _assetsSourceCode: { [key: string]: string };
-
- /** The TypeScript language service often refers to the resolved fileName of
- * a module, this is a shortcut to avoid unnecessary module resolution logic
- * for modules that may have been initially resolved by a `moduleSpecifier`
- * and `containingFile`. Also, `resolveModule()` throws when the module
- * cannot be resolved, which isn't always valid when dealing with the
- * TypeScript compiler, but the TypeScript compiler shouldn't be asking about
- * external modules that we haven't told it about yet.
- */
- private _getModuleMetaData(
- fileName: ModuleFileName
- ): ModuleMetaData | undefined {
- return (
- this._moduleMetaDataMap.get(fileName) ||
- (fileName.startsWith(ASSETS)
- ? this._resolveModule(fileName, "")
- : undefined)
- );
- }
-
- /** Log TypeScript diagnostics to the console and exit */
- private _logDiagnostics(diagnostics: ts.Diagnostic[]): never {
- const errMsg = this._os.noColor
- ? this._ts.formatDiagnostics(diagnostics, this)
- : this._ts.formatDiagnosticsWithColorAndContext(diagnostics, this);
-
- console.log(errMsg);
- // TODO The compiler isolate shouldn't exit. Errors should be forwarded to
- // to the caller and the caller exit.
- return this._os.exit(1);
- }
-
- /** Given a `moduleSpecifier` and `containingFile` retrieve the cached
- * `fileName` for a given module. If the module has yet to be resolved
- * this will return `undefined`.
- */
- private _resolveFileName(
- moduleSpecifier: ModuleSpecifier,
- containingFile: ContainingFile
- ): ModuleFileName | undefined {
- this._log("compiler._resolveFileName", { moduleSpecifier, containingFile });
- const innerMap = this._fileNamesMap.get(containingFile);
- if (innerMap) {
- return innerMap.get(moduleSpecifier);
- }
- return undefined;
- }
-
- /** Given a `moduleSpecifier` and `containingFile`, resolve the module and
- * return the `ModuleMetaData`.
- */
- private _resolveModule(
- moduleSpecifier: ModuleSpecifier,
- containingFile: ContainingFile
- ): ModuleMetaData {
- this._log("compiler._resolveModule", { moduleSpecifier, containingFile });
- assert(moduleSpecifier != null && moduleSpecifier.length > 0);
- let fileName = this._resolveFileName(moduleSpecifier, containingFile);
- if (fileName && this._moduleMetaDataMap.has(fileName)) {
- return this._moduleMetaDataMap.get(fileName)!;
- }
- let moduleId: ModuleId | undefined;
- let mediaType = msg.MediaType.Unknown;
- let sourceCode: SourceCode | undefined;
- if (
- moduleSpecifier.startsWith(ASSETS) ||
- containingFile.startsWith(ASSETS)
- ) {
- // Assets are compiled into the runtime javascript bundle.
- // we _know_ `.pop()` will return a string, but TypeScript doesn't so
- // not null assertion
- moduleId = moduleSpecifier.split("/").pop()!;
- const assetName = moduleId.includes(".") ? moduleId : `${moduleId}.d.ts`;
- assert(
- assetName in this._assetsSourceCode,
- `No such asset "${assetName}"`
- );
- mediaType = msg.MediaType.TypeScript;
- sourceCode = this._assetsSourceCode[assetName];
- fileName = `${ASSETS}/${assetName}`;
- } else {
- // We query Rust with a CodeFetch message. It will load the sourceCode,
- // and if there is any outputCode cached, will return that as well.
- const fetchResponse = this._os.fetchModuleMetaData(
- moduleSpecifier,
- containingFile
- );
- moduleId = fetchResponse.moduleName;
- fileName = fetchResponse.filename;
- mediaType = fetchResponse.mediaType;
- sourceCode = fetchResponse.sourceCode;
- }
- assert(moduleId != null, "No module ID.");
- assert(fileName != null, "No file name.");
- assert(
- mediaType !== msg.MediaType.Unknown,
- `Unknown media type for: "${moduleSpecifier}" from "${containingFile}".`
- );
- this._log(
- "resolveModule sourceCode length:",
- sourceCode && sourceCode.length
- );
- this._log("resolveModule has media type:", msg.MediaType[mediaType]);
- // fileName is asserted above, but TypeScript does not track so not null
- this._setFileName(moduleSpecifier, containingFile, fileName!);
- if (fileName && this._moduleMetaDataMap.has(fileName)) {
- return this._moduleMetaDataMap.get(fileName)!;
- }
- const moduleMetaData = new ModuleMetaData(
- moduleId!,
- fileName!,
- mediaType,
- sourceCode
- );
- this._moduleMetaDataMap.set(fileName!, moduleMetaData);
- return moduleMetaData;
- }
-
- /** Caches the resolved `fileName` in relationship to the `moduleSpecifier`
- * and `containingFile` in order to reduce calls to the privileged side
- * to retrieve the contents of a module.
- */
- private _setFileName(
- moduleSpecifier: ModuleSpecifier,
- containingFile: ContainingFile,
- fileName: ModuleFileName
- ): void {
- this._log("compiler._setFileName", { moduleSpecifier, containingFile });
- let innerMap = this._fileNamesMap.get(containingFile);
- if (!innerMap) {
- innerMap = new Map();
- this._fileNamesMap.set(containingFile, innerMap);
- }
- innerMap.set(moduleSpecifier, fileName);
- }
-
- constructor(assetsSourceCode: { [key: string]: string }) {
- this._assetsSourceCode = assetsSourceCode;
- this._service = this._ts.createLanguageService(this);
- }
-
- // Deno specific compiler API
-
- /** Retrieve the output of the TypeScript compiler for a given module.
- */
- compile(
- moduleSpecifier: ModuleSpecifier,
- containingFile: ContainingFile
- ): { outputCode: OutputCode; sourceMap: SourceMap } {
- this._log("compiler.compile", { moduleSpecifier, containingFile });
- const moduleMetaData = this._resolveModule(moduleSpecifier, containingFile);
- const { fileName, mediaType, sourceCode } = moduleMetaData;
- this._scriptFileNames = [fileName];
- let outputCode: string;
- let sourceMap = "";
- // Instead of using TypeScript to transpile JSON modules, we will just do
- // it directly.
- if (mediaType === msg.MediaType.Json) {
- outputCode = moduleMetaData.outputCode = jsonEsmTemplate(
- sourceCode,
- fileName
- );
- } else {
- const service = this._service;
- assert(
- mediaType === msg.MediaType.TypeScript ||
- mediaType === msg.MediaType.JavaScript
- );
- const output = service.getEmitOutput(fileName);
-
- // Get the relevant diagnostics - this is 3x faster than
- // `getPreEmitDiagnostics`.
- const diagnostics = [
- // TypeScript is overly opinionated that only CommonJS modules kinds can
- // support JSON imports. Allegedly this was fixed in
- // Microsoft/TypeScript#26825 but that doesn't seem to be working here,
- // so we will ignore complaints about this compiler setting.
- ...service
- .getCompilerOptionsDiagnostics()
- .filter((diagnostic): boolean => diagnostic.code !== 5070),
- ...service.getSyntacticDiagnostics(fileName),
- ...service.getSemanticDiagnostics(fileName)
- ];
- if (diagnostics.length > 0) {
- this._logDiagnostics(diagnostics);
- }
-
- assert(
- !output.emitSkipped,
- "The emit was skipped for an unknown reason."
- );
-
- assert(
- output.outputFiles.length === 2,
- `Expected 2 files to be emitted, got ${output.outputFiles.length}.`
- );
-
- const [sourceMapFile, outputFile] = output.outputFiles;
- assert(
- sourceMapFile.name.endsWith(".map"),
- "Expected first emitted file to be a source map"
- );
- assert(
- outputFile.name.endsWith(".js"),
- "Expected second emitted file to be JavaScript"
- );
- outputCode = moduleMetaData.outputCode = `${
- outputFile.text
- }\n//# sourceURL=${fileName}`;
- sourceMap = moduleMetaData.sourceMap = sourceMapFile.text;
- }
-
- moduleMetaData.scriptVersion = "1";
- return { outputCode, sourceMap };
- }
/** Take a configuration string, parse it, and use it to merge with the
* compiler's configuration options. The method returns an array of compiler
* options which were ignored, or `undefined`.
*/
configure(path: string, configurationText: string): string[] | undefined {
- this._log("compile.configure", path);
- const { config, error } = this._ts.parseConfigFileTextToJson(
+ util.log("compile.configure", path);
+ const { config, error } = ts.parseConfigFileTextToJson(
path,
configurationText
);
if (error) {
this._logDiagnostics([error]);
}
- const { options, errors } = this._ts.convertCompilerOptionsFromJson(
+ const { options, errors } = ts.convertCompilerOptionsFromJson(
config.compilerOptions,
cwd()
);
@@ -499,195 +223,201 @@ class Compiler implements ts.LanguageServiceHost, ts.FormatDiagnosticsHost {
return ignoredOptions.length ? ignoredOptions : undefined;
}
- // TypeScript Language Service and Format Diagnostic Host API
-
- getCanonicalFileName(fileName: string): string {
- this._log("getCanonicalFileName", fileName);
- return fileName;
- }
-
getCompilationSettings(): ts.CompilerOptions {
- this._log("getCompilationSettings()");
+ util.log("getCompilationSettings()");
return this._options;
}
- getNewLine(): string {
- return EOL;
+ /** Log TypeScript diagnostics to the console and exit */
+ _logDiagnostics(diagnostics: ReadonlyArray<ts.Diagnostic>): never {
+ const errMsg = os.noColor
+ ? ts.formatDiagnostics(diagnostics, this)
+ : ts.formatDiagnosticsWithColorAndContext(diagnostics, this);
+
+ console.log(errMsg);
+ // TODO The compiler isolate shouldn't call os.exit(). (In fact, it
+ // shouldn't even have access to call that op.) Errors should be forwarded
+ // to to the caller and the caller exit.
+ return os.exit(1);
}
- getScriptFileNames(): string[] {
- // This is equal to `"files"` in the `tsconfig.json`, therefore we only need
- // to include the actual base source files we are evaluating at the moment,
- // which would be what is set during the `.compile()`
- return this._scriptFileNames;
+ fileExists(_fileName: string): boolean {
+ return notImplemented();
}
- getScriptKind(fileName: ModuleFileName): ts.ScriptKind {
- this._log("getScriptKind()", fileName);
- const moduleMetaData = this._getModuleMetaData(fileName);
- if (moduleMetaData) {
- switch (moduleMetaData.mediaType) {
- case msg.MediaType.TypeScript:
- return ts.ScriptKind.TS;
- case msg.MediaType.JavaScript:
- return ts.ScriptKind.JS;
- case msg.MediaType.Json:
- return ts.ScriptKind.JSON;
- default:
- return this._options.allowJs ? ts.ScriptKind.JS : ts.ScriptKind.TS;
- }
- } else {
- return this._options.allowJs ? ts.ScriptKind.JS : ts.ScriptKind.TS;
+ readFile(_fileName: string): string | undefined {
+ return notImplemented();
+ }
+
+ getSourceFile(
+ fileName: string,
+ languageVersion: ts.ScriptTarget,
+ onError?: (message: string) => void,
+ shouldCreateNewSourceFile?: boolean
+ ): ts.SourceFile | undefined {
+ assert(!shouldCreateNewSourceFile);
+ util.log("getSourceFile", fileName);
+ const moduleMetaData = this._resolveModule(fileName, ".");
+ if (!moduleMetaData || !moduleMetaData.sourceCode) {
+ return undefined;
}
+ return ts.createSourceFile(
+ fileName,
+ moduleMetaData.sourceCode,
+ languageVersion
+ );
}
- getScriptVersion(fileName: ModuleFileName): string {
- const moduleMetaData = this._getModuleMetaData(fileName);
- const version = (moduleMetaData && moduleMetaData.scriptVersion) || "";
- this._log("getScriptVersion()", fileName, version);
- return version;
+ getDefaultLibFileName(_options: ts.CompilerOptions): string {
+ return ASSETS + "/lib.deno_runtime.d.ts";
}
- getScriptSnapshot(fileName: ModuleFileName): ts.IScriptSnapshot | undefined {
- this._log("getScriptSnapshot()", fileName);
- return this._getModuleMetaData(fileName);
+ writeFile(
+ fileName: string,
+ data: string,
+ writeByteOrderMark: boolean,
+ onError?: (message: string) => void,
+ sourceFiles?: ReadonlyArray<ts.SourceFile>
+ ): void {
+ util.log("writeFile", fileName);
+ assert(sourceFiles != null && sourceFiles.length == 1);
+ const sourceFileName = sourceFiles![0].fileName;
+
+ if (fileName.endsWith(".map")) {
+ // Source Map
+ cache(".map", sourceFileName, data);
+ } else if (fileName.endsWith(".js") || fileName.endsWith(".json")) {
+ // Compiled JavaScript
+ cache(".js", sourceFileName, data);
+ } else {
+ assert(false, "Trying to cache unhandled file type " + fileName);
+ }
}
getCurrentDirectory(): string {
- this._log("getCurrentDirectory()");
return "";
}
- getDefaultLibFileName(): string {
- this._log("getDefaultLibFileName()");
- const moduleSpecifier = LIB_RUNTIME;
- const moduleMetaData = this._getModuleMetaData(moduleSpecifier);
- assert(moduleMetaData != null);
- return moduleMetaData!.fileName;
+ getCanonicalFileName(fileName: string): string {
+ // console.log("getCanonicalFileName", fileName);
+ return fileName;
}
useCaseSensitiveFileNames(): boolean {
- this._log("useCaseSensitiveFileNames()");
return true;
}
- readFile(path: string): string | undefined {
- this._log("readFile()", path);
- return notImplemented();
+ getNewLine(): string {
+ return "\n";
}
- fileExists(fileName: string): boolean {
- const moduleMetaData = this._getModuleMetaData(fileName);
- const exists = moduleMetaData != null;
- this._log("fileExists()", fileName, exists);
- return exists;
+ resolveModuleNames(
+ moduleNames: string[],
+ containingFile: string
+ ): Array<ts.ResolvedModuleFull | undefined> {
+ util.log("resolveModuleNames()", { moduleNames, containingFile });
+ return moduleNames.map(
+ (moduleName): ts.ResolvedModuleFull | undefined => {
+ const moduleMetaData = this._resolveModule(moduleName, containingFile);
+ if (moduleMetaData.moduleName) {
+ const resolvedFileName = moduleMetaData.moduleName;
+ // This flags to the compiler to not go looking to transpile functional
+ // code, anything that is in `/$asset$/` is just library code
+ const isExternalLibraryImport = moduleName.startsWith(ASSETS);
+ const r = {
+ resolvedFileName,
+ isExternalLibraryImport,
+ extension: getExtension(resolvedFileName, moduleMetaData.mediaType)
+ };
+ return r;
+ } else {
+ return undefined;
+ }
+ }
+ );
}
- resolveModuleNames(
- moduleNames: ModuleSpecifier[],
- containingFile: ContainingFile
- ): Array<ts.ResolvedModuleFull | ts.ResolvedModule> {
- this._log("resolveModuleNames()", { moduleNames, containingFile });
- const resolvedModuleNames: ts.ResolvedModuleFull[] = [];
- for (const moduleName of moduleNames) {
- const moduleMetaData = this._resolveModule(moduleName, containingFile);
- // According to the interface we shouldn't return `undefined` but if we
- // fail to return the same length of modules to those we cannot resolve
- // then TypeScript fails on an assertion that the lengths can't be
- // different, so we have to return an "empty" resolved module
- // TODO: all this does is push the problem downstream, and TypeScript
- // will complain it can't identify the type of the file and throw
- // a runtime exception, so we need to handle missing modules better
- const resolvedFileName = moduleMetaData.fileName || "";
- // This flags to the compiler to not go looking to transpile functional
- // code, anything that is in `/$asset$/` is just library code
- const isExternalLibraryImport = resolvedFileName.startsWith(ASSETS);
- resolvedModuleNames.push({
- resolvedFileName,
- isExternalLibraryImport,
- extension: getExtension(resolvedFileName, moduleMetaData.mediaType)
- });
+ private _resolveModule(specifier: string, referrer: string): ModuleMetaData {
+ // Handle built-in assets specially.
+ if (specifier.startsWith(ASSETS)) {
+ const moduleName = specifier.split("/").pop()!;
+ const assetName = moduleName.includes(".")
+ ? moduleName
+ : `${moduleName}.d.ts`;
+ assert(assetName in assetSourceCode, `No such asset "${assetName}"`);
+ const sourceCode = assetSourceCode[assetName];
+ return {
+ moduleName,
+ filename: specifier,
+ mediaType: msg.MediaType.TypeScript,
+ sourceCode
+ };
}
- return resolvedModuleNames;
+ return fetchModuleMetaData(specifier, referrer);
}
}
-const compiler = new Compiler(assetSourceCode);
-
-// set global objects for compiler web worker
-window.clearTimeout = clearTimer;
-window.console = console;
-window.postMessage = postMessage;
-window.setTimeout = setTimeout;
-window.workerMain = workerMain;
-window.close = workerClose;
-window.TextDecoder = TextDecoder;
-window.TextEncoder = TextEncoder;
-
// provide the "main" function that will be called by the privileged side when
// lazy instantiating the compiler web worker
window.compilerMain = function compilerMain(): void {
// workerMain should have already been called since a compiler is a worker.
- window.onmessage = ({ data }: { data: CompilerLookup }): void => {
- const { specifier, referrer, cmdId } = data;
-
- try {
- const result = compiler.compile(specifier, referrer);
- postMessage({
- success: true,
- cmdId,
- data: result
- });
- } catch (e) {
- postMessage({
- success: false,
- cmdId,
- data: JSON.parse(core.errorToJSON(e))
- });
+ window.onmessage = ({ data }: { data: CompilerReq }): void => {
+ const { rootNames, configPath, config } = data;
+ const host = new Host();
+ if (config && config.length) {
+ const ignoredOptions = host.configure(configPath!, config);
+ if (ignoredOptions) {
+ console.warn(
+ yellow(`Unsupported compiler options in "${configPath}"\n`) +
+ cyan(` The following options were ignored:\n`) +
+ ` ${ignoredOptions
+ .map((value): string => bold(value))
+ .join(", ")}`
+ );
+ }
}
- };
-};
-const decoder = new TextDecoder();
+ const options = host.getCompilationSettings();
+ const program = ts.createProgram(rootNames, options, host);
+ const emitResult = program!.emit();
+
+ // TODO(ry) Print diagnostics in Rust.
+ // https://github.com/denoland/deno/pull/2310
+
+ const diagnostics = ts
+ .getPreEmitDiagnostics(program)
+ .concat(emitResult.diagnostics)
+ .filter(
+ ({ code }): boolean => {
+ if (code === 2649) return false;
+ // TS2691: An import path cannot end with a '.ts' extension. Consider
+ // importing 'bad-module' instead.
+ if (code === 2691) return false;
+ // TS5009: Cannot find the common subdirectory path for the input files.
+ if (code === 5009) return false;
+ // TS5055: Cannot write file
+ // 'http://localhost:4545/tests/subdir/mt_application_x_javascript.j4.js'
+ // because it would overwrite input file.
+ if (code === 5055) return false;
+ // TypeScript is overly opinionated that only CommonJS modules kinds can
+ // support JSON imports. Allegedly this was fixed in
+ // Microsoft/TypeScript#26825 but that doesn't seem to be working here,
+ // so we will ignore complaints about this compiler setting.
+ if (code === 5070) return false;
+ return true;
+ }
+ );
-// Perform the op to retrieve the compiler configuration if there was any
-// provided on startup.
-function getCompilerConfig(
- compilerType: string
-): { path: string; data: string } {
- const builder = flatbuffers.createBuilder();
- const compilerType_ = builder.createString(compilerType);
- msg.CompilerConfig.startCompilerConfig(builder);
- msg.CompilerConfig.addCompilerType(builder, compilerType_);
- const inner = msg.CompilerConfig.endCompilerConfig(builder);
- const baseRes = sendSync(builder, msg.Any.CompilerConfig, inner);
- assert(baseRes != null);
- assert(msg.Any.CompilerConfigRes === baseRes!.innerType());
- const res = new msg.CompilerConfigRes();
- assert(baseRes!.inner(res) != null);
-
- // the privileged side does not normalize path separators in windows, so we
- // will normalize them here
- const path = res.path()!.replace(/\\/g, "/");
- assert(path != null);
- const dataArray = res.dataArray()!;
- assert(dataArray != null);
- const data = decoder.decode(dataArray);
- return { path, data };
-}
+ if (diagnostics.length > 0) {
+ host._logDiagnostics(diagnostics);
+ // The above _logDiagnostics calls os.exit(). The return is here just for
+ // clarity.
+ return;
+ }
-export default function denoMain(): void {
- os.start("TS");
+ postMessage(emitResult);
- const { path, data } = getCompilerConfig("typescript");
- if (data.length) {
- const ignoredOptions = compiler.configure(path, data);
- if (ignoredOptions) {
- console.warn(
- yellow(`Unsupported compiler options in "${path}"\n`) +
- cyan(` The following options were ignored:\n`) +
- ` ${ignoredOptions.map((value): string => bold(value)).join(", ")}`
- );
- }
- }
-}
+ // The compiler isolate exits after a single messsage.
+ workerClose();
+ };
+};
diff --git a/js/os.ts b/js/os.ts
index ca227e5fd..0af4098c5 100644
--- a/js/os.ts
+++ b/js/os.ts
@@ -3,7 +3,6 @@ import * as msg from "gen/cli/msg_generated";
import { core } from "./core";
import { handleAsyncMsgFromRust, sendSync } from "./dispatch";
import * as flatbuffers from "./flatbuffers";
-import { TextDecoder } from "./text_encoding";
import { assert } from "./util";
import * as util from "./util";
import { window } from "./window";
@@ -24,13 +23,6 @@ function setGlobals(pid_: number, noColor_: boolean, execPath_: string): void {
execPath = execPath_;
}
-interface ResponseModuleMetaData {
- moduleName: string | undefined;
- filename: string | undefined;
- mediaType: msg.MediaType;
- sourceCode: string | undefined;
-}
-
/** Check if running in terminal.
*
* console.log(Deno.isTTY().stdout);
@@ -54,43 +46,6 @@ export function exit(exitCode = 0): never {
return util.unreachable();
}
-const decoder = new TextDecoder();
-
-// @internal
-export function fetchModuleMetaData(
- specifier: string,
- referrer: string
-): ResponseModuleMetaData {
- util.log("os.fetchModuleMetaData", { specifier, referrer });
- // Send FetchModuleMetaData message
- const builder = flatbuffers.createBuilder();
- const specifier_ = builder.createString(specifier);
- const referrer_ = builder.createString(referrer);
- const inner = msg.FetchModuleMetaData.createFetchModuleMetaData(
- builder,
- specifier_,
- referrer_
- );
- const baseRes = sendSync(builder, msg.Any.FetchModuleMetaData, inner);
- assert(baseRes != null);
- assert(
- msg.Any.FetchModuleMetaDataRes === baseRes!.innerType(),
- `base.innerType() unexpectedly is ${baseRes!.innerType()}`
- );
- const fetchModuleMetaDataRes = new msg.FetchModuleMetaDataRes();
- assert(baseRes!.inner(fetchModuleMetaDataRes) != null);
- const dataArray = fetchModuleMetaDataRes.dataArray();
- const sourceCode = dataArray ? decoder.decode(dataArray) : undefined;
- // flatbuffers returns `null` for an empty value, this does not fit well with
- // idiomatic TypeScript under strict null checks, so converting to `undefined`
- return {
- moduleName: fetchModuleMetaDataRes.moduleName() || undefined,
- filename: fetchModuleMetaDataRes.filename() || undefined,
- mediaType: fetchModuleMetaDataRes.mediaType(),
- sourceCode
- };
-}
-
function setEnv(key: string, value: string): void {
const builder = flatbuffers.createBuilder();
const key_ = builder.createString(key);
diff --git a/rollup.config.js b/rollup.config.js
index 31ee1dc23..635aace0d 100644
--- a/rollup.config.js
+++ b/rollup.config.js
@@ -231,10 +231,14 @@ export default function makeConfig(commandOptions) {
[typescriptPath]: [
"convertCompilerOptionsFromJson",
"createLanguageService",
+ "createProgram",
+ "createSourceFile",
+ "getPreEmitDiagnostics",
"formatDiagnostics",
"formatDiagnosticsWithColorAndContext",
"parseConfigFileTextToJson",
"version",
+ "CompilerHost",
"Extension",
"ModuleKind",
"ScriptKind",
diff --git a/tests/config.ts.out b/tests/config.ts.out
index 0f79e03d1..a57d3056b 100644
--- a/tests/config.ts.out
+++ b/tests/config.ts.out
@@ -1,7 +1,6 @@
-Unsupported compiler options in "[WILDCARD]tests/config.tsconfig.json"
+[WILDCARD]Unsupported compiler options in "[WILDCARD]config.tsconfig.json"
The following options were ignored:
module, target
-
[WILDCARD]tests/config.ts:3:5 - error TS2532: Object is possibly 'undefined'.
3 if (map.get("bar").foo) {
diff --git a/tests/error_004_missing_module.ts.out b/tests/error_004_missing_module.ts.out
index f6fbf5d9b..b3a9aa65e 100644
--- a/tests/error_004_missing_module.ts.out
+++ b/tests/error_004_missing_module.ts.out
@@ -3,9 +3,9 @@
at maybeError (js/errors.ts:[WILDCARD])
at maybeThrowError (js/errors.ts:[WILDCARD])
at sendSync (js/dispatch.ts:[WILDCARD])
- at fetchModuleMetaData (js/os.ts:[WILDCARD])
+ at fetchModuleMetaData (js/compiler.ts:[WILDCARD])
at _resolveModule (js/compiler.ts:[WILDCARD])
+ at js/compiler.ts:[WILDCARD]
at resolveModuleNames (js/compiler.ts:[WILDCARD])
- at compilerHost.resolveModuleNames ([WILDCARD]typescript.js:[WILDCARD])
at resolveModuleNamesWorker ([WILDCARD]typescript.js:[WILDCARD])
at resolveModuleNamesReusingOldState ([WILDCARD]typescript.js:[WILDCARD])
diff --git a/tests/error_005_missing_dynamic_import.ts.out b/tests/error_005_missing_dynamic_import.ts.out
index d7a01fbe4..78839e8d6 100644
--- a/tests/error_005_missing_dynamic_import.ts.out
+++ b/tests/error_005_missing_dynamic_import.ts.out
@@ -3,9 +3,8 @@
at maybeError (js/errors.ts:[WILDCARD])
at maybeThrowError (js/errors.ts:[WILDCARD])
at sendSync (js/dispatch.ts:[WILDCARD])
- at fetchModuleMetaData (js/os.ts:[WILDCARD])
+ at fetchModuleMetaData (js/compiler.ts:[WILDCARD])
at _resolveModule (js/compiler.ts:[WILDCARD])
- at resolveModuleNames (js/compiler.ts:[WILDCARD])
- at compilerHost.resolveModuleNames ([WILDCARD])
+ at js/compiler.ts:[WILDCARD]
at resolveModuleNamesWorker ([WILDCARD])
at resolveModuleNamesReusingOldState ([WILDCARD]typescript.js:[WILDCARD])
diff --git a/tests/error_006_import_ext_failure.ts.out b/tests/error_006_import_ext_failure.ts.out
index 7fe154aa0..8cd05d692 100644
--- a/tests/error_006_import_ext_failure.ts.out
+++ b/tests/error_006_import_ext_failure.ts.out
@@ -3,9 +3,8 @@
at maybeError (js/errors.ts:[WILDCARD])
at maybeThrowError (js/errors.ts:[WILDCARD])
at sendSync (js/dispatch.ts:[WILDCARD])
- at fetchModuleMetaData (js/os.ts:[WILDCARD])
+ at fetchModuleMetaData (js/compiler.ts:[WILDCARD])
at _resolveModule (js/compiler.ts:[WILDCARD])
- at resolveModuleNames (js/compiler.ts:[WILDCARD])
- at compilerHost.resolveModuleNames ([WILDCARD])
+ at js/compiler.ts:[WILDCARD]
at resolveModuleNamesWorker ([WILDCARD])
at resolveModuleNamesReusingOldState ([WILDCARD]typescript.js:[WILDCARD])