summaryrefslogtreecommitdiff
path: root/cli/compilers
diff options
context:
space:
mode:
Diffstat (limited to 'cli/compilers')
-rw-r--r--cli/compilers/js.rs14
-rw-r--r--cli/compilers/json.rs21
-rw-r--r--cli/compilers/ts.rs195
-rw-r--r--cli/compilers/wasm.rs22
4 files changed, 102 insertions, 150 deletions
diff --git a/cli/compilers/js.rs b/cli/compilers/js.rs
index 4e99017b1..e6142a57e 100644
--- a/cli/compilers/js.rs
+++ b/cli/compilers/js.rs
@@ -1,25 +1,21 @@
// Copyright 2018-2020 the Deno authors. All rights reserved. MIT license.
use crate::compilers::CompiledModule;
-use crate::compilers::CompiledModuleFuture;
use crate::file_fetcher::SourceFile;
-use futures::future::FutureExt;
-use std::pin::Pin;
+use deno_core::ErrBox;
use std::str;
pub struct JsCompiler {}
impl JsCompiler {
- pub fn compile_async(
+ pub async fn compile_async(
&self,
source_file: SourceFile,
- ) -> Pin<Box<CompiledModuleFuture>> {
- let module = CompiledModule {
+ ) -> Result<CompiledModule, ErrBox> {
+ Ok(CompiledModule {
code: str::from_utf8(&source_file.source_code)
.unwrap()
.to_string(),
name: source_file.url.to_string(),
- };
-
- futures::future::ok(module).boxed()
+ })
}
}
diff --git a/cli/compilers/json.rs b/cli/compilers/json.rs
index 765b79cd6..8d9ed1c4f 100644
--- a/cli/compilers/json.rs
+++ b/cli/compilers/json.rs
@@ -1,12 +1,8 @@
// Copyright 2018-2020 the Deno authors. All rights reserved. MIT license.
use crate::compilers::CompiledModule;
-use crate::compilers::CompiledModuleFuture;
use crate::file_fetcher::SourceFile;
-use crate::futures::future::FutureExt;
use deno_core::ErrBox;
use regex::Regex;
-use std::pin::Pin;
-use std::str;
// From https://github.com/mathiasbynens/mothereff.in/blob/master/js-variables/eff.js
static JS_RESERVED_WORDS: &str = r"^(?:do|if|in|for|let|new|try|var|case|else|enum|eval|false|null|this|true|void|with|await|break|catch|class|const|super|throw|while|yield|delete|export|import|public|return|static|switch|typeof|default|extends|finally|package|private|continue|debugger|function|arguments|interface|protected|implements|instanceof)$";
@@ -14,19 +10,18 @@ static JS_RESERVED_WORDS: &str = r"^(?:do|if|in|for|let|new|try|var|case|else|en
pub struct JsonCompiler {}
impl JsonCompiler {
- pub fn compile_async(
+ pub async fn compile_async(
&self,
source_file: &SourceFile,
- ) -> Pin<Box<CompiledModuleFuture>> {
- let maybe_json_value: serde_json::Result<serde_json::Value> =
- serde_json::from_str(&str::from_utf8(&source_file.source_code).unwrap());
+ ) -> Result<CompiledModule, ErrBox> {
+ let maybe_json_value = serde_json::from_slice(&source_file.source_code);
if let Err(err) = maybe_json_value {
- return futures::future::err(ErrBox::from(err)).boxed();
+ return Err(ErrBox::from(err));
}
let mut code = format!(
"export default {};\n",
- str::from_utf8(&source_file.source_code).unwrap()
+ std::str::from_utf8(&source_file.source_code).unwrap()
);
if let serde_json::Value::Object(m) = maybe_json_value.unwrap() {
@@ -47,11 +42,9 @@ impl JsonCompiler {
}
}
- let module = CompiledModule {
+ Ok(CompiledModule {
code,
name: source_file.url.to_string(),
- };
-
- futures::future::ok(module).boxed()
+ })
}
}
diff --git a/cli/compilers/ts.rs b/cli/compilers/ts.rs
index c7896ec61..c882f7d66 100644
--- a/cli/compilers/ts.rs
+++ b/cli/compilers/ts.rs
@@ -2,7 +2,6 @@
use super::compiler_worker::CompilerWorker;
use crate::compilers::CompilationResultFuture;
use crate::compilers::CompiledModule;
-use crate::compilers::CompiledModuleFuture;
use crate::diagnostics::Diagnostic;
use crate::disk_cache::DiskCache;
use crate::file_fetcher::SourceFile;
@@ -18,7 +17,6 @@ use deno_core::Buf;
use deno_core::ErrBox;
use deno_core::ModuleSpecifier;
use futures::future::FutureExt;
-use futures::Future;
use regex::Regex;
use serde_json::json;
use std::collections::HashMap;
@@ -26,10 +24,12 @@ use std::collections::HashSet;
use std::fs;
use std::hash::BuildHasher;
use std::io;
+use std::ops::Deref;
use std::path::PathBuf;
use std::pin::Pin;
use std::str;
use std::sync::atomic::Ordering;
+use std::sync::Arc;
use std::sync::Mutex;
use url::Url;
@@ -202,7 +202,7 @@ pub fn source_code_version_hash(
crate::checksum::gen(vec![source_code, version.as_bytes(), config_hash])
}
-pub struct TsCompiler {
+pub struct TsCompilerInner {
pub file_fetcher: SourceFileFetcher,
pub config: CompilerConfig,
pub disk_cache: DiskCache,
@@ -216,6 +216,16 @@ pub struct TsCompiler {
pub compile_js: bool,
}
+#[derive(Clone)]
+pub struct TsCompiler(Arc<TsCompilerInner>);
+
+impl Deref for TsCompiler {
+ type Target = TsCompilerInner;
+ fn deref(&self) -> &Self::Target {
+ &self.0
+ }
+}
+
impl TsCompiler {
pub fn new(
file_fetcher: SourceFileFetcher,
@@ -224,17 +234,14 @@ impl TsCompiler {
config_path: Option<String>,
) -> Result<Self, ErrBox> {
let config = CompilerConfig::load(config_path)?;
-
- let compiler = Self {
+ Ok(TsCompiler(Arc::new(TsCompilerInner {
file_fetcher,
disk_cache,
compile_js: config.compile_js,
config,
compiled: Mutex::new(HashSet::new()),
use_disk_cache,
- };
-
- Ok(compiler)
+ })))
}
/// Create a new V8 worker with snapshot of TS compiler and setup compiler's
@@ -261,12 +268,12 @@ impl TsCompiler {
worker
}
- pub fn bundle_async(
+ pub async fn bundle_async(
&self,
global_state: ThreadSafeGlobalState,
module_name: String,
out_file: Option<String>,
- ) -> impl Future<Output = Result<(), ErrBox>> {
+ ) -> Result<(), ErrBox> {
debug!(
"Invoking the compiler to bundle. module_name: {}",
module_name
@@ -282,41 +289,15 @@ impl TsCompiler {
true,
);
- // TODO(ry) The code below looks very similar to spawn_ts_compiler_worker.
- // Can we combine them?
- let (load_sender, load_receiver) =
- tokio::sync::oneshot::channel::<Result<(), ErrBox>>();
- std::thread::spawn(move || {
- let mut worker = TsCompiler::setup_worker(global_state);
- let handle = worker.thread_safe_handle();
-
- let fut = async move {
- if let Err(err) = handle.post_message(req_msg).await {
- load_sender.send(Err(err)).unwrap();
- return;
- }
- debug!("Sent message to worker");
- if let Err(err) = (&mut *worker).await {
- load_sender.send(Err(err)).unwrap();
- return;
- }
- let maybe_msg = handle.get_message().await;
- debug!("Received message from worker");
- if let Some(ref msg) = maybe_msg {
- let json_str = std::str::from_utf8(msg).unwrap();
- debug!("Message: {}", json_str);
- if let Some(diagnostics) = Diagnostic::from_emit_result(json_str) {
- let err = ErrBox::from(diagnostics);
- load_sender.send(Err(err)).unwrap();
- return;
- }
- }
- load_sender.send(Ok(())).unwrap();
+ let maybe_msg = execute_in_thread(global_state.clone(), req_msg).await?;
+ if let Some(ref msg) = maybe_msg {
+ let json_str = std::str::from_utf8(msg).unwrap();
+ debug!("Message: {}", json_str);
+ if let Some(diagnostics) = Diagnostic::from_emit_result(json_str) {
+ return Err(ErrBox::from(diagnostics));
}
- .boxed_local();
- crate::tokio_util::run_basic(fut);
- });
- async { load_receiver.await.unwrap() }.boxed_local()
+ }
+ Ok(())
}
/// Mark given module URL as compiled to avoid multiple compilations of same
@@ -342,17 +323,14 @@ impl TsCompiler {
///
/// If compilation is required then new V8 worker is spawned with fresh TS
/// compiler.
- pub fn compile_async(
+ pub async fn compile_async(
&self,
global_state: ThreadSafeGlobalState,
source_file: &SourceFile,
target: TargetLib,
- ) -> Pin<Box<CompiledModuleFuture>> {
+ ) -> Result<CompiledModule, ErrBox> {
if self.has_compiled(&source_file.url) {
- return match self.get_compiled_module(&source_file.url) {
- Ok(compiled) => futures::future::ok(compiled).boxed(),
- Err(err) => futures::future::err(err).boxed(),
- };
+ return self.get_compiled_module(&source_file.url);
}
if self.use_disk_cache {
@@ -373,7 +351,7 @@ impl TsCompiler {
self.get_compiled_module(&source_file.url)
{
self.mark_compiled(&source_file.url);
- return futures::future::ok(compiled_module).boxed();
+ return Ok(compiled_module);
}
}
}
@@ -394,51 +372,22 @@ impl TsCompiler {
false,
);
- // TODO(ry) The code below looks very similar to spawn_ts_compiler_worker.
- // Can we combine them?
- let (load_sender, load_receiver) =
- tokio::sync::oneshot::channel::<Result<CompiledModule, ErrBox>>();
- std::thread::spawn(move || {
- debug!(">>>>> compile_async START");
+ let ts_compiler = self.clone();
- let mut worker = TsCompiler::setup_worker(global_state.clone());
- let handle = worker.thread_safe_handle();
+ let compiling_job = global_state
+ .progress
+ .add("Compile", &module_url.to_string());
+ let maybe_msg = execute_in_thread(global_state.clone(), req_msg).await?;
- let compiling_job = global_state
- .progress
- .add("Compile", &module_url.to_string());
-
- let fut = async move {
- if let Err(err) = handle.post_message(req_msg).await {
- load_sender.send(Err(err)).unwrap();
- return;
- }
- if let Err(err) = (&mut *worker).await {
- load_sender.send(Err(err)).unwrap();
- return;
- }
- let maybe_msg = handle.get_message().await;
- if let Some(ref msg) = maybe_msg {
- let json_str = std::str::from_utf8(msg).unwrap();
- if let Some(diagnostics) = Diagnostic::from_emit_result(json_str) {
- let err = ErrBox::from(diagnostics);
- load_sender.send(Err(err)).unwrap();
- return;
- }
- }
- let compiled_module = global_state
- .ts_compiler
- .get_compiled_module(&source_file_.url)
- .expect("Expected to find compiled file");
- drop(compiling_job);
- debug!(">>>>> compile_sync END");
- load_sender.send(Ok(compiled_module)).unwrap();
+ if let Some(ref msg) = maybe_msg {
+ let json_str = std::str::from_utf8(msg).unwrap();
+ if let Some(diagnostics) = Diagnostic::from_emit_result(json_str) {
+ return Err(ErrBox::from(diagnostics));
}
- .boxed_local();
- crate::tokio_util::run_basic(fut);
- });
-
- async { load_receiver.await.unwrap() }.boxed_local()
+ }
+ let compiled_module = ts_compiler.get_compiled_module(&source_file_.url)?;
+ drop(compiling_job);
+ Ok(compiled_module)
}
/// Get associated `CompiledFileMetadata` for given module if it exists.
@@ -654,37 +603,47 @@ impl TsCompiler {
}
}
-// TODO(ry) this is pretty general purpose and should be lifted and generalized.
-fn spawn_ts_compiler_worker(
- req_msg: Buf,
+async fn execute_in_thread(
global_state: ThreadSafeGlobalState,
-) -> Pin<Box<CompilationResultFuture>> {
+ req: Buf,
+) -> Result<Option<Buf>, ErrBox> {
let (load_sender, load_receiver) =
- tokio::sync::oneshot::channel::<JsonResult>();
-
+ tokio::sync::oneshot::channel::<Result<Option<Buf>, ErrBox>>();
std::thread::spawn(move || {
- let mut worker = TsCompiler::setup_worker(global_state);
+ debug!(">>>>> compile_async START");
+
+ let mut worker = TsCompiler::setup_worker(global_state.clone());
let handle = worker.thread_safe_handle();
- let fut = async move {
- debug!("Sent message to worker");
- if let Err(err) = handle.post_message(req_msg).await {
- load_sender.send(Err(err)).unwrap();
- return;
- }
- if let Err(err) = (&mut *worker).await {
- load_sender.send(Err(err)).unwrap();
- return;
+ crate::tokio_util::run_basic(
+ async move {
+ if let Err(err) = handle.post_message(req).await {
+ load_sender.send(Err(err)).unwrap();
+ return;
+ }
+ if let Err(err) = (&mut *worker).await {
+ load_sender.send(Err(err)).unwrap();
+ return;
+ }
+ let maybe_msg = handle.get_message().await;
+ load_sender.send(Ok(maybe_msg)).unwrap();
+ debug!(">>>>> compile_sync END");
}
- let msg = handle.get_message().await.unwrap();
- let json_str = std::str::from_utf8(&msg).unwrap();
- load_sender.send(Ok(json!(json_str))).unwrap();
- };
- crate::tokio_util::run_basic(fut);
+ .boxed_local(),
+ );
});
- let fut = async { load_receiver.await.unwrap() };
- fut.boxed_local()
+ load_receiver.await.unwrap()
+}
+
+async fn execute_in_thread_json(
+ req_msg: Buf,
+ global_state: ThreadSafeGlobalState,
+) -> JsonResult {
+ let maybe_msg = execute_in_thread(global_state, req_msg).await?;
+ let msg = maybe_msg.unwrap();
+ let json_str = std::str::from_utf8(&msg).unwrap();
+ Ok(json!(json_str))
}
pub fn runtime_compile_async<S: BuildHasher>(
@@ -706,7 +665,7 @@ pub fn runtime_compile_async<S: BuildHasher>(
.into_boxed_str()
.into_boxed_bytes();
- spawn_ts_compiler_worker(req_msg, global_state)
+ execute_in_thread_json(req_msg, global_state).boxed_local()
}
pub fn runtime_transpile_async<S: BuildHasher>(
@@ -723,7 +682,7 @@ pub fn runtime_transpile_async<S: BuildHasher>(
.into_boxed_str()
.into_boxed_bytes();
- spawn_ts_compiler_worker(req_msg, global_state)
+ execute_in_thread_json(req_msg, global_state).boxed_local()
}
#[cfg(test)]
diff --git a/cli/compilers/wasm.rs b/cli/compilers/wasm.rs
index a3e578f01..63eebadb3 100644
--- a/cli/compilers/wasm.rs
+++ b/cli/compilers/wasm.rs
@@ -1,22 +1,26 @@
// Copyright 2018-2020 the Deno authors. All rights reserved. MIT license.
use super::compiler_worker::CompilerWorker;
use crate::compilers::CompiledModule;
-use crate::compilers::CompiledModuleFuture;
use crate::file_fetcher::SourceFile;
use crate::global_state::ThreadSafeGlobalState;
use crate::startup_data;
use crate::state::*;
use deno_core::ErrBox;
use deno_core::ModuleSpecifier;
-use futures::FutureExt;
use serde_derive::Deserialize;
use serde_json;
use std::collections::HashMap;
-use std::pin::Pin;
use std::sync::atomic::Ordering;
use std::sync::{Arc, Mutex};
use url::Url;
+// TODO(ry) The entire concept of spawning a thread, sending data to JS,
+// compiling WASM there, and moving the data back into the calling thread is
+// completelly wrong. V8 has native facilities for getting this information.
+// We might be lacking bindings for this currently in rusty_v8 but ultimately
+// this "compiler" should be calling into rusty_v8 directly, not spawning
+// threads.
+
// TODO(kevinkassimo): This is a hack to encode/decode data as base64 string.
// (Since Deno namespace might not be available, Deno.read can fail).
// Binary data is already available through source_file.source_code.
@@ -67,18 +71,19 @@ impl WasmCompiler {
worker
}
- pub fn compile_async(
+ pub async fn compile_async(
&self,
global_state: ThreadSafeGlobalState,
source_file: &SourceFile,
- ) -> Pin<Box<CompiledModuleFuture>> {
+ ) -> Result<CompiledModule, ErrBox> {
let cache = self.cache.clone();
+ let cache_ = self.cache.clone();
let source_file = source_file.clone();
+
let maybe_cached = { cache.lock().unwrap().get(&source_file.url).cloned() };
if let Some(m) = maybe_cached {
- return futures::future::ok(m).boxed();
+ return Ok(m);
}
- let cache_ = self.cache.clone();
let (load_sender, load_receiver) =
tokio::sync::oneshot::channel::<Result<CompiledModule, ErrBox>>();
@@ -133,8 +138,7 @@ impl WasmCompiler {
crate::tokio_util::run_basic(fut);
});
- let fut = async { load_receiver.await.unwrap() };
- fut.boxed_local()
+ load_receiver.await.unwrap()
}
}