Browse Source

Restore SSG serve logic (#4111)

* Build ssg routes when both the client and server builds are finished

* Fix default server fn encoding

* fix tokio console

* fix ssg

* add prerendering message

* render under the same environment variables as normal

* fix check

* use should_bundle_to_asset

* enable ssg playwright test

* fix clippy
Evan Almloff 4 days ago
parent
commit
fdd77d79c1

+ 7 - 0
packages/cli-opt/src/lib.rs

@@ -65,6 +65,13 @@ impl AssetManifest {
         self.assets.get(path)
     }
 
+    /// Get the first asset that matches the given source path
+    pub fn get_first_asset_for_source(&self, path: &Path) -> Option<&BundledAsset> {
+        self.assets
+            .get(path)
+            .and_then(|assets| assets.iter().next())
+    }
+
     /// Check if the manifest contains a specific asset
     pub fn contains(&self, asset: &BundledAsset) -> bool {
         self.assets

+ 1 - 1
packages/cli/Cargo.toml

@@ -150,7 +150,7 @@ built = { version = "0.7.5", features = ["git2"] }
 [features]
 default = []
 plugin = []
-tokio-console = ["dep:console-subscriber"]
+tokio-console = ["dep:console-subscriber", "tokio/tracing"]
 bundle = []
 no-downloads = []
 

+ 36 - 15
packages/cli/src/build/builder.rs

@@ -417,31 +417,20 @@ impl AppBuilder {
         }
     }
 
-    #[allow(clippy::too_many_arguments)]
-    pub(crate) async fn open(
+    /// Create a list of environment variables that the child process will use
+    pub(crate) fn child_environment_variables(
         &mut self,
-        devserver_ip: SocketAddr,
-        open_address: Option<SocketAddr>,
+        devserver_ip: Option<SocketAddr>,
         start_fullstack_on_address: Option<SocketAddr>,
-        open_browser: bool,
         always_on_top: bool,
         build_id: BuildId,
-        args: &[String],
-    ) -> Result<()> {
+    ) -> Vec<(&'static str, String)> {
         let krate = &self.build;
 
         // Set the env vars that the clients will expect
         // These need to be stable within a release version (ie 0.6.0)
         let mut envs = vec![
             (dioxus_cli_config::CLI_ENABLED_ENV, "true".to_string()),
-            (
-                dioxus_cli_config::DEVSERVER_IP_ENV,
-                devserver_ip.ip().to_string(),
-            ),
-            (
-                dioxus_cli_config::DEVSERVER_PORT_ENV,
-                devserver_ip.port().to_string(),
-            ),
             (
                 dioxus_cli_config::APP_TITLE_ENV,
                 krate.config.web.app.title.clone(),
@@ -457,6 +446,17 @@ impl AppBuilder {
             ),
         ];
 
+        if let Some(devserver_ip) = devserver_ip {
+            envs.push((
+                dioxus_cli_config::DEVSERVER_IP_ENV,
+                devserver_ip.ip().to_string(),
+            ));
+            envs.push((
+                dioxus_cli_config::DEVSERVER_PORT_ENV,
+                devserver_ip.port().to_string(),
+            ));
+        }
+
         if crate::VERBOSITY
             .get()
             .map(|f| f.verbose)
@@ -480,6 +480,27 @@ impl AppBuilder {
             envs.push((dioxus_cli_config::SERVER_PORT_ENV, addr.port().to_string()));
         }
 
+        envs
+    }
+
+    #[allow(clippy::too_many_arguments)]
+    pub(crate) async fn open(
+        &mut self,
+        devserver_ip: SocketAddr,
+        open_address: Option<SocketAddr>,
+        start_fullstack_on_address: Option<SocketAddr>,
+        open_browser: bool,
+        always_on_top: bool,
+        build_id: BuildId,
+        args: &[String],
+    ) -> Result<()> {
+        let envs = self.child_environment_variables(
+            Some(devserver_ip),
+            start_fullstack_on_address,
+            always_on_top,
+            build_id,
+        );
+
         // We try to use stdin/stdout to communicate with the app
         match self.build.platform {
             // Unfortunately web won't let us get a proc handle to it (to read its stdout/stderr) so instead

+ 2 - 0
packages/cli/src/build/mod.rs

@@ -12,6 +12,7 @@ mod assets;
 mod builder;
 mod context;
 mod patch;
+mod pre_render;
 mod request;
 mod tools;
 
@@ -19,5 +20,6 @@ pub(crate) use assets::*;
 pub(crate) use builder::*;
 pub(crate) use context::*;
 pub(crate) use patch::*;
+pub(crate) use pre_render::*;
 pub(crate) use request::*;
 pub(crate) use tools::*;

+ 148 - 0
packages/cli/src/build/pre_render.rs

@@ -0,0 +1,148 @@
+use anyhow::Context;
+use dioxus_cli_config::{server_ip, server_port};
+use dioxus_dx_wire_format::BuildStage;
+use futures_util::{stream::FuturesUnordered, StreamExt};
+use std::{
+    net::{IpAddr, Ipv4Addr, SocketAddr},
+    time::Duration,
+};
+use tokio::process::Command;
+
+use crate::BuildId;
+
+use super::{AppBuilder, BuilderUpdate};
+
+/// Pre-render the static routes, performing static-site generation
+pub(crate) async fn pre_render_static_routes(
+    devserver_ip: Option<SocketAddr>,
+    builder: &mut AppBuilder,
+    updates: Option<&futures_channel::mpsc::UnboundedSender<BuilderUpdate>>,
+) -> anyhow::Result<()> {
+    if let Some(updates) = updates {
+        updates
+            .unbounded_send(BuilderUpdate::Progress {
+                stage: BuildStage::Prerendering,
+            })
+            .unwrap();
+    }
+    let server_exe = builder.build.main_exe();
+
+    // Use the address passed in through environment variables or default to localhost:9999. We need
+    // to default to a value that is different than the CLI default address to avoid conflicts
+    let ip = server_ip().unwrap_or_else(|| IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)));
+    let port = server_port().unwrap_or(9999);
+    let fullstack_address = SocketAddr::new(ip, port);
+    let address = fullstack_address.ip().to_string();
+    let port = fullstack_address.port().to_string();
+
+    // Borrow port and address so we can easily move them into multiple tasks below
+    let address = &address;
+    let port = &port;
+
+    tracing::info!("Running SSG at http://{address}:{port} for {server_exe:?}");
+
+    let vars = builder.child_environment_variables(
+        devserver_ip,
+        Some(fullstack_address),
+        false,
+        BuildId::SERVER,
+    );
+    // Run the server executable
+    let _child = Command::new(&server_exe)
+        .envs(vars)
+        .current_dir(server_exe.parent().unwrap())
+        .stdout(std::process::Stdio::null())
+        .stderr(std::process::Stdio::null())
+        .kill_on_drop(true)
+        .spawn()?;
+
+    // Borrow reqwest_client so we only move the reference into the futures
+    let reqwest_client = reqwest::Client::new();
+    let reqwest_client = &reqwest_client;
+
+    // Get the routes from the `/static_routes` endpoint
+    let mut routes = None;
+
+    // The server may take a few seconds to start up. Try fetching the route up to 5 times with a one second delay
+    const RETRY_ATTEMPTS: usize = 5;
+    for i in 0..=RETRY_ATTEMPTS {
+        tracing::debug!(
+            "Attempting to get static routes from server. Attempt {i} of {RETRY_ATTEMPTS}"
+        );
+
+        let request = reqwest_client
+            .post(format!("http://{address}:{port}/api/static_routes"))
+            .body("{}".to_string())
+            .send()
+            .await;
+        match request {
+            Ok(request) => {
+                routes = Some(request
+                    .json::<Vec<String>>()
+                    .await
+                    .inspect(|text| tracing::debug!("Got static routes: {text:?}"))
+                    .context("Failed to parse static routes from the server. Make sure your server function returns Vec<String> with the (default) json encoding")?);
+                break;
+            }
+            Err(err) => {
+                // If the request fails, try  up to 5 times with a one second delay
+                // If it fails 5 times, return the error
+                if i == RETRY_ATTEMPTS {
+                    return Err(err).context("Failed to get static routes from server. Make sure you have a server function at the `/api/static_routes` endpoint that returns Vec<String> of static routes.");
+                }
+                tokio::time::sleep(std::time::Duration::from_secs(1)).await;
+            }
+        }
+    }
+
+    let routes = routes.expect(
+        "static routes should exist or an error should have been returned on the last attempt",
+    );
+
+    // Create a pool of futures that cache each route
+    let mut resolved_routes = routes
+        .into_iter()
+        .map(|route| async move {
+            tracing::info!("Rendering {route} for SSG");
+
+            // For each route, ping the server to force it to cache the response for ssg
+            let request = reqwest_client
+                .get(format!("http://{address}:{port}{route}"))
+                .header("Accept", "text/html")
+                .send()
+                .await?;
+
+            // If it takes longer than 30 seconds to resolve the route, log a warning
+            let warning_task = tokio::spawn({
+                let route = route.clone();
+                async move {
+                    tokio::time::sleep(Duration::from_secs(30)).await;
+                    tracing::warn!("Route {route} has been rendering for 30 seconds");
+                }
+            });
+
+            // Wait for the streaming response to completely finish before continuing. We don't use the html it returns directly
+            // because it may contain artifacts of intermediate streaming steps while the page is loading. The SSG app should write
+            // the final clean HTML to the disk automatically after the request completes.
+            let _html = request.text().await?;
+
+            // Cancel the warning task if it hasn't already run
+            warning_task.abort();
+
+            Ok::<_, reqwest::Error>(route)
+        })
+        .collect::<FuturesUnordered<_>>();
+
+    while let Some(route) = resolved_routes.next().await {
+        match route {
+            Ok(route) => tracing::debug!("ssg success: {route:?}"),
+            Err(err) => tracing::error!("ssg error: {err:?}"),
+        }
+    }
+
+    tracing::info!("SSG complete");
+
+    drop(_child);
+
+    Ok(())
+}

+ 44 - 14
packages/cli/src/build/request.rs

@@ -1412,6 +1412,10 @@ impl BuildRequest {
         // Now extract the assets from the fat binary
         self.collect_assets(&self.patch_exe(artifacts.time_start), ctx)?;
 
+        // If this is a web build, reset the index.html file in case it was modified by SSG
+        self.write_index_html(&artifacts.assets)
+            .context("Failed to write index.html")?;
+
         // Clean up the temps manually
         // todo: we might want to keep them around for debugging purposes
         for file in object_files {
@@ -3202,7 +3206,6 @@ impl BuildRequest {
             || will_wasm_opt
             || ctx.mode == BuildMode::Fat;
         let keep_names = will_wasm_opt || ctx.mode == BuildMode::Fat;
-        let package_to_asset = self.release && !should_bundle_split;
         let demangle = false;
         let wasm_opt_options = WasmOptConfig {
             memory_packing: self.wasm_split,
@@ -3345,25 +3348,53 @@ impl BuildRequest {
 
         // In release mode, we make the wasm and bindgen files into assets so they get bundled with max
         // optimizations.
-        let wasm_path = if package_to_asset {
+        if self.should_bundle_to_asset() {
+            // Register the main.js with the asset system so it bundles in the snippets and optimizes
+            assets.register_asset(
+                &self.wasm_bindgen_js_output_file(),
+                AssetOptions::Js(JsAssetOptions::new().with_minify(true).with_preload(true)),
+            )?;
+        }
+
+        if self.should_bundle_to_asset() {
             // Make sure to register the main wasm file with the asset system
-            let name = assets.register_asset(&post_bindgen_wasm, AssetOptions::Unknown)?;
+            assets.register_asset(&post_bindgen_wasm, AssetOptions::Unknown)?;
+        }
+
+        // Write the index.html file with the pre-configured contents we got from pre-rendering
+        self.write_index_html(assets)?;
+
+        Ok(())
+    }
+
+    /// Write the index.html file to the output directory. This must be called after the wasm and js
+    /// assets are registered with the asset system if this is a release build.
+    pub(crate) fn write_index_html(&self, assets: &AssetManifest) -> Result<()> {
+        // Get the path to the wasm-bindgen output files. Either the direct file or the opitmized one depending on the build mode
+        let wasm_bindgen_wasm_out = self.wasm_bindgen_wasm_output_file();
+        let wasm_path = if self.should_bundle_to_asset() {
+            let name = assets
+                .get_first_asset_for_source(&wasm_bindgen_wasm_out)
+                .expect("The wasm source must exist before creating index.html");
             format!("assets/{}", name.bundled_path())
         } else {
-            let asset = self.wasm_bindgen_wasm_output_file();
-            format!("wasm/{}", asset.file_name().unwrap().to_str().unwrap())
+            format!(
+                "wasm/{}",
+                wasm_bindgen_wasm_out.file_name().unwrap().to_str().unwrap()
+            )
         };
 
-        let js_path = if package_to_asset {
-            // Register the main.js with the asset system so it bundles in the snippets and optimizes
-            let name = assets.register_asset(
-                &self.wasm_bindgen_js_output_file(),
-                AssetOptions::Js(JsAssetOptions::new().with_minify(true).with_preload(true)),
-            )?;
+        let wasm_bindgen_js_out = self.wasm_bindgen_js_output_file();
+        let js_path = if self.should_bundle_to_asset() {
+            let name = assets
+                .get_first_asset_for_source(&wasm_bindgen_js_out)
+                .expect("The js source must exist before creating index.html");
             format!("assets/{}", name.bundled_path())
         } else {
-            let asset = self.wasm_bindgen_js_output_file();
-            format!("wasm/{}", asset.file_name().unwrap().to_str().unwrap())
+            format!(
+                "wasm/{}",
+                wasm_bindgen_js_out.file_name().unwrap().to_str().unwrap()
+            )
         };
 
         // Write the index.html file with the pre-configured contents we got from pre-rendering
@@ -3371,7 +3402,6 @@ impl BuildRequest {
             self.root_dir().join("index.html"),
             self.prepare_html(assets, &wasm_path, &js_path).unwrap(),
         )?;
-
         Ok(())
     }
 

+ 12 - 3
packages/cli/src/cli/build.rs

@@ -15,6 +15,10 @@ pub struct BuildArgs {
     #[clap(long)]
     pub(crate) fullstack: Option<bool>,
 
+    /// Pre-render all routes returned from the app's `/static_routes` endpoint [default: false]
+    #[clap(long)]
+    pub(crate) ssg: bool,
+
     /// Arguments for the build itself
     #[clap(flatten)]
     pub(crate) build_arguments: TargetArgs,
@@ -63,6 +67,7 @@ impl CommandWithPlatformOverrides<BuildArgs> {
     pub async fn build(self) -> Result<StructuredOutput> {
         tracing::info!("Building project...");
 
+        let ssg = self.shared.ssg;
         let targets = self.into_targets().await?;
 
         AppBuilder::start(&targets.client, BuildMode::Base)?
@@ -73,9 +78,13 @@ impl CommandWithPlatformOverrides<BuildArgs> {
 
         if let Some(server) = targets.server.as_ref() {
             // If the server is present, we need to build it as well
-            AppBuilder::start(server, BuildMode::Base)?
-                .finish_build()
-                .await?;
+            let mut server_build = AppBuilder::start(server, BuildMode::Base)?;
+            server_build.finish_build().await?;
+
+            // Run SSG and cache static routes
+            if ssg {
+                crate::pre_render_static_routes(None, &mut server_build, None).await?;
+            }
 
             tracing::info!(path = ?targets.client.root_dir(), "Server build completed successfully! 🚀");
         }

+ 1 - 141
packages/cli/src/cli/bundle.rs

@@ -1,17 +1,9 @@
 use crate::{AppBuilder, BuildArgs, BuildMode, BuildRequest, Platform};
 use anyhow::{anyhow, Context};
-use dioxus_cli_config::{server_ip, server_port};
-use futures_util::stream::FuturesUnordered;
-use futures_util::StreamExt;
 use path_absolutize::Absolutize;
 use std::collections::HashMap;
-use std::{
-    net::{IpAddr, Ipv4Addr, SocketAddr},
-    path::Path,
-    time::Duration,
-};
 use tauri_bundler::{BundleBinary, BundleSettings, PackageSettings, SettingsBuilder};
-use tokio::process::Command;
+
 use walkdir::WalkDir;
 
 use super::*;
@@ -35,10 +27,6 @@ pub struct Bundle {
     #[clap(long)]
     pub out_dir: Option<PathBuf>,
 
-    /// Run the ssg config of the app and generate the files
-    #[clap(long)]
-    pub(crate) ssg: bool,
-
     /// The arguments for the dioxus build
     #[clap(flatten)]
     pub(crate) args: CommandWithPlatformOverrides<BuildArgs>,
@@ -147,17 +135,6 @@ impl Bundle {
             );
         }
 
-        // Run SSG and cache static routes
-        if self.ssg {
-            if let Some(server) = server.as_ref() {
-                tracing::info!("Running SSG for static routes...");
-                Self::pre_render_static_routes(&server.main_exe()).await?;
-                tracing::info!("SSG complete");
-            } else {
-                tracing::error!("SSG is only supported for fullstack apps. Ensure you have the server feature enabled and try again.");
-            }
-        }
-
         Ok(StructuredOutput::BundleOutput { bundles })
     }
 
@@ -282,121 +259,4 @@ impl Bundle {
 
         Ok(bundles)
     }
-
-    /// Pre-render the static routes, performing static-site generation
-    async fn pre_render_static_routes(server_exe: &Path) -> anyhow::Result<()> {
-        // Use the address passed in through environment variables or default to localhost:9999. We need
-        // to default to a value that is different than the CLI default address to avoid conflicts
-        let ip = server_ip().unwrap_or_else(|| IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)));
-        let port = server_port().unwrap_or(9999);
-        let fullstack_address = SocketAddr::new(ip, port);
-        let address = fullstack_address.ip().to_string();
-        let port = fullstack_address.port().to_string();
-
-        // Borrow port and address so we can easily moe them into multiple tasks below
-        let address = &address;
-        let port = &port;
-
-        tracing::info!("Running SSG at http://{address}:{port} for {server_exe:?}");
-
-        // Run the server executable
-        let _child = Command::new(server_exe)
-            .env(dioxus_cli_config::SERVER_PORT_ENV, port)
-            .env(dioxus_cli_config::SERVER_IP_ENV, address)
-            .current_dir(server_exe.parent().unwrap())
-            .stdout(std::process::Stdio::null())
-            .stderr(std::process::Stdio::null())
-            .kill_on_drop(true)
-            .spawn()?;
-
-        // Borrow reqwest_client so we only move the reference into the futures
-        let reqwest_client = reqwest::Client::new();
-        let reqwest_client = &reqwest_client;
-
-        // Get the routes from the `/static_routes` endpoint
-        let mut routes = None;
-
-        // The server may take a few seconds to start up. Try fetching the route up to 5 times with a one second delay
-        const RETRY_ATTEMPTS: usize = 5;
-        for i in 0..=RETRY_ATTEMPTS {
-            tracing::debug!(
-                "Attempting to get static routes from server. Attempt {i} of {RETRY_ATTEMPTS}"
-            );
-
-            let request = reqwest_client
-                .post(format!("http://{address}:{port}/api/static_routes"))
-                .body("{}".to_string())
-                .send()
-                .await;
-            match request {
-                Ok(request) => {
-                    routes = Some(request
-                    .json::<Vec<String>>()
-                    .await
-                    .inspect(|text| tracing::debug!("Got static routes: {text:?}"))
-                    .context("Failed to parse static routes from the server. Make sure your server function returns Vec<String> with the (default) json encoding")?);
-                    break;
-                }
-                Err(err) => {
-                    // If the request fails, try  up to 5 times with a one second delay
-                    // If it fails 5 times, return the error
-                    if i == RETRY_ATTEMPTS {
-                        return Err(err).context("Failed to get static routes from server. Make sure you have a server function at the `/api/static_routes` endpoint that returns Vec<String> of static routes.");
-                    }
-                    tokio::time::sleep(std::time::Duration::from_secs(1)).await;
-                }
-            }
-        }
-
-        let routes = routes.expect(
-            "static routes should exist or an error should have been returned on the last attempt",
-        );
-
-        // Create a pool of futures that cache each route
-        let mut resolved_routes = routes
-            .into_iter()
-            .map(|route| async move {
-                tracing::info!("Rendering {route} for SSG");
-
-                // For each route, ping the server to force it to cache the response for ssg
-                let request = reqwest_client
-                    .get(format!("http://{address}:{port}{route}"))
-                    .header("Accept", "text/html")
-                    .send()
-                    .await?;
-
-                // If it takes longer than 30 seconds to resolve the route, log a warning
-                let warning_task = tokio::spawn({
-                    let route = route.clone();
-                    async move {
-                        tokio::time::sleep(Duration::from_secs(30)).await;
-                        tracing::warn!("Route {route} has been rendering for 30 seconds");
-                    }
-                });
-
-                // Wait for the streaming response to completely finish before continuing. We don't use the html it returns directly
-                // because it may contain artifacts of intermediate streaming steps while the page is loading. The SSG app should write
-                // the final clean HTML to the disk automatically after the request completes.
-                let _html = request.text().await?;
-
-                // Cancel the warning task if it hasn't already run
-                warning_task.abort();
-
-                Ok::<_, reqwest::Error>(route)
-            })
-            .collect::<FuturesUnordered<_>>();
-
-        while let Some(route) = resolved_routes.next().await {
-            match route {
-                Ok(route) => tracing::debug!("ssg success: {route:?}"),
-                Err(err) => tracing::error!("ssg error: {err:?}"),
-            }
-        }
-
-        tracing::info!("SSG complete");
-
-        drop(_child);
-
-        Ok(())
-    }
 }

+ 1 - 1
packages/cli/src/cli/run.rs

@@ -49,7 +49,7 @@ impl RunArgs {
                     match update {
                         BuilderUpdate::BuildReady { bundle } => {
                             _ = builder
-                                .open(bundle, &mut devserver)
+                                .open(&bundle, &mut devserver)
                                 .await
                                 .inspect_err(|e| tracing::error!("Failed to open app: {}", e));
 

+ 5 - 0
packages/cli/src/logging.rs

@@ -82,6 +82,11 @@ impl TraceController {
             ))
         };
 
+        #[cfg(feature = "tokio-console")]
+        let filter = filter
+            .add_directive("tokio=trace".parse().unwrap())
+            .add_directive("runtime=trace".parse().unwrap());
+
         let json_filter = tracing_subscriber::filter::filter_fn(move |meta| {
             if meta.fields().len() == 1 && meta.fields().iter().next().unwrap().name() == "json" {
                 return args.verbosity.json_output;

+ 4 - 1
packages/cli/src/serve/mod.rs

@@ -126,6 +126,9 @@ pub(crate) async fn serve_all(args: ServeArgs, tracer: &mut TraceController) ->
                 // And then update the websocketed clients with the new build status in case they want it
                 devserver.new_build_update(&update).await;
 
+                // Start the SSG build if we need to
+                builder.new_build_update(&update, &devserver).await;
+
                 // And then open the app if it's ready
                 match update {
                     BuilderUpdate::Progress { .. } => {}
@@ -161,7 +164,7 @@ pub(crate) async fn serve_all(args: ServeArgs, tracer: &mut TraceController) ->
                         }
                         BuildMode::Base | BuildMode::Fat => {
                             _ = builder
-                                .open(bundle, &mut devserver)
+                                .open(&bundle, &mut devserver)
                                 .await
                                 .inspect_err(|e| tracing::error!("Failed to open app: {}", e));
                         }

+ 1 - 0
packages/cli/src/serve/output.rs

@@ -550,6 +550,7 @@ impl Output {
             BuildStage::Linking => lines.push("Linking".yellow()),
             BuildStage::Hotpatching => lines.push("Hot-patching...".yellow()),
             BuildStage::ExtractingAssets => lines.push("Extracting assets".yellow()),
+            BuildStage::Prerendering => lines.push("Pre-rendering...".yellow()),
             _ => {}
         };
 

+ 38 - 5
packages/cli/src/serve/runner.rs

@@ -1,8 +1,8 @@
 use super::{AppBuilder, ServeUpdate, WebServer};
 use crate::{
     platform_override::CommandWithPlatformOverrides, BuildArtifacts, BuildId, BuildMode,
-    BuildTargets, Error, HotpatchModuleCache, Platform, Result, ServeArgs, TailwindCli, TraceSrc,
-    Workspace,
+    BuildTargets, BuilderUpdate, Error, HotpatchModuleCache, Platform, Result, ServeArgs,
+    TailwindCli, TraceSrc, Workspace,
 };
 use anyhow::Context;
 use dioxus_core::internal::{
@@ -66,6 +66,7 @@ pub(crate) struct AppServer {
     pub(crate) _wsl_file_poll_interval: u16,
     pub(crate) always_on_top: bool,
     pub(crate) fullstack: bool,
+    pub(crate) ssg: bool,
     pub(crate) watch_fs: bool,
 
     // resolve args related to the webserver
@@ -138,6 +139,7 @@ impl AppServer {
         let (watcher_tx, watcher_rx) = futures_channel::mpsc::unbounded();
         let watcher = create_notify_watcher(watcher_tx.clone(), wsl_file_poll_interval as u64);
 
+        let ssg = args.platform_args.shared.targets.ssg;
         let target_args = CommandWithPlatformOverrides {
             shared: args.platform_args.shared.targets,
             server: args.platform_args.server.map(|s| s.targets),
@@ -150,7 +152,7 @@ impl AppServer {
         let fullstack = server.is_some();
         let should_proxy_port = match client.platform {
             Platform::Server => true,
-            _ => fullstack,
+            _ => fullstack && !ssg,
         };
 
         let proxied_port = should_proxy_port
@@ -204,6 +206,7 @@ impl AppServer {
             _force_sequential: force_sequential,
             cross_origin_policy,
             fullstack,
+            ssg,
             tw_watcher,
             server_args,
             client_args,
@@ -226,6 +229,27 @@ impl AppServer {
         Ok(runner)
     }
 
+    pub(crate) async fn rebuild_ssg(&mut self, devserver: &WebServer) {
+        if self.client.stage != BuildStage::Success {
+            return;
+        }
+        // Run SSG and cache static routes if the server build is done
+        if let Some(server) = self.server.as_mut() {
+            if !self.ssg || server.stage != BuildStage::Success {
+                return;
+            }
+            if let Err(err) = crate::pre_render_static_routes(
+                Some(devserver.devserver_address()),
+                server,
+                Some(&server.tx.clone()),
+            )
+            .await
+            {
+                tracing::error!("Failed to pre-render static routes: {err}");
+            }
+        }
+    }
+
     pub(crate) async fn wait(&mut self) -> ServeUpdate {
         let client = &mut self.client;
         let server = self.server.as_mut();
@@ -294,6 +318,14 @@ impl AppServer {
         }
     }
 
+    /// Handle an update from the builder
+    pub(crate) async fn new_build_update(&mut self, update: &BuilderUpdate, devserver: &WebServer) {
+        if let BuilderUpdate::BuildReady { .. } = update {
+            // If the build is ready, we need to check if we need to pre-render with ssg
+            self.rebuild_ssg(devserver).await;
+        }
+    }
+
     /// Handle the list of changed files from the file watcher, attempting to aggressively prevent
     /// full rebuilds by hot-reloading RSX and hot-patching Rust code.
     ///
@@ -488,7 +520,7 @@ impl AppServer {
     /// Finally "bundle" this app and return a handle to it
     pub(crate) async fn open(
         &mut self,
-        artifacts: BuildArtifacts,
+        artifacts: &BuildArtifacts,
         devserver: &mut WebServer,
     ) -> Result<()> {
         // Make sure to save artifacts regardless of if we're opening the app or not
@@ -555,7 +587,8 @@ impl AppServer {
         let displayed_address = devserver.displayed_address();
 
         // Always open the server first after the client has been built
-        if let Some(server) = self.server.as_mut() {
+        // Only open the server if it isn't prerendered
+        if let Some(server) = self.server.as_mut().filter(|_| !self.ssg) {
             tracing::debug!("Opening server build");
             server.soft_kill().await;
             server

+ 1 - 0
packages/dx-wire-format/src/lib.rs

@@ -91,4 +91,5 @@ pub enum BuildStage {
     Aborted,
     Restarting,
     CompressingAssets,
+    Prerendering,
 }

+ 43 - 44
packages/playwright-tests/nested-suspense-ssg.spec.js

@@ -1,51 +1,50 @@
-// TODO: ssg is disabled in the CLI server
-// // @ts-check
-// const { test, expect } = require("@playwright/test");
+// @ts-check
+const { test, expect } = require("@playwright/test");
 
-// test("nested suspense resolves", async ({ page }) => {
-//   // Wait for the dev server to reload
-//   await page.goto("http://localhost:6060");
-//   // Then wait for the page to start loading
-//   await page.goto("http://localhost:6060", { waitUntil: "commit" });
+test("nested suspense resolves", async ({ page }) => {
+  // Wait for the dev server to reload
+  await page.goto("http://localhost:6060");
+  // Then wait for the page to start loading
+  await page.goto("http://localhost:6060", { waitUntil: "commit" });
 
-//   // Expect the page to contain the suspense result from the server
-//   const mainMessageTitle = page.locator("#title-0");
-//   await expect(mainMessageTitle).toContainText("The robot says hello world");
-//   const mainMessageBody = page.locator("#body-0");
-//   await expect(mainMessageBody).toContainText(
-//     "The robot becomes sentient and says hello world"
-//   );
+  // Expect the page to contain the suspense result from the server
+  const mainMessageTitle = page.locator("#title-0");
+  await expect(mainMessageTitle).toContainText("The robot says hello world");
+  const mainMessageBody = page.locator("#body-0");
+  await expect(mainMessageBody).toContainText(
+    "The robot becomes sentient and says hello world"
+  );
 
-//   // And expect the title to have resolved on the client
-//   await expect(page).toHaveTitle("The robot says hello world");
+  // And expect the title to have resolved on the client
+  await expect(page).toHaveTitle("The robot says hello world");
 
-//   // Nested suspense should be resolved
-//   const nestedMessageTitle1 = page.locator("#title-1");
-//   await expect(nestedMessageTitle1).toContainText("The world says hello back");
-//   const nestedMessageBody1 = page.locator("#body-1");
-//   await expect(nestedMessageBody1).toContainText(
-//     "In a stunning turn of events, the world collectively unites and says hello back"
-//   );
+  // Nested suspense should be resolved
+  const nestedMessageTitle1 = page.locator("#title-1");
+  await expect(nestedMessageTitle1).toContainText("The world says hello back");
+  const nestedMessageBody1 = page.locator("#body-1");
+  await expect(nestedMessageBody1).toContainText(
+    "In a stunning turn of events, the world collectively unites and says hello back"
+  );
 
-//   const nestedMessageDiv2 = page.locator("#children-2");
-//   await expect(nestedMessageDiv2).toBeEmpty();
-//   const nestedMessageTitle2 = page.locator("#title-2");
-//   await expect(nestedMessageTitle2).toContainText("Goodbye Robot");
-//   const nestedMessageBody2 = page.locator("#body-2");
-//   await expect(nestedMessageBody2).toContainText("The robot says goodbye");
+  const nestedMessageDiv2 = page.locator("#children-2");
+  await expect(nestedMessageDiv2).toBeEmpty();
+  const nestedMessageTitle2 = page.locator("#title-2");
+  await expect(nestedMessageTitle2).toContainText("Goodbye Robot");
+  const nestedMessageBody2 = page.locator("#body-2");
+  await expect(nestedMessageBody2).toContainText("The robot says goodbye");
 
-//   const nestedMessageDiv3 = page.locator("#children-3");
-//   await expect(nestedMessageDiv3).toBeEmpty();
-//   const nestedMessageTitle3 = page.locator("#title-3");
-//   await expect(nestedMessageTitle3).toContainText("Goodbye World");
-//   const nestedMessageBody3 = page.locator("#body-3");
-//   await expect(nestedMessageBody3).toContainText("The world says goodbye");
+  const nestedMessageDiv3 = page.locator("#children-3");
+  await expect(nestedMessageDiv3).toBeEmpty();
+  const nestedMessageTitle3 = page.locator("#title-3");
+  await expect(nestedMessageTitle3).toContainText("Goodbye World");
+  const nestedMessageBody3 = page.locator("#body-3");
+  await expect(nestedMessageBody3).toContainText("The world says goodbye");
 
-//   // Deeply nested suspense should be resolved
-//   const nestedMessageDiv4 = page.locator("#children-4");
-//   await expect(nestedMessageDiv4).toBeEmpty();
-//   const nestedMessageTitle4 = page.locator("#title-4");
-//   await expect(nestedMessageTitle4).toContainText("Hello World");
-//   const nestedMessageBody4 = page.locator("#body-4");
-//   await expect(nestedMessageBody4).toContainText("The world says hello again");
-// });
+  // Deeply nested suspense should be resolved
+  const nestedMessageDiv4 = page.locator("#children-4");
+  await expect(nestedMessageDiv4).toBeEmpty();
+  const nestedMessageTitle4 = page.locator("#title-4");
+  await expect(nestedMessageTitle4).toContainText("Hello World");
+  const nestedMessageBody4 = page.locator("#body-4");
+  await expect(nestedMessageBody4).toContainText("The world says hello again");
+});

+ 9 - 10
packages/playwright-tests/playwright.config.js

@@ -147,16 +147,15 @@ module.exports = defineConfig({
       reuseExistingServer: !process.env.CI,
       stdout: "pipe",
     },
-    // currently disabled - we want ssg to be a "bundle" feature, not a serve feature.
-    // {
-    //   cwd: path.join(process.cwd(), "nested-suspense"),
-    //   command:
-    //     'cargo run --package dioxus-cli --release -- serve --bin nested-suspense-ssg --force-sequential --platform web --ssg --addr "127.0.0.1" --port 6060',
-    //   port: 6060,
-    //   timeout: 50 * 60 * 1000,
-    //   reuseExistingServer: !process.env.CI,
-    //   stdout: "pipe",
-    // },
+    {
+      cwd: path.join(process.cwd(), "nested-suspense"),
+      command:
+        'cargo run --package dioxus-cli --release -- serve --bin nested-suspense-ssg --force-sequential --platform web --ssg --addr "127.0.0.1" --port 6060',
+      port: 6060,
+      timeout: 50 * 60 * 1000,
+      reuseExistingServer: !process.env.CI,
+      stdout: "pipe",
+    },
     {
       cwd: path.join(process.cwd(), "cli-optimization"),
       // Remove the cache folder for the cli-optimization build to force a full cache reset

+ 3 - 0
packages/server-macro/src/lib.rs

@@ -212,6 +212,9 @@ pub fn server(args: proc_macro::TokenStream, body: TokenStream) -> TokenStream {
     };
 
     parsed
+        .default_protocol(Some(
+            parse_quote!(server_fn::Http<server_fn::codec::Json, server_fn::codec::Json>),
+        ))
         .default_input_encoding(Some(parse_quote!(server_fn::codec::Json)))
         .default_output_encoding(Some(parse_quote!(server_fn::codec::Json)))
         .default_server_fn_path(Some(parse_quote!(server_fn)))