The edge runtime can be divided into two runtimes with different purposes.
@@ -30,7 +36,22 @@ The edge runtime can be divided into two runtimes with different purposes.
- User runtime:
- An instance for the _user runtime_ is responsible for executing users' code.
- Limits are required to be set such as: Memory and Timeouts.
- - Has access to environment variables explictly allowed by the main runtime.
+ - Has access to environment variables explicitly allowed by the main runtime.
+
+### Usage & Self-Hosting
+
+For completely usage and self-host details, visit [usage.md](/docs/usage.md)
+
+### Edge Runtime in Deep
+
+#### Conceptual
+
+- [EdgeRuntime Base](/crates/base/README.md): Overalls about how EdgeRuntime is based on Deno.
+
+#### Extension Modules
+
+- [AI](/ext/ai/README.md): Implements AI related features.
+- [NodeJs](/ext/node/README.md) & [NodeJs Polyfills](/ext/node/polyfills/README.md): Implements the NodeJs compatibility layer.
## Developers
diff --git a/assets/docs/ai/onnx-backend-dark.svg b/assets/docs/ai/onnx-backend-dark.svg
new file mode 100644
index 000000000..146c97388
--- /dev/null
+++ b/assets/docs/ai/onnx-backend-dark.svg
@@ -0,0 +1,4 @@
+
+
+
\ No newline at end of file
diff --git a/assets/docs/ai/onnx-backend.svg b/assets/docs/ai/onnx-backend.svg
new file mode 100644
index 000000000..9a2397d4d
--- /dev/null
+++ b/assets/docs/ai/onnx-backend.svg
@@ -0,0 +1,4 @@
+
+
+
\ No newline at end of file
diff --git a/assets/edge-runtime-diagram-dark.svg b/assets/edge-runtime-diagram-dark.svg
new file mode 100644
index 000000000..f19e8b5d2
--- /dev/null
+++ b/assets/edge-runtime-diagram-dark.svg
@@ -0,0 +1,4 @@
+
+
+
\ No newline at end of file
diff --git a/assets/edge-runtime-diagram.svg b/assets/edge-runtime-diagram.svg
index 63cfa5ffe..bc4036c70 100644
--- a/assets/edge-runtime-diagram.svg
+++ b/assets/edge-runtime-diagram.svg
@@ -1,21 +1,4 @@
-
+
+
+
\ No newline at end of file
diff --git a/cli/src/flags.rs b/cli/src/flags.rs
index 334a37ec6..0a85f32cd 100644
--- a/cli/src/flags.rs
+++ b/cli/src/flags.rs
@@ -364,6 +364,11 @@ fn get_bundle_command() -> Command {
.default_value("false")
.value_parser(FalseyValueParser::new()),
)
+ .arg(
+ arg!(--"timeout" )
+ .help("Maximum time in seconds that can be waited for the bundle to complete.")
+ .value_parser(value_parser!(u64).range(..u64::MAX))
+ )
}
fn get_unbundle_command() -> Command {
diff --git a/cli/src/main.rs b/cli/src/main.rs
index 4976c1654..82dcccca0 100644
--- a/cli/src/main.rs
+++ b/cli/src/main.rs
@@ -6,6 +6,7 @@ use std::path::PathBuf;
use std::process::ExitCode;
use std::str::FromStr;
use std::sync::Arc;
+use std::time::Duration;
use anyhow::bail;
use anyhow::Context;
@@ -34,6 +35,7 @@ use flags::EszipV2ChecksumKind;
use flags::OtelConsoleConfig;
use flags::OtelKind;
use log::warn;
+use tokio::time::timeout;
mod env;
mod flags;
@@ -319,6 +321,10 @@ fn main() -> Result {
} else {
vec![]
};
+ let timeout_dur = sub_matches
+ .get_one::("timeout")
+ .cloned()
+ .map(Duration::from_secs);
if import_map_path.is_some() {
warn!(concat!(
@@ -382,14 +388,24 @@ fn main() -> Result {
emitter_factory.set_deno_options(builder.build()?);
let mut metadata = Metadata::default();
- let eszip = generate_binary_eszip(
+ let eszip_fut = generate_binary_eszip(
&mut metadata,
Arc::new(emitter_factory),
None,
maybe_checksum_kind,
Some(static_patterns),
- )
- .await?;
+ );
+
+ let eszip = if let Some(dur) = timeout_dur {
+ match timeout(dur, eszip_fut).await {
+ Ok(eszip) => eszip,
+ Err(_) => {
+ bail!("Failed to complete the bundle within the given time.")
+ }
+ }
+ } else {
+ eszip_fut.await
+ }?;
let bin = eszip.into_bytes();
diff --git a/crates/base/README.md b/crates/base/README.md
new file mode 100644
index 000000000..a1374b042
--- /dev/null
+++ b/crates/base/README.md
@@ -0,0 +1,14 @@
+# Supabase EdgeRuntime base
+
+This crate is part of the Supabase Edge Runtime stack and implements the runtime
+core features.
+
+## Architecture
+
+
+
+
+
+
+
+
diff --git a/crates/base/src/runtime/mod.rs b/crates/base/src/runtime/mod.rs
index 0f3f4fdfc..3d41c7235 100644
--- a/crates/base/src/runtime/mod.rs
+++ b/crates/base/src/runtime/mod.rs
@@ -55,7 +55,6 @@ use deno_cache::SqliteBackedCache;
use deno_core::error::AnyError;
use deno_core::error::JsError;
use deno_core::serde_json;
-use deno_core::serde_json::Value;
use deno_core::url::Url;
use deno_core::v8;
use deno_core::v8::GCCallbackFlags;
@@ -80,7 +79,6 @@ use deno_facade::EmitterFactory;
use deno_facade::EszipPayloadKind;
use deno_facade::Metadata;
use either::Either;
-use ext_event_worker::events::EventMetadata;
use ext_event_worker::events::WorkerEventWithMetadata;
use ext_runtime::cert::ValueRootCertStoreProvider;
use ext_runtime::external_memory::CustomAllocator;
@@ -462,6 +460,7 @@ where
pub(crate) async fn new(mut worker: Worker) -> Result {
let init_opts = worker.init_opts.take();
let flags = worker.flags.clone();
+ let event_metadata = worker.event_metadata.clone();
debug_assert!(init_opts.is_some(), "init_opts must not be None");
@@ -658,6 +657,10 @@ where
base_url,
} = rt_provider;
+ let node_modules = metadata
+ .node_modules()
+ .ok()
+ .flatten();
let entrypoint = metadata.entrypoint.clone();
let main_module_url = match entrypoint.as_ref() {
Some(Entrypoint::Key(key)) => base_url.join(key)?,
@@ -717,6 +720,7 @@ where
let (fs, s3_fs) = build_file_system_fn(if is_user_worker {
Arc::new(StaticFs::new(
+ node_modules,
static_files,
if matches!(entrypoint, Some(Entrypoint::ModuleCode(_)) | None)
&& is_some_entry_point
@@ -1106,7 +1110,6 @@ where
s3_fs,
beforeunload_cpu_threshold,
beforeunload_mem_threshold,
- context,
..
} = match bootstrap_ret {
Ok(Ok(v)) => v,
@@ -1118,7 +1121,7 @@ where
}
};
- let context = context.unwrap_or_default();
+ let otel_attributes = event_metadata.otel_attributes.clone();
let span = Span::current();
let post_task_ret = unsafe {
spawn_blocking_non_send(|| {
@@ -1146,13 +1149,6 @@ where
op_state.put::>(HashMap::new());
}
- let mut otel_attributes = HashMap::new();
-
- otel_attributes.insert(
- "edge_runtime.worker.kind".into(),
- conf.to_worker_kind().to_string().into(),
- );
-
if conf.is_user_worker() {
let conf = conf.as_user_worker().unwrap();
let key = conf.key.map_or("".to_string(), |k| k.to_string());
@@ -1160,32 +1156,23 @@ where
// set execution id for user workers
env_vars.insert("SB_EXECUTION_ID".to_string(), key.clone());
- if let Some(Value::Object(attributes)) = context.get("otel") {
- for (k, v) in attributes {
- otel_attributes.insert(
- k.to_string().into(),
- match v {
- Value::String(str) => str.to_string().into(),
- others => others.to_string().into(),
- },
- );
- }
- }
-
if let Some(events_msg_tx) = conf.events_msg_tx.clone() {
op_state.put::>(
events_msg_tx,
);
- op_state.put::(EventMetadata {
- service_path: conf.service_path.clone(),
- execution_id: conf.key,
- });
+ op_state.put(event_metadata);
}
}
op_state.put(ext_env::EnvVars(env_vars));
op_state.put(DenoRuntimeDropToken(DropToken(drop_token.clone())));
- op_state.put(RuntimeOtelExtraAttributes(otel_attributes));
+ op_state.put(RuntimeOtelExtraAttributes(
+ otel_attributes
+ .unwrap_or_default()
+ .into_iter()
+ .map(|(k, v)| (k.into(), v.into()))
+ .collect(),
+ ));
}
if is_user_worker {
diff --git a/crates/base/src/utils/test_utils.rs b/crates/base/src/utils/test_utils.rs
index b6ce1aa58..499a85fee 100644
--- a/crates/base/src/utils/test_utils.rs
+++ b/crates/base/src/utils/test_utils.rs
@@ -271,6 +271,7 @@ impl TestBedBuilder {
}
}
+#[derive(Clone)]
pub struct TestBed {
pool_termination_token: TerminationToken,
main_termination_token: TerminationToken,
diff --git a/crates/base/src/worker/supervisor/strategy_per_worker.rs b/crates/base/src/worker/supervisor/strategy_per_worker.rs
index c952e4ecc..516e1882e 100644
--- a/crates/base/src/worker/supervisor/strategy_per_worker.rs
+++ b/crates/base/src/worker/supervisor/strategy_per_worker.rs
@@ -188,7 +188,7 @@ pub async fn supervise(args: Arguments) -> (ShutdownReason, i64) {
worker_timeout_ms
};
- let wall_clock_duration = Duration::from_millis(worker_timeout_ms);
+ let wall_clock_duration = Duration::from_millis(wall_clock_limit_ms);
// Split wall clock duration into 2 intervals.
// At the first interval, we will send a msg to retire the worker.
diff --git a/crates/base/src/worker/utils.rs b/crates/base/src/worker/utils.rs
index 20aba4b67..714119963 100644
--- a/crates/base/src/worker/utils.rs
+++ b/crates/base/src/worker/utils.rs
@@ -1,5 +1,8 @@
+use std::collections::HashMap;
+
use anyhow::anyhow;
use anyhow::bail;
+use deno_core::serde_json::Value;
use ext_event_worker::events::EventMetadata;
use ext_event_worker::events::WorkerEventWithMetadata;
use ext_event_worker::events::WorkerEvents;
@@ -15,18 +18,39 @@ use tokio::sync::oneshot;
use tokio_util::sync::CancellationToken;
pub fn get_event_metadata(conf: &WorkerRuntimeOpts) -> EventMetadata {
+ let mut otel_attributes = HashMap::new();
let mut event_metadata = EventMetadata {
service_path: None,
execution_id: None,
+ otel_attributes: None,
};
+
+ otel_attributes.insert(
+ "edge_runtime.worker.kind".to_string(),
+ conf.to_worker_kind().to_string(),
+ );
+
if conf.is_user_worker() {
let conf = conf.as_user_worker().unwrap();
- event_metadata = EventMetadata {
- service_path: conf.service_path.clone(),
- execution_id: conf.key,
- };
+ let context = conf.context.clone().unwrap_or_default();
+
+ event_metadata.service_path = conf.service_path.clone();
+ event_metadata.execution_id = conf.key;
+
+ if let Some(Value::Object(attributes)) = context.get("otel") {
+ for (k, v) in attributes {
+ otel_attributes.insert(
+ k.to_string(),
+ match v {
+ Value::String(str) => str.to_string(),
+ others => others.to_string(),
+ },
+ );
+ }
+ }
}
+ event_metadata.otel_attributes = Some(otel_attributes);
event_metadata
}
diff --git a/crates/base/src/worker/worker_inner.rs b/crates/base/src/worker/worker_inner.rs
index e4f22274b..a63450c5b 100644
--- a/crates/base/src/worker/worker_inner.rs
+++ b/crates/base/src/worker/worker_inner.rs
@@ -273,6 +273,12 @@ impl Worker {
let _ =
booter_signal.send(Ok((metric_src, new_runtime.drop_token.clone())));
+
+ let span = debug_span!(
+ "poll",
+ thread = ?std::thread::current().id(),
+ );
+
let supervise_fut = match imp.clone().supervise(&mut new_runtime) {
Some(v) => v.boxed(),
None if worker_kind.is_user_worker() => return None,
@@ -290,28 +296,41 @@ impl Worker {
}
});
- let result = imp.on_created(&mut new_runtime).await;
- let maybe_uncaught_exception_event = match result.as_ref() {
- Ok(WorkerEvents::UncaughtException(ev)) => Some(ev.clone()),
- Err(err) => Some(UncaughtExceptionEvent {
- cpu_time_used: 0,
- exception: err.to_string(),
- }),
+ let worker_poll_fut = async move {
+ let result = imp.on_created(&mut new_runtime).await;
+ let maybe_uncaught_exception_event = match result.as_ref() {
+ Ok(WorkerEvents::UncaughtException(ev)) => Some(ev.clone()),
+ Err(err) => Some(UncaughtExceptionEvent {
+ cpu_time_used: 0,
+ exception: err.to_string(),
+ }),
- _ => None,
- };
+ _ => None,
+ };
- if let Some(ev) = maybe_uncaught_exception_event {
- exit.set(WorkerExitStatus::WithUncaughtException(ev)).await;
- }
+ if let Some(ev) = maybe_uncaught_exception_event {
+ exit.set(WorkerExitStatus::WithUncaughtException(ev)).await;
+ }
- drop(new_runtime);
- let _ = supervise_fut.await;
+ drop(new_runtime);
+ let _ = supervise_fut.await;
- Some(result)
+ result
+ }
+ .instrument(span);
+
+ Some(
+ rt.spawn_pinned({
+ let fut = unsafe { MaskFutureAsSend::new(worker_poll_fut) };
+ move || tokio::task::spawn_local(fut)
+ })
+ .await
+ .map_err(anyhow::Error::from)
+ .and_then(|it| it.map_err(anyhow::Error::from))
+ .and_then(|it| it.into_inner()),
+ )
};
-
- let worker_fut = {
+ let worker_result_fut = {
let event_metadata = event_metadata.clone();
async move {
let Some(result) = worker_fut.await else {
@@ -349,13 +368,11 @@ impl Worker {
"worker",
id = worker_name.as_str(),
kind = %worker_kind,
- thread = ?std::thread::current().id(),
metadata = ?event_metadata
));
- drop(rt.spawn_pinned({
- let worker_fut = unsafe { MaskFutureAsSend::new(worker_fut) };
- move || tokio::task::spawn_local(worker_fut)
+ drop(tokio::spawn(unsafe {
+ MaskFutureAsSend::new(worker_result_fut)
}));
}
}
diff --git a/crates/base/test_cases/issue-func-284/baseline/index.ts b/crates/base/test_cases/issue-func-284/baseline/index.ts
new file mode 100644
index 000000000..320bacd09
--- /dev/null
+++ b/crates/base/test_cases/issue-func-284/baseline/index.ts
@@ -0,0 +1,3 @@
+Deno.serve(() => {
+ return new Response("meow");
+});
diff --git a/crates/base/test_cases/issue-func-284/noisy/index.ts b/crates/base/test_cases/issue-func-284/noisy/index.ts
new file mode 100644
index 000000000..3e04f3457
--- /dev/null
+++ b/crates/base/test_cases/issue-func-284/noisy/index.ts
@@ -0,0 +1,19 @@
+function mySlowFunction(baseNumber) {
+ console.time("mySlowFunction");
+ let now = Date.now();
+ let result = 0;
+ for (var i = Math.pow(baseNumber, 7); i >= 0; i--) {
+ result += Math.atan(i) * Math.tan(i);
+ }
+ let duration = Date.now() - now;
+ console.timeEnd("mySlowFunction");
+ return { result: result, duration: duration };
+}
+
+Deno.serve(async () => {
+ let count = 0;
+ while (++count != 300) {
+ mySlowFunction(8);
+ }
+ return new Response("meow");
+});
diff --git a/crates/base/test_cases/user-worker-san-check/.blocklisted b/crates/base/test_cases/user-worker-san-check/.blocklisted
index 398109c64..7aa5c3a90 100644
--- a/crates/base/test_cases/user-worker-san-check/.blocklisted
+++ b/crates/base/test_cases/user-worker-san-check/.blocklisted
@@ -21,8 +21,6 @@ futime
futimeSync
link
linkSync
-makeTempFile
-makeTempFileSync
readLink
readLinkSync
rename
diff --git a/crates/base/test_cases/user-worker-san-check/.whitelisted b/crates/base/test_cases/user-worker-san-check/.whitelisted
index a3a8afc60..299dd96f0 100644
--- a/crates/base/test_cases/user-worker-san-check/.whitelisted
+++ b/crates/base/test_cases/user-worker-san-check/.whitelisted
@@ -8,3 +8,4 @@ writeTextFileSync
readTextFileSync
readFileSync
makeTempDirSync
+makeTempFileSync
diff --git a/crates/base/tests/integration_tests.rs b/crates/base/tests/integration_tests.rs
index fc461f708..2178d84cb 100644
--- a/crates/base/tests/integration_tests.rs
+++ b/crates/base/tests/integration_tests.rs
@@ -44,6 +44,7 @@ use deno_facade::EszipPayloadKind;
use deno_facade::Metadata;
use ext_event_worker::events::LogLevel;
use ext_event_worker::events::ShutdownReason;
+use ext_event_worker::events::WorkerEventWithMetadata;
use ext_event_worker::events::WorkerEvents;
use ext_runtime::SharedMetricSource;
use ext_workers::context::MainWorkerRuntimeOpts;
@@ -2606,6 +2607,70 @@ async fn test_issue_func_280() {
run("mem", ShutdownReason::Memory).await;
}
+#[tokio::test]
+#[serial]
+async fn test_issue_func_284() {
+ async fn find_boot_event(
+ rx: &mut mpsc::UnboundedReceiver,
+ ) -> Option {
+ while let Some(ev) = rx.recv().await {
+ match ev.event {
+ WorkerEvents::Boot(ev) => return Some(ev.boot_time),
+ _ => continue,
+ }
+ }
+
+ None
+ }
+
+ let (tx, mut rx) = mpsc::unbounded_channel();
+ let tb = TestBedBuilder::new("./test_cases/main")
+ .with_per_worker_policy(None)
+ .with_worker_event_sender(Some(tx))
+ .build()
+ .await;
+
+ tokio::spawn({
+ let tb = tb.clone();
+ async move {
+ tb.request(|b| {
+ b.uri("/meow")
+ .header("x-service-path", "issue-func-284/noisy")
+ .body(Body::empty())
+ .context("can't make request")
+ })
+ .await
+ .unwrap();
+ }
+ });
+
+ timeout(Duration::from_secs(1), find_boot_event(&mut rx))
+ .await
+ .unwrap()
+ .unwrap();
+
+ tokio::spawn({
+ let tb = tb.clone();
+ async move {
+ tb.request(|b| {
+ b.uri("/meow")
+ .header("x-service-path", "issue-func-284/baseline")
+ .body(Body::empty())
+ .context("can't make request")
+ })
+ .await
+ .unwrap();
+ }
+ });
+
+ let boot_time = timeout(Duration::from_secs(1), find_boot_event(&mut rx))
+ .await
+ .unwrap()
+ .unwrap();
+
+ assert!(boot_time < 1000);
+}
+
#[tokio::test]
#[serial]
async fn test_should_render_detailed_failed_to_create_graph_error() {
diff --git a/crates/cpu_timer/src/lib.rs b/crates/cpu_timer/src/lib.rs
index 0be14d5e5..7762362e7 100644
--- a/crates/cpu_timer/src/lib.rs
+++ b/crates/cpu_timer/src/lib.rs
@@ -42,6 +42,7 @@ mod linux {
#[repr(C)]
#[derive(Clone, Default)]
pub struct CPUAlarmVal {
+ #[cfg(target_os = "linux")]
pub cpu_alarms_tx: Arc>>>,
}
@@ -131,9 +132,12 @@ impl CPUTimer {
pub async fn set_channel(&self) -> mpsc::UnboundedReceiver<()> {
let (tx, rx) = mpsc::unbounded_channel();
- let mut val = self.cpu_alarm_val.cpu_alarms_tx.lock().await;
+ #[cfg(target_os = "linux")]
+ {
+ let mut val = self.cpu_alarm_val.cpu_alarms_tx.lock().await;
+ *val = Some(tx);
+ }
- *val = Some(tx);
rx
}
@@ -176,11 +180,6 @@ impl CPUTimer {
log::error!("CPU timer: not enabled (need Linux)");
Ok(Self {})
}
-
- #[cfg(not(target_os = "linux"))]
- pub fn reset(&self) -> Result<(), Error> {
- Ok(())
- }
}
pub fn get_thread_time() -> Result {
diff --git a/crates/deno_facade/eszip/mod.rs b/crates/deno_facade/eszip/mod.rs
index 134830aa6..fc9066851 100644
--- a/crates/deno_facade/eszip/mod.rs
+++ b/crates/deno_facade/eszip/mod.rs
@@ -52,6 +52,7 @@ use futures::AsyncReadExt;
use futures::AsyncSeekExt;
use futures::FutureExt;
use glob::glob;
+use indexmap::IndexMap;
use once_cell::sync::Lazy;
use regex::Regex;
use scopeguard::ScopeGuard;
@@ -773,7 +774,7 @@ pub async fn generate_binary_eszip(
let root_dir_url = EszipRelativeFileBaseUrl::new(&root_dir_url);
let root_path = root_dir_url.inner().to_file_path().unwrap();
- let mut contents = HashMap::new();
+ let mut contents = IndexMap::new();
let mut vfs_count = 0;
let mut vfs_content_callback_fn = |_path: &_, _key: &_, content: Vec| {
let key = format!("vfs://{}", vfs_count);
diff --git a/crates/fs/impl/static_fs.rs b/crates/fs/impl/static_fs.rs
index ccbaad997..0580ca61d 100644
--- a/crates/fs/impl/static_fs.rs
+++ b/crates/fs/impl/static_fs.rs
@@ -6,6 +6,7 @@ use std::path::PathBuf;
use std::rc::Rc;
use std::sync::Arc;
+use deno::standalone::binary::NodeModules;
use deno_core::normalize_path;
use deno_fs::AccessCheckCb;
use deno_fs::FsDirEntry;
@@ -26,22 +27,34 @@ pub struct StaticFs {
static_files: EszipStaticFiles,
base_dir_path: PathBuf,
vfs_path: PathBuf,
+ byonm_node_modules_path: Option,
snapshot: Option,
vfs: Arc,
}
impl StaticFs {
pub fn new(
+ node_modules: Option,
static_files: EszipStaticFiles,
base_dir_path: PathBuf,
vfs_path: PathBuf,
vfs: Arc,
snapshot: Option,
) -> Self {
+ let byonm_node_modules_path = if let Some(NodeModules::Byonm {
+ root_node_modules_dir: Some(path),
+ }) = node_modules
+ {
+ Some(vfs_path.join(path))
+ } else {
+ None
+ };
+
Self {
vfs,
static_files,
base_dir_path,
+ byonm_node_modules_path,
vfs_path,
snapshot,
}
@@ -375,7 +388,13 @@ impl deno_fs::FileSystem for StaticFs {
_access_check: Option,
) -> FsResult> {
let is_npm = self.is_valid_npm_package(path);
- if is_npm {
+ let is_byonm_path = self
+ .byonm_node_modules_path
+ .as_ref()
+ .map(|it| path.starts_with(it))
+ .unwrap_or_default();
+
+ if is_npm || is_byonm_path {
let options = OpenOptions::read();
let file = self.open_sync(path, options, None)?;
let buf = file.read_all_sync()?;
diff --git a/deno/standalone/binary.rs b/deno/standalone/binary.rs
index a18be402b..b0e41f748 100644
--- a/deno/standalone/binary.rs
+++ b/deno/standalone/binary.rs
@@ -9,7 +9,7 @@ use indexmap::IndexMap;
use serde::Deserialize;
use serde::Serialize;
-#[derive(Deserialize, Serialize)]
+#[derive(Debug, Clone, Deserialize, Serialize)]
pub enum NodeModules {
Managed {
/// Relative path for the node_modules directory in the vfs.
diff --git a/docs/usage.md b/docs/usage.md
new file mode 100644
index 000000000..aad38d922
--- /dev/null
+++ b/docs/usage.md
@@ -0,0 +1,314 @@
+# Usage & Self-Hosting
+
+## The edge-runtime CLI
+
+Edge runtime is based on [Deno](https://deno.land) and try to follows the same
+concepts of it. One of them is the edge-runtime is primary a CLI based app and
+have different commands as well parameters.
+
+The easiest way to use edge-runtime is by running it from docker:
+
+```bash
+docker run --rm -it supabase/edge-runtime:v1.69.9
+```
+
+> The version above may be outdated.
+
+
+ CLI output
+
+```
+A server based on Deno runtime, capable of running JavaScript, TypeScript, and WASM services
+
+Usage: edge-runtime [OPTIONS] [COMMAND]
+
+Commands:
+start Start the server
+bundle Creates an 'eszip' file that can be executed by the EdgeRuntime. Such file contains all the modules in contained in a single binary.
+unbundle Unbundles an .eszip file into the specified directory
+help Print this message or the help of the given subcommand(s)
+
+Options:
+-v, --verbose Use verbose output
+-q, --quiet Do not print any log messages
+ --log-source Include source file and line in log messages
+-h, --help Print help
+-V, --version Print version
+```
+
+
+
+### `start` command
+
+The start command allows you to run JavaScript/TypeScript code in a similar way of standard Deno.
+But with the difference that it expects a main-service entrypoint with the given format: `path/index.ts`
+
+
+ Example
+
+```ts
+// main.ts
+
+console.log('Hello from Edge Runtime!!');
+```
+
+Running it from docker by using `--main-service` parameter:
+
+```bash
+docker run --rm -it -v $(pwd)/main.ts:/home/deno/main/index.ts supabase/edge-runtime:v1.69.9 start --main-service /home/deno/main
+```
+
+In the command above we did first map our local `main.ts` script to an `index.ts` file located at `/home/deno/main` in the docker volume.
+So that we only need to supply this path as "main-service" entrypoint.
+
+Edge runtime will then run the script and print the following output:
+
+```
+Hello from Edge Runtime!!
+main worker has been destroyed
+```
+
+Notice that a *"main worker has been destroyed"* was printed out.
+It means that our main service worker has nothing more to execute so the process will be finished.
+
+
+
+## Edge functions
+
+In the previous section we discussed in how `edge-runtime` cli can be used to run a JavaScript/TypeScript code.
+
+**But how about serving edge functions?** In order to achieve that we must first understand the edge-runtime execution flow.
+
+
+
+
+
+
+
+
+
+### Main Service
+
+The main service is the initial script supplied on `start` command and it's should acts as root level of edge functions. By intercepting the incomming http request and spawing an **User Worker** to handle it.
+
+All code execution here is more permisse, it means that main scope can access filesystem as well other privilegied APIs - like a `sudo` mode!
+
+> When using Supabase Hosting Platform you don't have access to **Main Service** since its implicit managed by Supabase team.
+
+### User Worker
+
+Here's where your edge functions will really be executed!
+
+The user worker is a more restricted and isolated environment, great to put user-specific code.
+
+> When using `supabase functions deploy`, in a Supabase Hosting Platform, all your function code will be executed inside an **User Worker**.
+
+### API Comparison
+
+
+ Deno APIs
+
+
+
+| API | Main Worker | User Worker |
+|---|---|---|
+| [`cloud`](https://docs.deno.com/api/deno/cloud) | ❌ | ❌ |
+| [`fetch`](https://docs.deno.com/api/deno/fetch) | ✅ | ✅ |
+| [`ffi`](https://docs.deno.com/api/deno/ffi) | ❌ | ❌ |
+| [`file-system`](https://docs.deno.com/api/deno/file-system) | ✅ | ❌ |
+| [`gpu`](https://docs.deno.com/api/deno/gpu) | ❌ | ❌ |
+| [`http-server`](https://docs.deno.com/api/deno/http-server) | ✅ | ✅ |
+| [`io`](https://docs.deno.com/api/deno/io) | ❌ | ❌ |
+| [`jupyter`](https://docs.deno.com/api/deno/jupyter) | ❌ | ❌ |
+| [`linter`](https://docs.deno.com/api/deno/linter) | ❌ | ❌ |
+| [`network`](https://docs.deno.com/api/deno/network) | ✅ | ✅ |
+| [`permissions`](https://docs.deno.com/api/deno/permissions) | ✅ | ✅ |
+| [`runtime`](https://docs.deno.com/api/deno/runtime) | ⚠️ | ⚠️ |
+| [`subprocess`](https://docs.deno.com/api/deno/subprocess) | ❌ | ❌ |
+| [`telemetry`](https://docs.deno.com/api/deno/telemetry) | ✅ | ✅ |
+| [`testing`](https://docs.deno.com/api/deno/testing) | ✅ | ✅ |
+| [`websockets`](https://docs.deno.com/api/deno/websockets) | ✅ | ✅ |
+
+> ❌ Not supported
+> ✅ Supported
+> ⚠️Partial supported
+
+
+
+
+ Web APIs
+
+
+
+| API | Main Worker | User Worker |
+|---|---|---|
+| [`cache`](https://docs.deno.com/api/web/cache) | ❌ | ⚠️ **ai models** only |
+| [`canvas`](https://docs.deno.com/api/web/canvas) | ❌ | ❌ |
+| [`crypto`](https://docs.deno.com/api/web/crypto) | ✅ | ✅ |
+| [`encoding`](https://docs.deno.com/api/web/encoding) | ✅ | ✅ |
+| [`events`](https://docs.deno.com/api/web/events) | ✅ | ✅ |
+| [`events`](https://docs.deno.com/api/web/events) | ✅ | ✅ |
+| [`fetch`](https://docs.deno.com/api/web/fetch) | ✅ | ✅ |
+| [`file`](https://docs.deno.com/api/web/file) | ✅ | ✅ |
+| [`gpu`](https://docs.deno.com/api/web/gpu) | ❌ | ❌ |
+| [`io`](https://docs.deno.com/api/web/io) | ✅ | ✅ |
+| [`io`](https://docs.deno.com/api/web/io) | ✅ | ✅ |
+| [`intl`](https://docs.deno.com/api/web/intl) | ✅ | ✅ |
+| [`messaging`](https://docs.deno.com/api/web/messaging) | ✅ | ✅ |
+| [`performance`](https://docs.deno.com/api/web/performance) | ✅ | ✅ |
+| [`platform`](https://docs.deno.com/api/web/platform) | ❌ | ❌ |
+| [`storage`](https://docs.deno.com/api/web/storage) | ❌ | ❌ |
+| [`streams`](https://docs.deno.com/api/web/streams) | ✅ | ✅ |
+| [`temporal`](https://docs.deno.com/api/web/temporal) | ✅ | ✅ |
+| [`url`](https://docs.deno.com/api/web/url) | ✅ | ✅ |
+| [`wasm`](https://docs.deno.com/api/web/wasm) | ✅ | ✅ |
+| [`websockets`](https://docs.deno.com/api/web/websockets) | ✅ | ✅ |
+| [`workers`](https://docs.deno.com/api/web/storage) | ❌ | ❌ |
+
+> ❌ Not supported
+> ✅ Supported
+> ⚠️Partial supported
+
+
+
+
+ Node APIs
+
+
+
+| API | Main Worker | User Worker |
+|---|---|---|
+| [`assert`](https://docs.deno.com/api/node/assert) | ✅ | ✅ |
+| [`assert/strict`](https://docs.deno.com/api/node/assert/~/assert.strict) | ✅ | ✅ |
+| [`assync_hooks`](https://docs.deno.com/api/node/assync_hooks) | ✅ | ✅ |
+| [`buffer`](https://docs.deno.com/api/node/buffer) | ✅ | ✅ |
+| [`child_process`](https://docs.deno.com/api/node/child_process) | ❌ | ❌ |
+| [`cluster`](https://docs.deno.com/api/node/cluster) | ❌ | ❌ |
+| [`console`](https://docs.deno.com/api/node/console) | ⚠️all outputs are trimmed out | ⚠️ same as *Main Worker* |
+| [`crypto`](https://docs.deno.com/api/node/crypto) | ✅ | ✅ |
+| [`dgram`](https://docs.deno.com/api/node/dgram) | ✅ | ✅ |
+| [`diagnostics_channel`](https://docs.deno.com/api/node/diagnostics_channel) | ❌ | ❌ |
+| [`dns`](https://docs.deno.com/api/node/dns) | ✅ | ✅ |
+| [`dns/promisses`](https://docs.deno.com/api/node/dns/promisses) | ✅ | ✅ |
+| [`domain`](https://docs.deno.com/api/node/domain) | ✅ | ✅ |
+| [`events`](https://docs.deno.com/api/node/events) | ✅ | ✅ |
+| [`fs`](https://docs.deno.com/api/node/fs) | ✅ | ❌ |
+| [`fs/promisses`](https://docs.deno.com/api/node/fs/promisses) | ✅ | ❌ |
+| [`http`](https://docs.deno.com/api/node/http) | ✅ | ✅ |
+| [`http2`](https://docs.deno.com/api/node/http2) | ✅ | ✅ |
+| [`https`](https://docs.deno.com/api/node/https) | ✅ | ✅ |
+| [`inspector`](https://docs.deno.com/api/node/inspector) | ✅ | ✅ |
+| [`inspector/promisses`](https://docs.deno.com/api/node/inspector/promisses) | ✅ | ✅ |
+| [`module`](https://docs.deno.com/api/node/module) | ✅ | ✅ |
+| [`net`](https://docs.deno.com/api/node/net) | ✅ | ✅ |
+| [`os`](https://docs.deno.com/api/node/os) | ✅ | ✅ |
+| [`path`](https://docs.deno.com/api/node/path) | ✅ | ✅ |
+| [`perf_hooks`](https://docs.deno.com/api/node/perf_hooks) | ✅ | ✅ |
+| [`process`](https://docs.deno.com/api/node/process) | ✅ | ✅ |
+| [`querystring`](https://docs.deno.com/api/node/querystring) | ✅ | ✅ |
+| [`readline`](https://docs.deno.com/api/node/readline) | ❌ | ❌ |
+| [`readline/promisses`](https://docs.deno.com/api/node/readline/promisses) | ❌ | ❌ |
+| [`repl`](https://docs.deno.com/api/node/repl) | ❌ | ❌ |
+| [`sea`](https://docs.deno.com/api/node/sea) | ❌ | ❌ |
+| [`sqlite`](https://docs.deno.com/api/node/sqlite) | ❌ | ❌ |
+| [`stream`](https://docs.deno.com/api/node/stream) | ✅ | ✅ |
+| [`stream/consumers`](https://docs.deno.com/api/node/stream/consumers) | ✅ | ✅ |
+| [`stream/promisses`](https://docs.deno.com/api/node/stream/promisses) | ✅ | ✅ |
+| [`stream/web`](https://docs.deno.com/api/node/stream/web) | ✅ | ✅ |
+| [`string_decoder`](https://docs.deno.com/api/node/string_decoder) | ✅ | ✅ |
+| [`test`](https://docs.deno.com/api/node/test) | ✅ | ✅ |
+| [`test/reporters`](https://docs.deno.com/api/node/test/reporters) | ✅ | ✅ |
+| [`timers`](https://docs.deno.com/api/node/timers) | ✅ | ✅ |
+| [`timers/promisses`](https://docs.deno.com/api/node/timers/promisses) | ✅ | ✅ |
+| [`tls`](https://docs.deno.com/api/node/tls) | ✅ | ✅ |
+| [`trace_events`](https://docs.deno.com/api/node/trace_events) | ✅ | ✅ |
+| [`tty`](https://docs.deno.com/api/node/tty) | ❌ | ❌ |
+| [`url`](https://docs.deno.com/api/node/url) | ✅ | ✅ |
+| [`util`](https://docs.deno.com/api/node/util) | ✅ | ✅ |
+| [`util/types`](https://docs.deno.com/api/node/util/types) | ✅ | ✅ |
+| [`v8`](https://docs.deno.com/api/node/v8) | ❌ | ❌ |
+| [`vm`](https://docs.deno.com/api/node/vm) | ❌ | ❌ |
+| [`wasi`](https://docs.deno.com/api/node/wasi) | ✅ | ✅ |
+| [`worker_threads`](https://docs.deno.com/api/node/worker_threads) | ✅ | ✅ |
+| [`zlib`](https://docs.deno.com/api/node/zlib) | ✅ | ✅ |
+
+> ❌ Not supported
+> ✅ Supported
+> ⚠️Partial supported
+
+
+
+### Self-Hosting
+
+To self-host edge-functions you should manage the main service as well the user worker spawn logic, like [main.ts template](https://github.com/supabase/supabase/blob/d91ea9d4e24c211f666e6e0ff01d290a9f3831cb/docker/volumes/functions/main/index.ts)
+
+The core idea is to have a root level `Deno.serve()` inside the main service and then foward the request to an user worker.
+
+
+ Example
+
+Creating a edge function to say hello!
+
+```ts
+// functions/hello-world/index.ts
+
+Deno.serve(async (req: Request) => {
+ const { name } = await req.json();
+
+ const message = `Hello ${name} from foo!`;
+
+ return Response.json({ message });
+});
+```
+
+Handling http requests at main service level and passing it to a user worker:
+
+```ts
+// main/index.ts
+
+import { exists } from 'jsr:@std/fs/exists';
+
+Deno.serve(async (req: Request) => {
+ console.log('new request', req.url); // http:localhost:9000/hello-world
+
+ const edgeFunctionName = new URL(req.url).pathname; // "hello-world"
+
+ // path relative to docker volume
+ const edgeFunctionFilepath = `/home/deno/functions/${edgeFunctionName}`;
+
+ // ensuring file exists
+ if (!await exists(edgeFunctionFilepath)) {
+ return new Response(null, { status: 404 });
+ }
+
+ try {
+ // spawning a user worker
+ const worker = await EdgeRuntime.userWorkers.create({
+ servicePath: edgeFunctionFilepath,
+ memoryLimitMb: 150,
+ workerTimeoutMs: 1 * 60 * 1000,
+ });
+
+ // fowarding the request to user worker
+ return await worker.fetch(req);
+ } catch (error) {
+ return Response.json({ error }, { status: 500 });
+ }
+});
+```
+
+Executing with docker
+
+```bash
+docker run --rm -it -p 9000:9000 -v $(pwd):/home/deno supabase/edge-runtime:v1.69.9 start --main-service /home/deno/main
+```
+
+Calling the edge function
+
+```bash
+$ curl localhost:9000/hello-world --data '{"name": "Kalleby Santos"}'
+
+{"message":"Hello Kalleby Santos from foo!"}
+```
+
+
diff --git a/ext/ai/README.md b/ext/ai/README.md
new file mode 100644
index 000000000..7633f327e
--- /dev/null
+++ b/ext/ai/README.md
@@ -0,0 +1,105 @@
+# Supabase AI module
+
+This crate is part of the Supabase Edge Runtime stack and implements AI related
+features for the `Supabase.ai` namespace.
+
+## Model Execution Engine
+
+
+
+
+
+
+
+
+
+`Supabase.ai` uses [onnxruntime](https://onnxruntime.ai/) as internal model
+execution engine, backend by [ort pyke](https://ort.pyke.io/) rust bindings.
+
+The **onnxruntime** API is available from `globalThis` and shares similar specs of [onnxruntime-common](https://github.com/microsoft/onnxruntime/tree/main/js/common).
+
+The available items are:
+
+- `Tensor`: Represent a basic tensor with specified dimensions and data type. - "The AI input/output"
+- `InferenceSession`: Represent the inner model session. - "The AI model itself"
+
+
+Usage
+
+It can be used from the exported `globalThis[Symbol.for("onnxruntime")]` -
+but manipulating it directly is not trivial, so in the future you may use the [Inference API #501](https://github.com/supabase/edge-runtime/pull/501) for a more user friendly API.
+
+```typescript
+const { InferenceSession, Tensor } = globalThis[Symbol.for("onnxruntime")];
+
+// 'create()' supports an url string buffer or the binary data
+const modelUrlBuffer = new TextEncoder().encode("https://huggingface.co/Supabase/gte-small/resolve/main/onnx/model_quantized.onnx");
+const session = await InferenceSession.create(modelUrlBuffer);
+
+// Example only, in real 'feature-extraction' tensors must be created from the tokenizer step.
+const inputs = {
+ input_ids: new Tensor('float32', [1, 2, 3...], [1, 384]),
+ attention_mask: new Tensor('float32', [...], [1, 384]),
+ token_types_ids: new Tensor('float32', [...], [1, 384])
+};
+
+const { last_hidden_state } = await session.run(inputs);
+console.log(last_hidden_state);
+```
+
+
+
+### Third party libs
+
+Originaly this backend was created to implicit integrate with [transformers.js](https://github.com/huggingface/transformers.js/). This way users can still consuming a high-level lib at same time they benefits of all Supabase's Model Execution Engine features, like model optimization and caching.
+For further information please check the [PR #436](https://github.com/supabase/edge-runtime/pull/436) as well the [tests folder](/crates/base/test_cases/ai-ort-rust-backend/transformers-js)
+
+> [!WARNING]
+> At this moment users need to explicit target `device: 'auto'` to enable the platform compatibility.
+
+```typescript
+import { env, pipeline } from 'https://cdn.jsdelivr.net/npm/@huggingface/transformers@3.0.1';
+
+// Broswer cache is now supported for `onnx` models
+env.useBrowserCache = true;
+env.allowLocalModels = false;
+
+const pipe = await pipeline('feature-extraction', 'supabase/gte-small', { device: 'auto' });
+
+const output = await pipe("This embed will be generated from rust land", {
+ pooling: 'mean',
+ normalize: true
+});
+```
+
+### Self-Hosting
+
+**Caching filepath**:
+The `EXT_AI_CACHE_DIR` environment variable can be use to set a custom cache path
+
+**Memory clean up**:
+For Self-Hosting users an extra method is available for `main/index.ts` scope and should be used to clean up unused sessions, consider adding it into your main entrypoint file:
+
+```typescript
+// cleanup unused sessions every 30s
+setInterval(async () => {
+ try {
+ const cleanupCount = await EdgeRuntime.ai.tryCleanupUnusedSession();
+ if (cleanupCount == 0) {
+ return;
+ }
+ console.log('EdgeRuntime.ai.tryCleanupUnusedSession', cleanupCount);
+ } catch (e) {
+ console.error(e.toString());
+ }
+}, 30 * 1000);
+```
+
+## The `Session` class
+
+Prior versions has [introduced](https://supabase.com/blog/ai-inference-now-available-in-supabase-edge-functions) the `Session` class as alternative to `transformers.js` for *gte-small* model and then was used to provide a [LLM interface](https://supabase.com/docs/guides/functions/ai-models?queryGroups=platform&platform=ollama#using-large-language-models-llm) for Ollama and some other providers.
+
+Since the **Model Execution Engine** was created the `Session` class now can focus on LLM interface while the `Session('gte-small')` is for compatibility purposes only.
+
+> [!WARNING]
+> Docs for Session class will end here - There's a open [PR #539](https://github.com/supabase/edge-runtime/pull/539) that may change a lot of things for it.
diff --git a/ext/event_worker/events.rs b/ext/event_worker/events.rs
index 70a5c25b2..9706aef04 100644
--- a/ext/event_worker/events.rs
+++ b/ext/event_worker/events.rs
@@ -1,3 +1,6 @@
+use std::collections::HashMap;
+use std::convert::Infallible;
+
use base_mem_check::MemCheckState;
use enum_as_inner::EnumAsInner;
use serde::Deserialize;
@@ -50,14 +53,30 @@ pub struct LogEvent {
pub level: LogLevel,
}
-#[derive(Serialize, Deserialize, Debug, PartialEq, Eq)]
+#[derive(Serialize, Deserialize, Debug, Default, PartialEq, Eq)]
+#[repr(u8)]
pub enum LogLevel {
+ #[default]
Debug,
Info,
Warning,
Error,
}
+impl TryFrom for LogLevel {
+ type Error = Infallible;
+
+ fn try_from(value: u8) -> Result {
+ match value {
+ 0 => Ok(Self::Debug),
+ 1 => Ok(Self::Info),
+ 2 => Ok(Self::Warning),
+ 3 => Ok(Self::Error),
+ _ => Ok(Self::Debug),
+ }
+ }
+}
+
#[derive(Serialize, Deserialize, Debug, EnumAsInner)]
pub enum WorkerEvents {
Boot(BootEvent),
@@ -88,6 +107,7 @@ impl WorkerEvents {
pub struct EventMetadata {
pub service_path: Option,
pub execution_id: Option,
+ pub otel_attributes: Option>,
}
#[derive(Serialize, Deserialize, Debug)]
diff --git a/ext/event_worker/js_interceptors.rs b/ext/event_worker/js_interceptors.rs
index 9a967a06c..bdc66b3d6 100644
--- a/ext/event_worker/js_interceptors.rs
+++ b/ext/event_worker/js_interceptors.rs
@@ -12,14 +12,11 @@ use tokio::sync::mpsc;
fn op_user_worker_log(
state: &mut OpState,
#[string] msg: &str,
- is_err: bool,
+ #[smi] level: i32,
) -> Result<(), AnyError> {
let maybe_tx =
state.try_borrow::>();
- let mut level = LogLevel::Info;
- if is_err {
- level = LogLevel::Error;
- }
+ let level = LogLevel::try_from(level as u8).unwrap_or_default();
if let Some(tx) = maybe_tx {
let event_metadata = state
diff --git a/ext/node/README.md b/ext/node/README.md
index d154d8cb6..87d08a664 100644
--- a/ext/node/README.md
+++ b/ext/node/README.md
@@ -1,3 +1,11 @@
-# deno_node
+# Supabase Node module
+
+This crate is part of the Supabase Edge Runtime stack and implements NodeJs
+related features.
+
+To see all compatible features, please check the
+[NodeJs Polyfills](/ext/node/polyfills/README.md) section.
+
+## deno_node
`require` and other node related functionality for Deno.
diff --git a/ext/node/polyfills/README.md b/ext/node/polyfills/README.md
index 26527278e..1e6e82bdc 100644
--- a/ext/node/polyfills/README.md
+++ b/ext/node/polyfills/README.md
@@ -1,6 +1,7 @@
-# Deno Node.js compatibility
+# Supabase Node.js compatibility module
-This module is meant to have a compatibility layer for the
+This crate is part of the Supabase Edge Runtime stack and implements a
+compatibility layer for the
[Node.js standard library](https://nodejs.org/docs/latest/api/).
**Warning**: Any function of this module should not be referred anywhere in the
@@ -59,7 +60,7 @@ Deno standard library as it's a compatibility module.
- [x] worker_threads
- [ ] zlib
-* [x] node globals _partly_
+- [x] node globals _partly_
### Deprecated
diff --git a/ext/runtime/js/bootstrap.js b/ext/runtime/js/bootstrap.js
index 5dc96bc3d..a4f6adcdd 100644
--- a/ext/runtime/js/bootstrap.js
+++ b/ext/runtime/js/bootstrap.js
@@ -595,7 +595,7 @@ globalThis.bootstrapSBEdge = (opts, ctx) => {
ObjectDefineProperties(globalThis, {
console: nonEnumerable(
new console.Console((msg, level) => {
- return ops.op_user_worker_log(msg, level > 1);
+ return ops.op_user_worker_log(msg, level);
}),
),
});
@@ -680,6 +680,7 @@ globalThis.bootstrapSBEdge = (opts, ctx) => {
"readTextFile": true,
"mkdir": true,
"makeTempDir": true,
+ "makeTempFile": true,
"readDir": true,
"kill": "mock",
@@ -696,6 +697,7 @@ globalThis.bootstrapSBEdge = (opts, ctx) => {
"readTextFileSync": "allowIfRuntimeIsInInit",
"mkdirSync": "allowIfRuntimeIsInInit",
"makeTempDirSync": "allowIfRuntimeIsInInit",
+ "makeTempFileSync": "allowIfRuntimeIsInInit",
"readDirSync": "allowIfRuntimeIsInInit",
// TODO: use a non-hardcoded path