turbopack_core/module_graph/
mod.rs

1use core::panic;
2use std::{
3    collections::{BinaryHeap, VecDeque, hash_map::Entry},
4    future::Future,
5};
6
7use anyhow::{Context, Result, bail};
8use auto_hash_map::AutoSet;
9use bincode::{Decode, Encode};
10use petgraph::{
11    Direction,
12    graph::{DiGraph, EdgeIndex, NodeIndex},
13    visit::{EdgeRef, IntoNeighbors, IntoNodeReferences, NodeIndexable, Reversed},
14};
15use rustc_hash::{FxHashMap, FxHashSet};
16use serde::{Deserialize, Serialize};
17use tracing::{Instrument, Level, Span};
18use turbo_rcstr::RcStr;
19use turbo_tasks::{
20    CollectiblesSource, FxIndexMap, NonLocalValue, ReadRef, ResolvedVc, TaskInput, TryJoinIterExt,
21    ValueToString, Vc,
22    debug::ValueDebugFormat,
23    graph::{AdjacencyMap, GraphTraversal, Visit, VisitControlFlow},
24    trace::TraceRawVcs,
25};
26use turbo_tasks_fs::FileSystemPath;
27
28use crate::{
29    chunk::{AsyncModuleInfo, ChunkingContext, ChunkingType},
30    issue::{ImportTrace, ImportTracer, ImportTraces, Issue},
31    module::Module,
32    module_graph::{
33        async_module_info::{AsyncModulesInfo, compute_async_module_info},
34        binding_usage_info::BindingUsageInfo,
35        chunk_group_info::{ChunkGroupEntry, ChunkGroupInfo, compute_chunk_group_info},
36        merged_modules::{MergedModuleInfo, compute_merged_modules},
37        module_batches::{ModuleBatchesGraph, compute_module_batches},
38        style_groups::{StyleGroups, StyleGroupsConfig, compute_style_groups},
39        traced_di_graph::TracedDiGraph,
40    },
41    reference::{ModuleReference, primary_chunkable_referenced_modules},
42    resolve::BindingUsage,
43};
44
45pub mod async_module_info;
46pub mod binding_usage_info;
47pub mod chunk_group_info;
48pub mod merged_modules;
49pub mod module_batch;
50pub(crate) mod module_batches;
51mod side_effect_module_info;
52pub(crate) mod style_groups;
53mod traced_di_graph;
54
55pub use self::module_batches::BatchingConfig;
56
57#[derive(
58    Debug,
59    Copy,
60    Clone,
61    Eq,
62    PartialOrd,
63    Ord,
64    Hash,
65    PartialEq,
66    Serialize,
67    Deserialize,
68    TraceRawVcs,
69    Encode,
70    Decode,
71)]
72pub struct GraphNodeIndex {
73    #[turbo_tasks(trace_ignore)]
74    graph_idx: u32,
75    #[turbo_tasks(trace_ignore)]
76    #[bincode(with_serde)]
77    node_idx: NodeIndex,
78}
79impl GraphNodeIndex {
80    fn new(graph_idx: u32, node_idx: NodeIndex) -> Self {
81        Self {
82            graph_idx,
83            node_idx,
84        }
85    }
86}
87
88unsafe impl NonLocalValue for GraphNodeIndex {}
89
90#[derive(
91    Debug,
92    Copy,
93    Clone,
94    Eq,
95    PartialOrd,
96    Ord,
97    Hash,
98    PartialEq,
99    TraceRawVcs,
100    NonLocalValue,
101    Encode,
102    Decode,
103)]
104pub struct GraphEdgeIndex {
105    graph_idx: u32,
106    #[turbo_tasks(trace_ignore)]
107    #[bincode(with_serde)]
108    edge_idx: EdgeIndex,
109}
110
111impl GraphEdgeIndex {
112    fn new(graph_idx: u32, edge_idx: EdgeIndex) -> Self {
113        Self {
114            graph_idx,
115            edge_idx,
116        }
117    }
118}
119
120#[turbo_tasks::value]
121#[derive(Clone, Debug)]
122pub struct VisitedModules {
123    #[bincode(with = "turbo_bincode::indexmap")]
124    pub modules: FxIndexMap<ResolvedVc<Box<dyn Module>>, GraphNodeIndex>,
125    next_graph_idx: u32,
126}
127
128#[turbo_tasks::value_impl]
129impl VisitedModules {
130    #[turbo_tasks::function]
131    pub fn empty() -> Vc<Self> {
132        Self {
133            modules: Default::default(),
134            next_graph_idx: 0,
135        }
136        .cell()
137    }
138
139    #[turbo_tasks::function]
140    pub async fn from_graph(graph: Vc<SingleModuleGraph>) -> Result<Vc<Self>> {
141        Ok(Self {
142            modules: graph
143                .await?
144                .enumerate_nodes()
145                .flat_map(|(node_idx, module)| match module {
146                    SingleModuleGraphNode::Module(module) => Some((
147                        *module,
148                        GraphNodeIndex {
149                            graph_idx: 0,
150                            node_idx,
151                        },
152                    )),
153                    SingleModuleGraphNode::VisitedModule { .. } => None,
154                })
155                .collect(),
156            next_graph_idx: 1,
157        }
158        .cell())
159    }
160
161    #[turbo_tasks::function]
162    pub fn with_incremented_index(&self) -> Result<Vc<Self>> {
163        Ok(Self {
164            modules: self.modules.clone(),
165            next_graph_idx: self.next_graph_idx + 1,
166        }
167        .cell())
168    }
169
170    #[turbo_tasks::function]
171    pub async fn concatenate(&self, graph: Vc<SingleModuleGraph>) -> Result<Vc<Self>> {
172        let graph = graph.await?;
173        let iter = self
174            .modules
175            .iter()
176            .map(|(module, idx)| (*module, *idx))
177            .chain(
178                graph
179                    .enumerate_nodes()
180                    .flat_map(|(node_idx, module)| match module {
181                        SingleModuleGraphNode::Module(module) => Some((
182                            *module,
183                            GraphNodeIndex {
184                                graph_idx: self.next_graph_idx,
185                                node_idx,
186                            },
187                        )),
188                        SingleModuleGraphNode::VisitedModule { .. } => None,
189                    }),
190            );
191
192        let mut map = FxIndexMap::with_capacity_and_hasher(
193            self.modules.len() + graph.number_of_modules,
194            Default::default(),
195        );
196        for (k, v) in iter {
197            map.entry(k).or_insert(v);
198        }
199        map.shrink_to_fit();
200
201        Ok(Self {
202            modules: map,
203            next_graph_idx: self.next_graph_idx + 1,
204        }
205        .cell())
206    }
207}
208
209pub type GraphEntriesT = Vec<ChunkGroupEntry>;
210
211#[turbo_tasks::value(transparent)]
212pub struct GraphEntries(GraphEntriesT);
213
214#[turbo_tasks::value_impl]
215impl GraphEntries {
216    #[turbo_tasks::function]
217    pub fn empty() -> Vc<Self> {
218        Vc::cell(Vec::new())
219    }
220}
221
222#[turbo_tasks::value(cell = "new", eq = "manual")]
223#[derive(Clone, Default)]
224pub struct SingleModuleGraph {
225    pub graph: TracedDiGraph<SingleModuleGraphNode, RefData>,
226
227    /// The number of modules in the graph (excluding VisitedModule nodes)
228    pub number_of_modules: usize,
229
230    // NodeIndex isn't necessarily stable (because of swap_remove), but we never remove nodes.
231    //
232    // HashMaps have nondeterministic order, but this map is only used for lookups (in
233    // `get_module`) and not iteration.
234    //
235    // This contains Vcs, but they are already contained in the graph, so no need to trace this.
236    #[turbo_tasks(trace_ignore)]
237    #[bincode(with_serde)]
238    modules: FxHashMap<ResolvedVc<Box<dyn Module>>, NodeIndex>,
239
240    #[turbo_tasks(trace_ignore)]
241    pub entries: GraphEntriesT,
242}
243
244#[derive(
245    Debug,
246    Clone,
247    Hash,
248    TraceRawVcs,
249    Serialize,
250    Deserialize,
251    Eq,
252    PartialEq,
253    ValueDebugFormat,
254    NonLocalValue,
255)]
256pub struct RefData {
257    pub chunking_type: ChunkingType,
258    pub binding_usage: BindingUsage,
259    pub reference: ResolvedVc<Box<dyn ModuleReference>>,
260}
261
262impl SingleModuleGraph {
263    /// Walks the graph starting from the given entries and collects all reachable nodes, skipping
264    /// nodes listed in `visited_modules`
265    /// The resulting graph's outgoing edges are in reverse order.
266    async fn new_inner(
267        entries: &GraphEntriesT,
268        visited_modules: &FxIndexMap<ResolvedVc<Box<dyn Module>>, GraphNodeIndex>,
269        include_traced: bool,
270        include_binding_usage: bool,
271    ) -> Result<Vc<Self>> {
272        let emit_spans = tracing::enabled!(Level::INFO);
273        let root_nodes = entries
274            .iter()
275            .flat_map(|e| e.entries())
276            .map(|e| SingleModuleGraphBuilderNode::new_module(emit_spans, e))
277            .try_join()
278            .await?;
279
280        let children_nodes_iter = AdjacencyMap::new()
281            .visit(
282                root_nodes,
283                SingleModuleGraphBuilder {
284                    visited_modules,
285                    emit_spans,
286                    include_traced,
287                    include_binding_usage,
288                },
289            )
290            .await
291            .completed()?;
292        let node_count = children_nodes_iter.len();
293
294        let mut graph: DiGraph<SingleModuleGraphNode, RefData> = DiGraph::with_capacity(
295            node_count,
296            // From real world measurements each module has about 3-4 children
297            // If it has more this would cause an additional allocation, but that's fine
298            node_count * 4,
299        );
300
301        let mut number_of_modules = 0;
302        let mut modules: FxHashMap<ResolvedVc<Box<dyn Module>>, NodeIndex> =
303            FxHashMap::with_capacity_and_hasher(node_count, Default::default());
304        {
305            let _span = tracing::info_span!("build module graph").entered();
306            for (parent, current) in children_nodes_iter.into_breadth_first_edges() {
307                let (module, graph_node, count) = match current {
308                    SingleModuleGraphBuilderNode::Module { module, ident: _ } => {
309                        (module, SingleModuleGraphNode::Module(module), 1)
310                    }
311                    SingleModuleGraphBuilderNode::VisitedModule { module, idx } => (
312                        module,
313                        SingleModuleGraphNode::VisitedModule { idx, module },
314                        0,
315                    ),
316                };
317
318                // Find the current node, if it was already added
319                let current_idx = if let Some(current_idx) = modules.get(&module) {
320                    *current_idx
321                } else {
322                    let idx = graph.add_node(graph_node);
323                    number_of_modules += count;
324                    modules.insert(module, idx);
325                    idx
326                };
327                // Add the edge
328                if let Some((SingleModuleGraphBuilderNode::Module { module, .. }, ref_data)) =
329                    parent
330                {
331                    let parent_idx = *modules.get(&module).unwrap();
332                    graph.add_edge(parent_idx, current_idx, ref_data);
333                }
334            }
335        }
336
337        graph.shrink_to_fit();
338
339        #[cfg(debug_assertions)]
340        {
341            use once_cell::sync::Lazy;
342            static CHECK_FOR_DUPLICATE_MODULES: Lazy<bool> = Lazy::new(|| {
343                match std::env::var_os("TURBOPACK_TEMP_DISABLE_DUPLICATE_MODULES_CHECK") {
344                    Some(v) => v != "1" && v != "true",
345                    None => true,
346                }
347            });
348            if *CHECK_FOR_DUPLICATE_MODULES {
349                let mut duplicates = Vec::new();
350                let mut set = FxHashSet::default();
351                for &module in modules.keys() {
352                    let ident = module.ident().to_string().await?;
353                    if !set.insert(ident.clone()) {
354                        duplicates.push(ident)
355                    }
356                }
357                if !duplicates.is_empty() {
358                    panic!("Duplicate module idents in graph: {duplicates:#?}");
359                }
360            }
361        }
362
363        let graph = SingleModuleGraph {
364            graph: TracedDiGraph::new(graph),
365            number_of_modules,
366            modules,
367            entries: entries.clone(),
368        }
369        .cell();
370
371        turbo_tasks::emit(ResolvedVc::upcast::<Box<dyn ImportTracer>>(
372            ModuleGraphImportTracer::new(graph).to_resolved().await?,
373        ));
374        Ok(graph)
375    }
376
377    /// Iterate over all nodes in the graph
378    pub fn iter_nodes(&self) -> impl Iterator<Item = ResolvedVc<Box<dyn Module>>> + '_ {
379        self.graph.node_weights().filter_map(|n| match n {
380            SingleModuleGraphNode::Module(node) => Some(*node),
381            SingleModuleGraphNode::VisitedModule { .. } => None,
382        })
383    }
384
385    /// Returns true if the given module is in this graph and is an entry module
386    pub fn has_entry_module(&self, module: ResolvedVc<Box<dyn Module>>) -> bool {
387        if let Some(index) = self.modules.get(&module) {
388            self.graph
389                .edges_directed(*index, Direction::Incoming)
390                .next()
391                .is_none()
392        } else {
393            false
394        }
395    }
396
397    /// Iterate over graph entry points
398    pub fn entry_modules(&self) -> impl Iterator<Item = ResolvedVc<Box<dyn Module>>> + '_ {
399        self.entries.iter().flat_map(|e| e.entries())
400    }
401
402    /// Enumerate all nodes in the graph
403    pub fn enumerate_nodes(
404        &self,
405    ) -> impl Iterator<Item = (NodeIndex, &'_ SingleModuleGraphNode)> + '_ {
406        self.graph.node_references()
407    }
408
409    fn traverse_cycles<'l>(
410        &'l self,
411        edge_filter: impl Fn(&'l RefData) -> bool,
412        mut visit_cycle: impl FnMut(&[&'l ResolvedVc<Box<dyn Module>>]) -> Result<()>,
413        graph_idx: u32,
414        binding_usage: &'l Option<ReadRef<BindingUsageInfo>>,
415    ) -> Result<()> {
416        // see https://en.wikipedia.org/wiki/Tarjan%27s_strongly_connected_components_algorithm
417        // but iteratively instead of recursively
418
419        #[derive(Clone)]
420        struct NodeState {
421            index: u32,
422            lowlink: u32,
423            on_stack: bool,
424        }
425        enum VisitStep {
426            UnvisitedNode(NodeIndex),
427            EdgeAfterVisit { parent: NodeIndex, child: NodeIndex },
428            AfterVisit(NodeIndex),
429        }
430        let mut node_states = vec![None; self.graph.node_bound()];
431        let mut stack = Vec::new();
432        let mut visit_stack = Vec::new();
433        let mut index = 0;
434        let mut scc = Vec::new();
435        for initial_index in self.graph.node_indices() {
436            // Skip over already visited nodes
437            if node_states[initial_index.index()].is_some() {
438                continue;
439            }
440            visit_stack.push(VisitStep::UnvisitedNode(initial_index));
441            while let Some(step) = visit_stack.pop() {
442                match step {
443                    VisitStep::UnvisitedNode(node) => {
444                        node_states[node.index()] = Some(NodeState {
445                            index,
446                            lowlink: index,
447                            on_stack: true,
448                        });
449                        index += 1;
450                        stack.push(node);
451                        visit_stack.push(VisitStep::AfterVisit(node));
452                        let mut neighbors = self.graph.neighbors(node).detach();
453                        while let Some((edge, succ)) = neighbors.next(&self.graph) {
454                            if binding_usage.as_ref().is_some_and(|binding_usage| {
455                                binding_usage
456                                    .is_reference_unused_edge(&GraphEdgeIndex::new(graph_idx, edge))
457                            }) {
458                                continue;
459                            }
460
461                            let edge_weight = self.graph.edge_weight(edge).unwrap();
462                            if !edge_filter(edge_weight) {
463                                continue;
464                            }
465                            let node_state = &node_states[succ.index()];
466                            if let Some(node_state) = node_state {
467                                if node_state.on_stack {
468                                    let index = node_state.index;
469                                    let parent_state = node_states[node.index()].as_mut().unwrap();
470                                    parent_state.lowlink = parent_state.lowlink.min(index);
471                                }
472                            } else {
473                                visit_stack.push(VisitStep::EdgeAfterVisit {
474                                    parent: node,
475                                    child: succ,
476                                });
477                                visit_stack.push(VisitStep::UnvisitedNode(succ));
478                            }
479                        }
480                    }
481                    VisitStep::EdgeAfterVisit { parent, child } => {
482                        let child_state = node_states[child.index()].as_ref().unwrap();
483                        let lowlink = child_state.lowlink;
484
485                        let parent_state = node_states[parent.index()].as_mut().unwrap();
486                        parent_state.lowlink = parent_state.lowlink.min(lowlink);
487                    }
488                    VisitStep::AfterVisit(node) => {
489                        let node_state = node_states[node.index()].as_ref().unwrap();
490                        if node_state.lowlink == node_state.index {
491                            loop {
492                                let poppped = stack.pop().unwrap();
493                                let popped_state = node_states[poppped.index()].as_mut().unwrap();
494                                popped_state.on_stack = false;
495                                if let SingleModuleGraphNode::Module(module) =
496                                    self.graph.node_weight(poppped).unwrap()
497                                {
498                                    scc.push(module);
499                                }
500                                if poppped == node {
501                                    break;
502                                }
503                            }
504                            if scc.len() > 1 {
505                                visit_cycle(&scc)?;
506                            }
507                            scc.clear();
508                        }
509                    }
510                }
511            }
512        }
513        Ok(())
514    }
515
516    /// For each issue computes a (possibly empty) list of traces from the file that produced the
517    /// issue to roots in this module graph.
518    /// There are potentially multiple traces because a given file may get assigned to multiple
519    /// modules depend on how it is used in the application.  Consider a simple utility that is used
520    /// by SSR pages, client side code, and the edge runtime.  This may lead to there being 3
521    /// traces.
522    /// The returned map is guaranteed to have an entry for every issue.
523    pub async fn compute_import_traces_for_issues(
524        &self,
525        issues: &AutoSet<ResolvedVc<Box<dyn Issue>>>,
526    ) -> Result<FxHashMap<ResolvedVc<Box<dyn Issue>>, Vec<ImportTrace>>> {
527        let issue_paths = issues
528            .iter()
529            .map(|issue| issue.file_path().owned())
530            .try_join()
531            .await?;
532        let mut file_path_to_traces: FxHashMap<FileSystemPath, Vec<ImportTrace>> =
533            FxHashMap::with_capacity_and_hasher(issue_paths.len(), Default::default());
534        // initialize an empty vec for each path we care about
535        for issue in &issue_paths {
536            file_path_to_traces.entry(issue.clone()).or_default();
537        }
538
539        {
540            let modules =
541                self.modules
542                    .iter()
543                    .map(|(module, &index)| async move {
544                        Ok((module.ident().path().owned().await?, index))
545                    })
546                    .try_join()
547                    .await?;
548            // Reverse the graph so we can find paths to roots
549            let reversed_graph = Reversed(&self.graph.0);
550            for (path, module_idx) in modules {
551                if let Entry::Occupied(mut entry) = file_path_to_traces.entry(path) {
552                    // compute the path from this index to a root of the graph.
553                    let Some((_, path)) = petgraph::algo::astar(
554                        &reversed_graph,
555                        module_idx,
556                        |n| reversed_graph.neighbors(n).next().is_none(),
557                        // Edge weights
558                        |e| match e.weight().chunking_type {
559                            // Prefer following normal imports/requires when we can
560                            ChunkingType::Parallel { .. } => 0,
561                            _ => 1,
562                        },
563                        // `astar` can be accelerated with a distance estimation heuristic, as long
564                        // as our estimate is never > the actual distance.
565                        // However we don't have a mechanism, so just
566                        // estimate 0 which essentially makes this behave like
567                        // dijktra's shortest path algorithm.  `petgraph` has an implementation of
568                        // dijkstra's but it doesn't report  paths, just distances.
569                        // NOTE: dijkstra's with integer weights can be accelerated with incredibly
570                        // efficient priority queue structures (basically with only 0 and 1 as
571                        // weights you can use a `VecDeque`!).  However,
572                        // this is unlikely to be a performance concern.
573                        // Furthermore, if computing paths _does_ become a performance concern, the
574                        // solution would be a hand written implementation of dijkstras so we can
575                        // hoist redundant work out of this loop.
576                        |_| 0,
577                    ) else {
578                        unreachable!("there must be a path to a root");
579                    };
580                    // Represent the path as a sequence of AssetIdents
581                    // TODO: consider hinting at various transitions (e.g. was this an
582                    // import/require/dynamic-import?)
583                    let path = path
584                        .into_iter()
585                        .map(async |n| {
586                            Ok(self
587                                .graph
588                                .node_weight(n)
589                                .unwrap()
590                                .module()
591                                .ident()
592                                .await?
593                                .clone())
594                        })
595                        .try_join()
596                        .await?;
597                    entry.get_mut().push(path);
598                }
599            }
600        }
601        let mut issue_to_traces: FxHashMap<ResolvedVc<Box<dyn Issue>>, Vec<ImportTrace>> =
602            FxHashMap::with_capacity_and_hasher(issues.len(), Default::default());
603        // Map filepaths back to issues
604        // We can do this by zipping the issue_paths with the issues since they are in the same
605        // order.
606        for (path, issue) in issue_paths.iter().zip(issues) {
607            if let Some(traces) = file_path_to_traces.get(path) {
608                issue_to_traces.insert(*issue, traces.clone());
609            }
610        }
611        Ok(issue_to_traces)
612    }
613}
614
615#[turbo_tasks::value]
616struct ModuleGraphImportTracer {
617    graph: ResolvedVc<SingleModuleGraph>,
618}
619
620#[turbo_tasks::value(shared)]
621struct PathToModulesMap {
622    map: FxHashMap<FileSystemPath, Vec<ResolvedVc<Box<dyn Module>>>>,
623}
624
625#[turbo_tasks::value_impl]
626impl ModuleGraphImportTracer {
627    #[turbo_tasks::function]
628    fn new(graph: ResolvedVc<SingleModuleGraph>) -> Vc<Self> {
629        Self::cell(Self { graph })
630    }
631
632    // Compute this mapping on demand since it might not always be needed.
633    #[turbo_tasks::function]
634    async fn path_to_modules(&self) -> Result<Vc<PathToModulesMap>> {
635        let path_and_modules = self
636            .graph
637            .await?
638            .modules
639            .iter()
640            .map(|(&module, _)| async move { Ok((module.ident().path().owned().await?, module)) })
641            .try_join()
642            .await?;
643        let mut map: FxHashMap<FileSystemPath, Vec<ResolvedVc<Box<dyn Module>>>> =
644            FxHashMap::default();
645        for (path, module) in path_and_modules {
646            map.entry(path).or_default().push(module)
647        }
648        Ok(PathToModulesMap::cell(PathToModulesMap { map }))
649    }
650}
651
652#[turbo_tasks::value_impl]
653impl ImportTracer for ModuleGraphImportTracer {
654    #[turbo_tasks::function]
655    async fn get_traces(self: Vc<Self>, path: FileSystemPath) -> Result<Vc<ImportTraces>> {
656        let path_to_modules = self.path_to_modules().await?;
657        let Some(modules) = path_to_modules.map.get(&path) else {
658            return Ok(Vc::default()); // This isn't unusual, the file just might not be in this
659            // graph.
660        };
661        debug_assert!(!modules.is_empty(), "modules should not be an empty vec");
662        let graph = &*self.await?.graph.await?;
663
664        let reversed_graph = Reversed(&graph.graph.0);
665        return Ok(ImportTraces::cell(ImportTraces(
666            modules
667                .iter()
668                .map(|m| async move {
669                    let Some(&module_idx) = graph.modules.get(m) else {
670                        // The only way this could really happen is if `path_to_modules` is computed
671                        // from a different graph than graph`.  Just error out.
672                        bail!("inconsistent read?")
673                    };
674                    // compute the path from this index to a root of the graph.
675                    let Some((_, path)) = petgraph::algo::astar(
676                        &reversed_graph,
677                        module_idx,
678                        |n| reversed_graph.neighbors(n).next().is_none(),
679                        // Edge weights
680                        |e| match e.weight().chunking_type {
681                            // Prefer following normal imports/requires when we can
682                            ChunkingType::Parallel { .. } => 0,
683                            _ => 1,
684                        },
685                        // `astar` can be accelerated with a distance estimation heuristic, as long
686                        // as our estimate is never > the actual distance.
687                        // However we don't have a mechanism, so just
688                        // estimate 0 which essentially makes this behave like
689                        // dijktra's shortest path algorithm.  `petgraph` has an implementation of
690                        // dijkstra's but it doesn't report  paths, just distances.
691                        // NOTE: dijkstra's with integer weights can be accelerated with incredibly
692                        // efficient priority queue structures (basically with only 0 and 1 as
693                        // weights you can use a `VecDeque`!).  However,
694                        // this is unlikely to be a performance concern.
695                        // Furthermore, if computing paths _does_ become a performance concern, the
696                        // solution would be a hand written implementation of dijkstras so we can
697                        // hoist redundant work out of this loop.
698                        |_| 0,
699                    ) else {
700                        unreachable!("there must be a path to a root");
701                    };
702
703                    // Represent the path as a sequence of AssetIdents
704                    // TODO: consider hinting at various transitions (e.g. was this an
705                    // import/require/dynamic-import?)
706                    let path = path
707                        .into_iter()
708                        .map(async |n| {
709                            graph
710                                .graph
711                                .node_weight(n)
712                                .unwrap() // This is safe since `astar`` only returns indices from the graph
713                                .module()
714                                .ident()
715                                .await
716                        })
717                        .try_join()
718                        .await?;
719                    Ok(path)
720                })
721                .try_join()
722                .await?,
723        )));
724    }
725}
726
727#[turbo_tasks::value(shared)]
728#[derive(Clone, Default)]
729pub struct ModuleGraph {
730    pub graphs: Vec<ResolvedVc<SingleModuleGraph>>,
731
732    pub binding_usage: Option<ResolvedVc<BindingUsageInfo>>,
733}
734
735#[turbo_tasks::value_impl]
736impl ModuleGraph {
737    #[turbo_tasks::function]
738    pub fn from_graphs(graphs: Vec<ResolvedVc<SingleModuleGraph>>) -> Vc<Self> {
739        Self {
740            graphs,
741            binding_usage: None,
742        }
743        .cell()
744    }
745
746    #[turbo_tasks::function]
747    pub fn from_single_graph(graph: ResolvedVc<SingleModuleGraph>) -> Vc<Self> {
748        Self {
749            graphs: vec![graph],
750            binding_usage: None,
751        }
752        .cell()
753    }
754
755    #[turbo_tasks::function]
756    pub fn from_entry_module(
757        module: ResolvedVc<Box<dyn Module>>,
758        include_traced: bool,
759        include_binding_usage: bool,
760    ) -> Vc<Self> {
761        Self::from_single_graph(SingleModuleGraph::new_with_entries(
762            Vc::cell(vec![ChunkGroupEntry::Entry(vec![module])]),
763            include_traced,
764            include_binding_usage,
765        ))
766    }
767
768    #[turbo_tasks::function]
769    pub fn from_modules(
770        modules: Vc<GraphEntries>,
771        include_traced: bool,
772        include_binding_usage: bool,
773    ) -> Vc<Self> {
774        Self::from_single_graph(SingleModuleGraph::new_with_entries(
775            modules,
776            include_traced,
777            include_binding_usage,
778        ))
779    }
780
781    #[turbo_tasks::function]
782    pub async fn chunk_group_info(self: Vc<Self>) -> Result<Vc<ChunkGroupInfo>> {
783        compute_chunk_group_info(&self.read_graphs().await?).await
784    }
785
786    #[turbo_tasks::function]
787    pub async fn merged_modules(self: Vc<Self>) -> Result<Vc<MergedModuleInfo>> {
788        compute_merged_modules(self).await
789    }
790
791    #[turbo_tasks::function]
792    pub async fn module_batches(
793        self: Vc<Self>,
794        config: Vc<BatchingConfig>,
795    ) -> Result<Vc<ModuleBatchesGraph>> {
796        compute_module_batches(self, &*config.await?).await
797    }
798
799    #[turbo_tasks::function]
800    pub async fn style_groups(
801        self: Vc<Self>,
802        chunking_context: Vc<Box<dyn ChunkingContext>>,
803        config: StyleGroupsConfig,
804    ) -> Result<Vc<StyleGroups>> {
805        compute_style_groups(self, chunking_context, &config).await
806    }
807
808    #[turbo_tasks::function]
809    pub async fn async_module_info(self: Vc<Self>) -> Result<Vc<AsyncModulesInfo>> {
810        // `compute_async_module_info` calls `module.is_self_async()`, so we need to again ignore
811        // all issues such that they aren't emitted multiple times.
812        async move {
813            let result_op = compute_async_module_info(self.to_resolved().await?);
814            let result_vc = result_op.resolve_strongly_consistent().await?;
815            result_op.drop_collectibles::<Box<dyn Issue>>();
816            anyhow::Ok(*result_vc)
817        }
818        .instrument(tracing::info_span!("compute async module info"))
819        .await
820    }
821
822    #[turbo_tasks::function]
823    pub async fn referenced_async_modules(
824        self: Vc<Self>,
825        module: ResolvedVc<Box<dyn Module>>,
826    ) -> Result<Vc<AsyncModuleInfo>> {
827        let graph_ref = self.read_graphs().await?;
828        let async_modules_info = self.async_module_info().await?;
829
830        let entry = graph_ref.get_entry(module)?;
831        let referenced_modules = graph_ref
832            .iter_graphs_neighbors_rev(entry, Direction::Outgoing)
833            .filter(|(edge_idx, _)| {
834                let ty = graph_ref.get_edge(*edge_idx).unwrap();
835                ty.chunking_type.is_inherit_async()
836            })
837            .map(|(_, child_idx)| anyhow::Ok(graph_ref.get_node(child_idx)?.module()))
838            .collect::<Result<Vec<_>>>()?
839            .into_iter()
840            .rev()
841            .filter(|m| async_modules_info.contains(m))
842            .map(|m| *m)
843            .collect();
844
845        Ok(AsyncModuleInfo::new(referenced_modules))
846    }
847
848    /// Analyze the module graph and remove unused references (by determining the used exports and
849    /// removing unused imports).
850    ///
851    /// In particular, this removes ChunkableModuleReference-s that list only unused exports in the
852    /// `import_usage()`
853    #[turbo_tasks::function]
854    pub async fn without_unused_references(
855        self: ResolvedVc<Self>,
856        binding_usage: ResolvedVc<BindingUsageInfo>,
857    ) -> Result<Vc<Self>> {
858        Ok(Self {
859            graphs: self.await?.graphs.clone(),
860            binding_usage: Some(binding_usage),
861        }
862        .cell())
863    }
864}
865
866impl ModuleGraph {
867    /// Reads the ModuleGraph into a ModuleGraphRef, awaiting all underlying graphs.
868    pub async fn read_graphs(self: Vc<ModuleGraph>) -> Result<ModuleGraphRef> {
869        let this = self.await?;
870        Ok(ModuleGraphRef {
871            graphs: this.graphs.iter().try_join().await?,
872            skip_visited_module_children: false,
873            graph_idx_override: None,
874            binding_usage: if let Some(binding_usage) = this.binding_usage {
875                Some(binding_usage.await?)
876            } else {
877                None
878            },
879        })
880    }
881
882    /// Returns the underlying graphs as a list, to be used for individual graph traversals.
883    pub fn iter_graphs(
884        self: &ModuleGraph,
885    ) -> impl Iterator<Item = SingleModuleGraphWithBindingUsage> {
886        self.graphs
887            .iter()
888            .enumerate()
889            .map(|(graph_idx, graph)| SingleModuleGraphWithBindingUsage {
890                graph: *graph,
891                graph_idx: graph_idx as u32,
892                binding_usage: self.binding_usage,
893            })
894    }
895}
896
897#[derive(
898    Clone, Debug, PartialEq, Eq, Hash, TaskInput, TraceRawVcs, NonLocalValue, Encode, Decode,
899)]
900pub struct SingleModuleGraphWithBindingUsage {
901    pub graph: ResolvedVc<SingleModuleGraph>,
902    pub graph_idx: u32,
903    pub binding_usage: Option<ResolvedVc<BindingUsageInfo>>,
904}
905
906impl SingleModuleGraphWithBindingUsage {
907    pub async fn read(self: &SingleModuleGraphWithBindingUsage) -> Result<ModuleGraphRef> {
908        Ok(ModuleGraphRef {
909            graphs: vec![self.graph.await?],
910            skip_visited_module_children: true,
911            graph_idx_override: Some(self.graph_idx),
912            binding_usage: if let Some(binding_usage) = &self.binding_usage {
913                Some(binding_usage.await?)
914            } else {
915                None
916            },
917        })
918    }
919}
920
921/// The ReadRef version of ModuleGraph. This is better for eventual consistency, as the graphs
922/// aren't awaited multiple times within the same task.
923pub struct ModuleGraphRef {
924    pub graphs: Vec<ReadRef<SingleModuleGraph>>,
925    // Whether to simply ignore SingleModuleGraphNode::VisitedModule during traversals. For single
926    // module graph usecases, this is what you want. For the whole graph, there should be an error.
927    skip_visited_module_children: bool,
928
929    pub graph_idx_override: Option<u32>,
930
931    pub binding_usage: Option<ReadRef<BindingUsageInfo>>,
932}
933
934impl ModuleGraphRef {
935    fn get_entry(&self, entry: ResolvedVc<Box<dyn Module>>) -> Result<GraphNodeIndex> {
936        if self.graph_idx_override.is_some() {
937            debug_assert_eq!(self.graphs.len(), 1,);
938        }
939
940        let Some(idx) = self
941            .graphs
942            .iter()
943            .enumerate()
944            .find_map(|(graph_idx, graph)| {
945                graph.modules.get(&entry).map(|node_idx| GraphNodeIndex {
946                    graph_idx: self.graph_idx_override.unwrap_or(graph_idx as u32),
947                    node_idx: *node_idx,
948                })
949            })
950        else {
951            bail!("Couldn't find entry module {entry:?} in module graph");
952        };
953        Ok(idx)
954    }
955
956    pub fn entries(&self) -> impl Iterator<Item = ChunkGroupEntry> + '_ {
957        self.graphs.iter().flat_map(|g| g.entries.iter().cloned())
958    }
959
960    fn get_graph(&self, graph_idx: u32) -> &ReadRef<SingleModuleGraph> {
961        if self.graph_idx_override.is_some() {
962            self.graphs.first().unwrap()
963        } else {
964            &self.graphs[graph_idx as usize]
965        }
966    }
967
968    fn get_node(&self, node: GraphNodeIndex) -> Result<&SingleModuleGraphNode> {
969        let graph = self.get_graph(node.graph_idx);
970        graph
971            .graph
972            .node_weight(node.node_idx)
973            .context("Expected graph node")
974    }
975
976    fn get_edge(&self, edge: GraphEdgeIndex) -> Result<&RefData> {
977        let graph = self.get_graph(edge.graph_idx);
978        graph
979            .graph
980            .edge_weight(edge.edge_idx)
981            .context("Expected graph node")
982    }
983
984    fn should_visit_node(&self, node: &SingleModuleGraphNode, direction: Direction) -> bool {
985        if self.skip_visited_module_children && direction == Direction::Outgoing {
986            !matches!(node, SingleModuleGraphNode::VisitedModule { .. })
987        } else {
988            true
989        }
990    }
991
992    pub fn enumerate_nodes(
993        &self,
994    ) -> impl Iterator<Item = (NodeIndex, &'_ SingleModuleGraphNode)> + '_ {
995        self.graphs.iter().flat_map(|g| g.enumerate_nodes())
996    }
997
998    /// Iterate the edges of a node REVERSED!
999    fn iter_graphs_neighbors_rev<'a>(
1000        &'a self,
1001        node: GraphNodeIndex,
1002        direction: Direction,
1003    ) -> impl Iterator<Item = (GraphEdgeIndex, GraphNodeIndex)> + 'a {
1004        let graph = &*self.get_graph(node.graph_idx).graph;
1005
1006        if cfg!(debug_assertions) && direction == Direction::Outgoing {
1007            let node_weight = graph.node_weight(node.node_idx).unwrap();
1008            if let SingleModuleGraphNode::VisitedModule { .. } = node_weight {
1009                panic!("iter_graphs_neighbors_rev called on VisitedModule node");
1010            }
1011        }
1012
1013        let mut walker = graph.neighbors_directed(node.node_idx, direction).detach();
1014        std::iter::from_fn(move || {
1015            while let Some((edge_idx, succ_idx)) = walker.next(graph) {
1016                let edge_idx = GraphEdgeIndex::new(node.graph_idx, edge_idx);
1017                if self
1018                    .binding_usage
1019                    .as_ref()
1020                    .is_some_and(|binding_usage| binding_usage.is_reference_unused_edge(&edge_idx))
1021                {
1022                    // Don't just return None here, that would end the iterator
1023                    continue;
1024                }
1025
1026                return Some((edge_idx, GraphNodeIndex::new(node.graph_idx, succ_idx)));
1027            }
1028            None
1029        })
1030    }
1031
1032    /// Returns a map of all modules in the graphs to their identifiers.
1033    /// This is primarily useful for debugging.
1034    pub async fn get_ids(&self) -> Result<FxHashMap<ResolvedVc<Box<dyn Module>>, ReadRef<RcStr>>> {
1035        Ok(self
1036            .graphs
1037            .iter()
1038            .flat_map(|g| g.iter_nodes())
1039            .map(async |n| Ok((n, n.ident().to_string().await?)))
1040            .try_join()
1041            .await?
1042            .into_iter()
1043            .collect::<FxHashMap<_, _>>())
1044    }
1045
1046    /// Traverses all reachable nodes exactly once and calls the visitor.
1047    ///
1048    /// * `entries` - The entry modules to start the traversal from
1049    /// * `state` mutable state to be shared across the visitors
1050    /// * `visit_preorder` - Called before visiting the children of a node.
1051    ///    - Receives the module and the `state`
1052    ///    - Can return [GraphTraversalAction]s to control the traversal
1053    /// * `visit_postorder` - Called after visiting children of a node.
1054    pub fn traverse_nodes_dfs<S>(
1055        &self,
1056        entries: impl IntoIterator<Item = ResolvedVc<Box<dyn Module>>>,
1057        state: &mut S,
1058        visit_preorder: impl Fn(ResolvedVc<Box<dyn Module>>, &mut S) -> Result<GraphTraversalAction>,
1059        mut visit_postorder: impl FnMut(ResolvedVc<Box<dyn Module>>, &mut S) -> Result<()>,
1060    ) -> Result<()> {
1061        let entries = entries.into_iter().collect::<Vec<_>>();
1062
1063        enum Pass {
1064            Visit,
1065            ExpandAndVisit,
1066        }
1067        let mut stack: Vec<(Pass, GraphNodeIndex)> = Vec::with_capacity(entries.len());
1068        for entry in entries.into_iter().rev() {
1069            stack.push((Pass::ExpandAndVisit, self.get_entry(entry)?));
1070        }
1071        let mut expanded = FxHashSet::default();
1072        while let Some((pass, current)) = stack.pop() {
1073            let current_node = self.get_node(current)?;
1074            match pass {
1075                Pass::Visit => {
1076                    visit_postorder(current_node.module(), state)?;
1077                }
1078                Pass::ExpandAndVisit => {
1079                    if !expanded.insert(current) {
1080                        continue;
1081                    }
1082                    let action = visit_preorder(current_node.module(), state)?;
1083                    if action == GraphTraversalAction::Exclude {
1084                        continue;
1085                    }
1086                    stack.push((Pass::Visit, current));
1087                    if action == GraphTraversalAction::Continue
1088                        && self.should_visit_node(current_node, Direction::Outgoing)
1089                    {
1090                        let current = current_node
1091                            .target_idx(Direction::Outgoing)
1092                            .unwrap_or(current);
1093                        stack.extend(
1094                            self.iter_graphs_neighbors_rev(current, Direction::Outgoing)
1095                                .map(|(_, child)| (Pass::ExpandAndVisit, child)),
1096                        );
1097                    }
1098                }
1099            }
1100        }
1101
1102        Ok(())
1103    }
1104
1105    /// Traverses all reachable edges exactly once and calls the visitor with the edge source and
1106    /// target.
1107    ///
1108    /// This means that target nodes can be revisited (once per incoming edge).
1109    ///
1110    /// * `entry` - The entry module to start the traversal from
1111    /// * `visitor` - Called before visiting the children of a node.
1112    ///    - Receives (originating &SingleModuleGraphNode, edge &ChunkingType), target
1113    ///      &SingleModuleGraphNode, state &S
1114    ///    - Can return [GraphTraversalAction]s to control the traversal
1115    pub fn traverse_edges_bfs(
1116        &self,
1117        entries: impl IntoIterator<Item = ResolvedVc<Box<dyn Module>>>,
1118        mut visitor: impl FnMut(
1119            Option<(ResolvedVc<Box<dyn Module>>, &'_ RefData)>,
1120            ResolvedVc<Box<dyn Module>>,
1121        ) -> Result<GraphTraversalAction>,
1122    ) -> Result<()> {
1123        let mut queue = VecDeque::from(
1124            entries
1125                .into_iter()
1126                .map(|e| self.get_entry(e))
1127                .collect::<Result<Vec<_>>>()?,
1128        );
1129        let mut visited = FxHashSet::default();
1130        for entry_node in &queue {
1131            visitor(None, self.get_node(*entry_node)?.module())?;
1132        }
1133        while let Some(node) = queue.pop_front() {
1134            if visited.insert(node) {
1135                let node_weight = self.get_node(node)?;
1136                for (edge, succ) in self.iter_graphs_neighbors_rev(node, Direction::Outgoing) {
1137                    let succ_weight = self.get_node(succ)?;
1138                    let action = visitor(
1139                        Some((node_weight.module(), self.get_edge(edge)?)),
1140                        succ_weight.module(),
1141                    )?;
1142                    if !self.should_visit_node(succ_weight, Direction::Outgoing) {
1143                        continue;
1144                    }
1145                    let succ = succ_weight.target_idx(Direction::Outgoing).unwrap_or(succ);
1146                    if !visited.contains(&succ) && action == GraphTraversalAction::Continue {
1147                        queue.push_back(succ);
1148                    }
1149                }
1150            }
1151        }
1152
1153        Ok(())
1154    }
1155
1156    /// Traverses all edges exactly once (in an unspecified order) and calls the visitor with the
1157    /// edge source and target.
1158    ///
1159    /// This means that target nodes can be revisited (once per incoming edge).
1160    ///
1161    /// * `visitor` - Called before visiting the children of a node.
1162    ///    - Receives (originating &SingleModuleGraphNode, edge &ChunkingType), target
1163    ///      &SingleModuleGraphNode
1164    pub fn traverse_edges_unordered(
1165        &self,
1166        mut visitor: impl FnMut(
1167            Option<(ResolvedVc<Box<dyn Module>>, &'_ RefData)>,
1168            ResolvedVc<Box<dyn Module>>,
1169        ) -> Result<()>,
1170    ) -> Result<()> {
1171        let entries = self.graphs.iter().flat_map(|g| g.entry_modules());
1172
1173        // Despite the name we need to do a DFS to respect 'reachability' if an edge was trimmed we
1174        // should not follow it, and this is a reasonable way to do that.
1175        self.traverse_edges_dfs(
1176            entries,
1177            &mut (),
1178            |parent, target, _| {
1179                visitor(parent, target)?;
1180                Ok(GraphTraversalAction::Continue)
1181            },
1182            |_, _, _| Ok(()),
1183        )
1184    }
1185
1186    /// Traverses all reachable edges in dfs order. The preorder visitor can be used to
1187    /// forward state down the graph, and to skip subgraphs
1188    ///
1189    /// Use this to collect modules in evaluation order.
1190    ///
1191    /// Target nodes can be revisited (once per incoming edge) in the preorder_visitor, in the post
1192    /// order visitor they are visited exactly once with the first edge they were discovered with.
1193    /// Edges are traversed in normal order, so should correspond to reference order.
1194    ///
1195    /// * `entries` - The entry modules to start the traversal from
1196    /// * `state` - The state to be passed to the visitors
1197    /// * `visit_preorder` - Called before visiting the children of a node.
1198    ///    - Receives: (originating &SingleModuleGraphNode, edge &ChunkingType), target
1199    ///      &SingleModuleGraphNode, state &S
1200    ///    - Can return [GraphTraversalAction]s to control the traversal
1201    /// * `visit_postorder` - Called after visiting the children of a node. Return
1202    ///    - Receives: (originating &SingleModuleGraphNode, edge &ChunkingType), target
1203    ///      &SingleModuleGraphNode, state &S
1204    pub fn traverse_edges_dfs<S>(
1205        &self,
1206        entries: impl IntoIterator<Item = ResolvedVc<Box<dyn Module>>>,
1207        state: &mut S,
1208        visit_preorder: impl FnMut(
1209            Option<(ResolvedVc<Box<dyn Module>>, &'_ RefData)>,
1210            ResolvedVc<Box<dyn Module>>,
1211            &mut S,
1212        ) -> Result<GraphTraversalAction>,
1213        visit_postorder: impl FnMut(
1214            Option<(ResolvedVc<Box<dyn Module>>, &'_ RefData)>,
1215            ResolvedVc<Box<dyn Module>>,
1216            &mut S,
1217        ) -> Result<()>,
1218    ) -> Result<()> {
1219        self.traverse_edges_dfs_impl::<S>(
1220            entries,
1221            state,
1222            visit_preorder,
1223            visit_postorder,
1224            Direction::Outgoing,
1225        )
1226    }
1227
1228    /// Traverses all reachable edges in dfs order over the reversed graph. The preorder visitor can
1229    /// be used to forward state up the graph, and to skip subgraphs
1230    ///
1231    /// Target nodes can be revisited (once per incoming edge) in the preorder_visitor, in the post
1232    /// order visitor they are visited exactly once with the first edge they were discovered with.
1233    /// Edges are traversed in normal order, so should correspond to reference order.
1234    ///
1235    /// * `entries` - The entry modules to start the traversal from
1236    /// * `state` - The state to be passed to the visitors
1237    /// * `visit_preorder` - Called before visiting the children of a node.
1238    ///    - Receives: (originating &SingleModuleGraphNode, edge &ChunkingType), target
1239    ///      &SingleModuleGraphNode, state &S
1240    ///    - Can return [GraphTraversalAction]s to control the traversal
1241    /// * `visit_postorder` - Called after visiting the parents of a node. Return
1242    ///    - Receives: (originating &SingleModuleGraphNode, edge &ChunkingType), target
1243    ///      &SingleModuleGraphNode, state &S
1244    pub fn traverse_edges_reverse_dfs<S>(
1245        &self,
1246        entries: impl IntoIterator<Item = ResolvedVc<Box<dyn Module>>>,
1247        state: &mut S,
1248        visit_preorder: impl FnMut(
1249            Option<(ResolvedVc<Box<dyn Module>>, &'_ RefData)>,
1250            ResolvedVc<Box<dyn Module>>,
1251            &mut S,
1252        ) -> Result<GraphTraversalAction>,
1253        visit_postorder: impl FnMut(
1254            Option<(ResolvedVc<Box<dyn Module>>, &'_ RefData)>,
1255            ResolvedVc<Box<dyn Module>>,
1256            &mut S,
1257        ) -> Result<()>,
1258    ) -> Result<()> {
1259        self.traverse_edges_dfs_impl::<S>(
1260            entries,
1261            state,
1262            visit_preorder,
1263            visit_postorder,
1264            Direction::Incoming,
1265        )
1266    }
1267
1268    fn traverse_edges_dfs_impl<S>(
1269        &self,
1270        entries: impl IntoIterator<Item = ResolvedVc<Box<dyn Module>>>,
1271        state: &mut S,
1272        mut visit_preorder: impl FnMut(
1273            Option<(ResolvedVc<Box<dyn Module>>, &'_ RefData)>,
1274            ResolvedVc<Box<dyn Module>>,
1275            &mut S,
1276        ) -> Result<GraphTraversalAction>,
1277        mut visit_postorder: impl FnMut(
1278            Option<(ResolvedVc<Box<dyn Module>>, &'_ RefData)>,
1279            ResolvedVc<Box<dyn Module>>,
1280            &mut S,
1281        ) -> Result<()>,
1282        direction: Direction,
1283    ) -> Result<()> {
1284        if direction == Direction::Incoming {
1285            debug_assert!(
1286                self.skip_visited_module_children,
1287                "Can only trace reverse edges in a single layer graph. We do not model cross \
1288                 graph reverse edges"
1289            );
1290        }
1291        let entries = entries.into_iter().collect::<Vec<_>>();
1292
1293        enum Pass {
1294            Visit,
1295            ExpandAndVisit,
1296        }
1297        #[allow(clippy::type_complexity)] // This is a temporary internal structure
1298        let mut stack: Vec<(
1299            Pass,
1300            Option<(GraphNodeIndex, GraphEdgeIndex)>,
1301            GraphNodeIndex,
1302        )> = Vec::with_capacity(entries.len());
1303        for entry in entries.into_iter().rev() {
1304            stack.push((Pass::ExpandAndVisit, None, self.get_entry(entry)?));
1305        }
1306        let mut expanded = FxHashSet::default();
1307        while let Some((pass, parent, current)) = stack.pop() {
1308            let parent_arg = match parent {
1309                Some((parent_node, parent_edge)) => Some((
1310                    self.get_node(parent_node)?.module(),
1311                    self.get_edge(parent_edge)?,
1312                )),
1313                None => None,
1314            };
1315            let current_node = self.get_node(current)?;
1316            match pass {
1317                Pass::Visit => {
1318                    visit_postorder(parent_arg, current_node.module(), state)?;
1319                }
1320                Pass::ExpandAndVisit => {
1321                    let action = visit_preorder(parent_arg, current_node.module(), state)?;
1322                    if action == GraphTraversalAction::Exclude {
1323                        continue;
1324                    }
1325                    stack.push((Pass::Visit, parent, current));
1326                    if action == GraphTraversalAction::Continue
1327                        && expanded.insert(current)
1328                        && self.should_visit_node(current_node, direction)
1329                    {
1330                        let current = current_node.target_idx(direction).unwrap_or(current);
1331                        stack.extend(self.iter_graphs_neighbors_rev(current, direction).map(
1332                            |(edge, child)| (Pass::ExpandAndVisit, Some((current, edge)), child),
1333                        ));
1334                    }
1335                }
1336            }
1337        }
1338
1339        Ok(())
1340    }
1341
1342    /// Traverse all cycles in the graph (where the edge filter returns true for the whole cycle)
1343    /// and call the visitor with the nodes in the cycle.
1344    pub fn traverse_cycles(
1345        &self,
1346        edge_filter: impl Fn(&RefData) -> bool,
1347        mut visit_cycle: impl FnMut(&[&ResolvedVc<Box<dyn Module>>]) -> Result<()>,
1348    ) -> Result<()> {
1349        for (graph_idx, graph) in self.graphs.iter().enumerate() {
1350            graph.traverse_cycles(
1351                &edge_filter,
1352                &mut visit_cycle,
1353                graph_idx as u32,
1354                &self.binding_usage,
1355            )?;
1356        }
1357        Ok(())
1358    }
1359
1360    /// Traverses all reachable nodes and also continue revisiting them as long the visitor returns
1361    /// GraphTraversalAction::Continue. The visitor is responsible for the runtime complexity and
1362    /// eventual termination of the traversal. This corresponds to computing a fixed point state for
1363    /// the graph.
1364    ///
1365    /// Nodes are (re)visited according to the returned priority of the node, prioritizing high
1366    /// values. This priority is intended to be used a heuristic to reduce the number of
1367    /// retraversals.
1368    ///
1369    /// * `entries` - The entry modules to start the traversal from
1370    /// * `state` - The state to be passed to the callbacks
1371    /// * `visit` - Called for a specific edge
1372    ///    - Receives: (originating &SingleModuleGraphNode, edge &ChunkingType), target
1373    ///      &SingleModuleGraphNode, state &S
1374    ///    - Return [GraphTraversalAction]s to control the traversal
1375    /// * `priority` - Called for before visiting the children of a node to determine its priority.
1376    ///    - Receives: target &SingleModuleGraphNode, state &S
1377    ///    - Return a priority value for the node
1378    ///
1379    /// Returns the number of node visits (i.e. higher than the node count if there are
1380    /// retraversals).
1381    pub fn traverse_edges_fixed_point_with_priority<S, P: Ord>(
1382        &self,
1383        entries: impl IntoIterator<Item = (ResolvedVc<Box<dyn Module>>, P)>,
1384        state: &mut S,
1385        mut visit: impl FnMut(
1386            Option<(ResolvedVc<Box<dyn Module>>, &'_ RefData, GraphEdgeIndex)>,
1387            ResolvedVc<Box<dyn Module>>,
1388            &mut S,
1389        ) -> Result<GraphTraversalAction>,
1390        priority: impl Fn(ResolvedVc<Box<dyn Module>>, &mut S) -> Result<P>,
1391    ) -> Result<usize> {
1392        if self.skip_visited_module_children {
1393            panic!(
1394                "traverse_edges_fixed_point_with_priority musn't be called on individual graphs"
1395            );
1396        }
1397
1398        let mut visit_order = 0usize;
1399        let mut order = || {
1400            let order = visit_order;
1401            visit_order += 1;
1402            order
1403        };
1404        #[derive(PartialEq, Eq)]
1405        struct NodeWithPriority<T: Ord> {
1406            node: GraphNodeIndex,
1407            priority: T,
1408            visit_order: usize,
1409        }
1410        impl<T: Ord> PartialOrd for NodeWithPriority<T> {
1411            fn partial_cmp(&self, other: &Self) -> Option<std::cmp::Ordering> {
1412                Some(self.cmp(other))
1413            }
1414        }
1415        impl<T: Ord> Ord for NodeWithPriority<T> {
1416            fn cmp(&self, other: &Self) -> std::cmp::Ordering {
1417                // BinaryHeap prioritizes high values
1418
1419                self.priority
1420                    .cmp(&other.priority)
1421                    // Use visit_order, so when there are ties we prioritize earlier discovered
1422                    // nodes, reverting to a BFS in the the case where all priorities are equal
1423                    .then(self.visit_order.cmp(&other.visit_order))
1424            }
1425        }
1426
1427        let mut queue_set = FxHashSet::default();
1428        let mut queue = BinaryHeap::from_iter(
1429            entries
1430                .into_iter()
1431                .map(|(m, priority)| {
1432                    Ok(NodeWithPriority {
1433                        node: self.get_entry(m)?,
1434                        priority,
1435                        visit_order: order(),
1436                    })
1437                })
1438                .collect::<Result<Vec<_>>>()?,
1439        );
1440
1441        for entry_node in &queue {
1442            visit(None, self.get_node(entry_node.node)?.module(), state)?;
1443        }
1444
1445        let mut visit_count = 0usize;
1446        while let Some(NodeWithPriority { node, .. }) = queue.pop() {
1447            queue_set.remove(&node);
1448            let node_weight = self.get_node(node)?;
1449            let node = node_weight.target_idx(Direction::Outgoing).unwrap_or(node);
1450
1451            visit_count += 1;
1452
1453            for (edge, succ) in self.iter_graphs_neighbors_rev(node, Direction::Outgoing) {
1454                let succ_weight = self.get_node(succ)?;
1455
1456                let action = visit(
1457                    Some((node_weight.module(), self.get_edge(edge)?, edge)),
1458                    succ_weight.module(),
1459                    state,
1460                )?;
1461
1462                let succ = succ_weight.target_idx(Direction::Outgoing).unwrap_or(succ);
1463                if action == GraphTraversalAction::Continue && queue_set.insert(succ) {
1464                    queue.push(NodeWithPriority {
1465                        node: succ,
1466                        priority: priority(succ_weight.module(), state)?,
1467                        visit_order: order(),
1468                    });
1469                }
1470            }
1471        }
1472
1473        Ok(visit_count)
1474    }
1475}
1476
1477#[turbo_tasks::value_impl]
1478impl SingleModuleGraph {
1479    #[turbo_tasks::function]
1480    pub async fn new_with_entries(
1481        entries: Vc<GraphEntries>,
1482        include_traced: bool,
1483        include_binding_usage: bool,
1484    ) -> Result<Vc<Self>> {
1485        SingleModuleGraph::new_inner(
1486            &*entries.await?,
1487            &Default::default(),
1488            include_traced,
1489            include_binding_usage,
1490        )
1491        .await
1492    }
1493
1494    #[turbo_tasks::function]
1495    pub async fn new_with_entries_visited(
1496        entries: Vc<GraphEntries>,
1497        visited_modules: Vc<VisitedModules>,
1498        include_traced: bool,
1499        include_binding_usage: bool,
1500    ) -> Result<Vc<Self>> {
1501        SingleModuleGraph::new_inner(
1502            &*entries.await?,
1503            &visited_modules.await?.modules,
1504            include_traced,
1505            include_binding_usage,
1506        )
1507        .await
1508    }
1509
1510    #[turbo_tasks::function]
1511    pub async fn new_with_entries_visited_intern(
1512        // This must not be a Vc<Vec<_>> to ensure layout segment optimization hits the cache
1513        entries: GraphEntriesT,
1514        visited_modules: Vc<VisitedModules>,
1515        include_traced: bool,
1516        include_binding_usage: bool,
1517    ) -> Result<Vc<Self>> {
1518        SingleModuleGraph::new_inner(
1519            &entries,
1520            &visited_modules.await?.modules,
1521            include_traced,
1522            include_binding_usage,
1523        )
1524        .await
1525    }
1526}
1527
1528#[derive(Clone, Debug, Serialize, Deserialize, TraceRawVcs, NonLocalValue)]
1529pub enum SingleModuleGraphNode {
1530    Module(ResolvedVc<Box<dyn Module>>),
1531    // Models a module that is referenced but has already been visited by an earlier graph.
1532    VisitedModule {
1533        idx: GraphNodeIndex,
1534        module: ResolvedVc<Box<dyn Module>>,
1535    },
1536}
1537
1538impl SingleModuleGraphNode {
1539    pub fn module(&self) -> ResolvedVc<Box<dyn Module>> {
1540        match self {
1541            SingleModuleGraphNode::Module(module) => *module,
1542            SingleModuleGraphNode::VisitedModule { module, .. } => *module,
1543        }
1544    }
1545    pub fn target_idx(&self, direction: Direction) -> Option<GraphNodeIndex> {
1546        match self {
1547            SingleModuleGraphNode::VisitedModule { idx, .. } => match direction {
1548                Direction::Outgoing => Some(*idx),
1549                Direction::Incoming => None,
1550            },
1551            SingleModuleGraphNode::Module(_) => None,
1552        }
1553    }
1554}
1555
1556#[derive(PartialEq, Eq, Debug)]
1557pub enum GraphTraversalAction {
1558    /// Continue visiting children
1559    Continue,
1560    /// Skip the immediate children, but visit the node in postorder
1561    Skip,
1562    /// Skip the immediate children and the node in postorder
1563    Exclude,
1564}
1565
1566// These nodes are created while walking the Turbopack modules references, and are used to then
1567// afterwards build the SingleModuleGraph.
1568#[derive(Clone, Hash, PartialEq, Eq)]
1569enum SingleModuleGraphBuilderNode {
1570    /// A regular module
1571    Module {
1572        module: ResolvedVc<Box<dyn Module>>,
1573        // module.ident().to_string(), eagerly computed for tracing
1574        ident: Option<ReadRef<RcStr>>,
1575    },
1576    /// A reference to a module that is already listed in visited_modules
1577    VisitedModule {
1578        module: ResolvedVc<Box<dyn Module>>,
1579        idx: GraphNodeIndex,
1580    },
1581}
1582
1583impl SingleModuleGraphBuilderNode {
1584    async fn new_module(emit_spans: bool, module: ResolvedVc<Box<dyn Module>>) -> Result<Self> {
1585        Ok(Self::Module {
1586            module,
1587            ident: if emit_spans {
1588                // INVALIDATION: we don't need to invalidate when the span name changes
1589                Some(module.ident_string().untracked().await?)
1590            } else {
1591                None
1592            },
1593        })
1594    }
1595    fn new_visited_module(module: ResolvedVc<Box<dyn Module>>, idx: GraphNodeIndex) -> Self {
1596        Self::VisitedModule { module, idx }
1597    }
1598}
1599
1600struct SingleModuleGraphBuilder<'a> {
1601    visited_modules: &'a FxIndexMap<ResolvedVc<Box<dyn Module>>, GraphNodeIndex>,
1602
1603    emit_spans: bool,
1604
1605    /// Whether to walk ChunkingType::Traced references
1606    include_traced: bool,
1607
1608    /// Whether to read ChunkableModuleReference::binding_usage()
1609    include_binding_usage: bool,
1610}
1611impl Visit<SingleModuleGraphBuilderNode, RefData> for SingleModuleGraphBuilder<'_> {
1612    type EdgesIntoIter = Vec<(SingleModuleGraphBuilderNode, RefData)>;
1613    type EdgesFuture = impl Future<Output = Result<Self::EdgesIntoIter>>;
1614
1615    fn visit(
1616        &mut self,
1617        node: &SingleModuleGraphBuilderNode,
1618        edge: Option<&RefData>,
1619    ) -> VisitControlFlow {
1620        if let Some(edge) = edge
1621            && matches!(edge.chunking_type, ChunkingType::Traced)
1622        {
1623            // The graph behind traced references is not part of the module graph traversal
1624            return VisitControlFlow::Skip;
1625        }
1626        match node {
1627            SingleModuleGraphBuilderNode::Module { .. } => VisitControlFlow::Continue,
1628            // Module was already visited previously
1629            SingleModuleGraphBuilderNode::VisitedModule { .. } => VisitControlFlow::Skip,
1630        }
1631    }
1632
1633    fn edges(
1634        &mut self,
1635        // The `skip_duplicates_with_key()` above ensures only a single `edges()` call per module
1636        // (and not per `(module, export)` pair), so the export must not be read here!
1637        node: &SingleModuleGraphBuilderNode,
1638    ) -> Self::EdgesFuture {
1639        // Destructure beforehand to not have to clone the whole node when entering the async block
1640        let &SingleModuleGraphBuilderNode::Module { module, .. } = node else {
1641            // These are always skipped in `visit()`
1642            unreachable!()
1643        };
1644        let visited_modules = self.visited_modules;
1645        let emit_spans = self.emit_spans;
1646        let include_traced = self.include_traced;
1647        let include_binding_usage = self.include_binding_usage;
1648        async move {
1649            let refs_cell = primary_chunkable_referenced_modules(
1650                *module,
1651                include_traced,
1652                include_binding_usage,
1653            );
1654            let refs = match refs_cell.await {
1655                Ok(refs) => refs,
1656                Err(e) => {
1657                    return Err(e.context(module.ident().to_string().await?));
1658                }
1659            };
1660
1661            refs.iter()
1662                .flat_map(|(reference, resolved)| {
1663                    resolved.modules.iter().map(|m| {
1664                        (
1665                            *reference,
1666                            resolved.chunking_type.clone(),
1667                            resolved.binding_usage.clone(),
1668                            *m,
1669                        )
1670                    })
1671                })
1672                .map(async |(reference, ty, binding_usage, target)| {
1673                    let to = if let Some(idx) = visited_modules.get(&target) {
1674                        SingleModuleGraphBuilderNode::new_visited_module(target, *idx)
1675                    } else {
1676                        SingleModuleGraphBuilderNode::new_module(emit_spans, target).await?
1677                    };
1678                    Ok((
1679                        to,
1680                        RefData {
1681                            chunking_type: ty,
1682                            binding_usage,
1683                            reference,
1684                        },
1685                    ))
1686                })
1687                .try_join()
1688                .await
1689        }
1690    }
1691
1692    fn span(
1693        &mut self,
1694        node: &SingleModuleGraphBuilderNode,
1695        edge: Option<&RefData>,
1696    ) -> tracing::Span {
1697        if !self.emit_spans {
1698            return Span::none();
1699        }
1700
1701        let mut span = match node {
1702            SingleModuleGraphBuilderNode::Module {
1703                ident: Some(ident), ..
1704            } => {
1705                tracing::info_span!("module", name = display(ident))
1706            }
1707            SingleModuleGraphBuilderNode::VisitedModule { .. } => {
1708                tracing::info_span!("visited module")
1709            }
1710            _ => unreachable!(),
1711        };
1712
1713        if let Some(edge) = edge {
1714            match &edge.chunking_type {
1715                ChunkingType::Parallel {
1716                    inherit_async: _,
1717                    hoisted: _,
1718                } => {}
1719                ChunkingType::Traced => {
1720                    let _span = span.entered();
1721                    span = tracing::info_span!("traced reference");
1722                }
1723                ChunkingType::Async => {
1724                    let _span = span.entered();
1725                    span = tracing::info_span!("async reference");
1726                }
1727                ChunkingType::Isolated { _ty: ty, merge_tag } => {
1728                    let _span = span.entered();
1729                    span = tracing::info_span!(
1730                        "isolated reference",
1731                        ty = debug(&ty),
1732                        merge_tag = debug(&merge_tag)
1733                    );
1734                }
1735                ChunkingType::Shared {
1736                    inherit_async: _,
1737                    merge_tag,
1738                } => {
1739                    let _span = span.entered();
1740                    span = tracing::info_span!("shared reference", merge_tag = debug(&merge_tag));
1741                }
1742            };
1743        }
1744
1745        span
1746    }
1747}
1748
1749#[cfg(test)]
1750pub mod tests {
1751    use anyhow::Result;
1752    use rustc_hash::FxHashMap;
1753    use turbo_rcstr::{RcStr, rcstr};
1754    use turbo_tasks::{ResolvedVc, TryJoinIterExt, ValueToString, Vc};
1755    use turbo_tasks_backend::{BackendOptions, TurboTasksBackend, noop_backing_storage};
1756    use turbo_tasks_fs::{FileSystem, FileSystemPath, VirtualFileSystem};
1757
1758    use crate::{
1759        asset::{Asset, AssetContent},
1760        ident::AssetIdent,
1761        module::{Module, ModuleSideEffects},
1762        module_graph::{
1763            GraphEntries, GraphTraversalAction, ModuleGraph, ModuleGraphRef, SingleModuleGraph,
1764            VisitedModules, chunk_group_info::ChunkGroupEntry,
1765        },
1766        reference::{ModuleReference, ModuleReferences, SingleChunkableModuleReference},
1767        resolve::ExportUsage,
1768    };
1769
1770    #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
1771    async fn test_traverse_dfs_from_entries_diamond() {
1772        run_graph_test(
1773            vec![rcstr!("a.js")],
1774            {
1775                let mut deps = FxHashMap::default();
1776                // A classic diamond dependency on d
1777                deps.insert(rcstr!("a.js"), vec![rcstr!("b.js"), rcstr!("c.js")]);
1778                deps.insert(rcstr!("b.js"), vec![rcstr!("d.js")]);
1779                deps.insert(rcstr!("c.js"), vec![rcstr!("d.js")]);
1780                deps
1781            },
1782            |graph, entry_modules, module_to_name| {
1783                let mut preorder_visits = Vec::new();
1784                let mut postorder_visits = Vec::new();
1785
1786                graph.traverse_edges_dfs(
1787                    entry_modules,
1788                    &mut (),
1789                    |parent, target, _| {
1790                        preorder_visits.push((
1791                            parent.map(|(node, _)| module_to_name.get(&node).unwrap().clone()),
1792                            module_to_name.get(&target).unwrap().clone(),
1793                        ));
1794                        Ok(GraphTraversalAction::Continue)
1795                    },
1796                    |parent, target, _| {
1797                        postorder_visits.push((
1798                            parent.map(|(node, _)| module_to_name.get(&node).unwrap().clone()),
1799                            module_to_name.get(&target).unwrap().clone(),
1800                        ));
1801                        Ok(())
1802                    },
1803                )?;
1804                assert_eq!(
1805                    vec![
1806                        (None, rcstr!("a.js")),
1807                        (Some(rcstr!("a.js")), rcstr!("b.js")),
1808                        (Some(rcstr!("b.js")), rcstr!("d.js")),
1809                        (Some(rcstr!("a.js")), rcstr!("c.js")),
1810                        (Some(rcstr!("c.js")), rcstr!("d.js"))
1811                    ],
1812                    preorder_visits
1813                );
1814                assert_eq!(
1815                    vec![
1816                        (Some(rcstr!("b.js")), rcstr!("d.js")),
1817                        (Some(rcstr!("a.js")), rcstr!("b.js")),
1818                        (Some(rcstr!("c.js")), rcstr!("d.js")),
1819                        (Some(rcstr!("a.js")), rcstr!("c.js")),
1820                        (None, rcstr!("a.js"))
1821                    ],
1822                    postorder_visits
1823                );
1824                Ok(())
1825            },
1826        )
1827        .await;
1828    }
1829
1830    #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
1831    async fn test_traverse_dfs_from_entries_cycle() {
1832        run_graph_test(
1833            vec![rcstr!("a.js")],
1834            {
1835                let mut deps = FxHashMap::default();
1836                // A cycle of length 3
1837                deps.insert(rcstr!("a.js"), vec![rcstr!("b.js")]);
1838                deps.insert(rcstr!("b.js"), vec![rcstr!("c.js")]);
1839                deps.insert(rcstr!("c.js"), vec![rcstr!("a.js")]);
1840                deps
1841            },
1842            |graph, entry_modules, module_to_name| {
1843                let mut preorder_visits = Vec::new();
1844                let mut postorder_visits = Vec::new();
1845
1846                graph.traverse_edges_dfs(
1847                    entry_modules,
1848                    &mut (),
1849                    |parent, target, _| {
1850                        preorder_visits.push((
1851                            parent.map(|(node, _)| module_to_name.get(&node).unwrap().clone()),
1852                            module_to_name.get(&target).unwrap().clone(),
1853                        ));
1854                        Ok(GraphTraversalAction::Continue)
1855                    },
1856                    |parent, target, _| {
1857                        postorder_visits.push((
1858                            parent.map(|(node, _)| module_to_name.get(&node).unwrap().clone()),
1859                            module_to_name.get(&target).unwrap().clone(),
1860                        ));
1861                        Ok(())
1862                    },
1863                )?;
1864                assert_eq!(
1865                    vec![
1866                        (None, rcstr!("a.js")),
1867                        (Some(rcstr!("a.js")), rcstr!("b.js")),
1868                        (Some(rcstr!("b.js")), rcstr!("c.js")),
1869                        (Some(rcstr!("c.js")), rcstr!("a.js")),
1870                    ],
1871                    preorder_visits
1872                );
1873                assert_eq!(
1874                    vec![
1875                        (Some(rcstr!("c.js")), rcstr!("a.js")),
1876                        (Some(rcstr!("b.js")), rcstr!("c.js")),
1877                        (Some(rcstr!("a.js")), rcstr!("b.js")),
1878                        (None, rcstr!("a.js"))
1879                    ],
1880                    postorder_visits
1881                );
1882                Ok(())
1883            },
1884        )
1885        .await;
1886    }
1887
1888    #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
1889    async fn test_traverse_edges_fixed_point_with_priority_cycle() {
1890        run_graph_test(
1891            vec![rcstr!("a.js")],
1892            {
1893                let mut deps = FxHashMap::default();
1894                // A cycle of length 3
1895                deps.insert(rcstr!("a.js"), vec![rcstr!("b.js")]);
1896                deps.insert(rcstr!("b.js"), vec![rcstr!("c.js")]);
1897                deps.insert(rcstr!("c.js"), vec![rcstr!("a.js")]);
1898                deps
1899            },
1900            |graph, entry_modules, module_to_name| {
1901                let mut visits = Vec::new();
1902                let mut count = 0;
1903
1904                graph.traverse_edges_fixed_point_with_priority(
1905                    entry_modules.into_iter().map(|m| (m, 0)),
1906                    &mut (),
1907                    |parent, target, _| {
1908                        visits.push((
1909                            parent.map(|(node, _, _)| module_to_name.get(&node).unwrap().clone()),
1910                            module_to_name.get(&target).unwrap().clone(),
1911                        ));
1912                        count += 1;
1913
1914                        // We are a cycle so we need to break the loop eventually
1915                        Ok(if count < 6 {
1916                            GraphTraversalAction::Continue
1917                        } else {
1918                            GraphTraversalAction::Skip
1919                        })
1920                    },
1921                    |_, _| Ok(0),
1922                )?;
1923                assert_eq!(
1924                    vec![
1925                        (None, rcstr!("a.js")),
1926                        (Some(rcstr!("a.js")), rcstr!("b.js")),
1927                        (Some(rcstr!("b.js")), rcstr!("c.js")),
1928                        (Some(rcstr!("c.js")), rcstr!("a.js")),
1929                        // we start following the cycle again
1930                        (Some(rcstr!("a.js")), rcstr!("b.js")),
1931                        (Some(rcstr!("b.js")), rcstr!("c.js")),
1932                    ],
1933                    visits
1934                );
1935
1936                Ok(())
1937            },
1938        )
1939        .await;
1940    }
1941
1942    #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
1943    async fn test_traverse_edges_fixed_point_no_priority_is_bfs() {
1944        run_graph_test(
1945            vec![rcstr!("a.js")],
1946            {
1947                let mut deps = FxHashMap::default();
1948                // a simple triangle
1949                //        a
1950                //      b   c
1951                //   d    e    f
1952                deps.insert(rcstr!("a.js"), vec![rcstr!("b.js"), rcstr!("c.js")]);
1953                deps.insert(rcstr!("b.js"), vec![rcstr!("d.js"), rcstr!("e.js")]);
1954                deps.insert(rcstr!("c.js"), vec![rcstr!("e.js"), rcstr!("f.js")]);
1955                deps
1956            },
1957            |graph, entry_modules, module_to_name| {
1958                let mut visits = Vec::new();
1959                let mut count = 0;
1960
1961                graph.traverse_edges_fixed_point_with_priority(
1962                    entry_modules.into_iter().map(|m| (m, 0)),
1963                    &mut (),
1964                    |parent, target, _| {
1965                        visits.push((
1966                            parent.map(|(node, _, _)| module_to_name.get(&node).unwrap().clone()),
1967                            module_to_name.get(&target).unwrap().clone(),
1968                        ));
1969                        count += 1;
1970
1971                        // We are a cycle so we need to break the loop eventually
1972                        Ok(if count < 6 {
1973                            GraphTraversalAction::Continue
1974                        } else {
1975                            GraphTraversalAction::Skip
1976                        })
1977                    },
1978                    |_, _| Ok(0),
1979                )?;
1980
1981                assert_eq!(
1982                    vec![
1983                        (None, rcstr!("a.js")),
1984                        (Some(rcstr!("a.js")), rcstr!("c.js")),
1985                        (Some(rcstr!("a.js")), rcstr!("b.js")),
1986                        (Some(rcstr!("b.js")), rcstr!("e.js")),
1987                        (Some(rcstr!("b.js")), rcstr!("d.js")),
1988                        (Some(rcstr!("c.js")), rcstr!("f.js")),
1989                        (Some(rcstr!("c.js")), rcstr!("e.js")),
1990                    ],
1991                    visits
1992                );
1993
1994                Ok(())
1995            },
1996        )
1997        .await;
1998    }
1999
2000    #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
2001    async fn test_reverse_edges_through_layered_graph() {
2002        let tt = turbo_tasks::TurboTasks::new(TurboTasksBackend::new(
2003            BackendOptions::default(),
2004            noop_backing_storage(),
2005        ));
2006        tt.run_once(async move {
2007            let fs = VirtualFileSystem::new_with_name(rcstr!("test"));
2008            let root = fs.root().await?;
2009
2010            // a simple linear graph a -> b ->c
2011            // but b->c is in a parent graph and a is in the child
2012            let graph = {
2013                let mut deps = FxHashMap::default();
2014
2015                deps.insert(rcstr!("a.js"), vec![rcstr!("b.js"), rcstr!("d.js")]);
2016                deps.insert(rcstr!("b.js"), vec![rcstr!("c.js")]);
2017                deps
2018            };
2019            let repo = TestRepo {
2020                repo: graph
2021                    .iter()
2022                    .map(|(k, v)| {
2023                        (
2024                            root.join(k).unwrap(),
2025                            v.iter().map(|f| root.join(f).unwrap()).collect(),
2026                        )
2027                    })
2028                    .collect(),
2029            }
2030            .cell();
2031            let make_module = |name| {
2032                Vc::upcast::<Box<dyn Module>>(MockModule::new(root.join(name).unwrap(), repo))
2033                    .to_resolved()
2034            };
2035            let a_module = make_module("a.js").await?;
2036            let b_module = make_module("b.js").await?;
2037
2038            let parent_graph = SingleModuleGraph::new_with_entries(
2039                GraphEntries::cell(GraphEntries(vec![ChunkGroupEntry::Entry(vec![b_module])])),
2040                false,
2041                false,
2042            );
2043
2044            let module_graph = ModuleGraph::from_graphs(vec![
2045                parent_graph,
2046                SingleModuleGraph::new_with_entries_visited(
2047                    GraphEntries::cell(GraphEntries(vec![ChunkGroupEntry::Entry(vec![a_module])])),
2048                    VisitedModules::from_graph(parent_graph),
2049                    false,
2050                    false,
2051                ),
2052            ])
2053            .await?;
2054            let child_graph = module_graph.iter_graphs().nth(1).unwrap().read().await?;
2055            // test traversing forward from a in the child graph
2056            {
2057                let mut visited_forward = Vec::new();
2058                child_graph.traverse_edges_dfs(
2059                    vec![a_module],
2060                    &mut (),
2061                    |_parent, child, _state_| {
2062                        visited_forward.push(child);
2063                        Ok(GraphTraversalAction::Continue)
2064                    },
2065                    |_, _, _| Ok(()),
2066                )?;
2067
2068                assert_eq!(
2069                    visited_forward
2070                        .iter()
2071                        .map(|m| m.ident().to_string().owned())
2072                        .try_join()
2073                        .await?,
2074                    vec![
2075                        rcstr!("[test]/a.js"),
2076                        rcstr!("[test]/b.js"),
2077                        rcstr!("[test]/d.js")
2078                    ]
2079                );
2080            }
2081
2082            // test traversing backwards from 'd' which is only in the child graph
2083            {
2084                use turbo_tasks::TryFlatJoinIterExt;
2085                let d_module = child_graph
2086                    .enumerate_nodes()
2087                    .map(|(_index, module)| async move {
2088                        Ok(match module {
2089                            crate::module_graph::SingleModuleGraphNode::Module(module) => {
2090                                if module.ident().to_string().owned().await.unwrap()
2091                                    == "[test]/d.js"
2092                                {
2093                                    Some(*module)
2094                                } else {
2095                                    None
2096                                }
2097                            }
2098                            crate::module_graph::SingleModuleGraphNode::VisitedModule {
2099                                ..
2100                            } => None,
2101                        })
2102                    })
2103                    .try_flat_join()
2104                    .await?
2105                    .into_iter()
2106                    .next()
2107                    .unwrap();
2108                let mut visited_reverse = Vec::new();
2109                child_graph.traverse_edges_reverse_dfs(
2110                    vec![d_module],
2111                    &mut (),
2112                    |_parent, child, _state_| {
2113                        visited_reverse.push(child);
2114                        Ok(GraphTraversalAction::Continue)
2115                    },
2116                    |_, _, _| Ok(()),
2117                )?;
2118                assert_eq!(
2119                    visited_reverse
2120                        .iter()
2121                        .map(|m| m.ident().to_string().owned())
2122                        .try_join()
2123                        .await?,
2124                    vec![rcstr!("[test]/d.js"), rcstr!("[test]/a.js")]
2125                );
2126            }
2127            // test traversing backwards from `b` which is in the parent graph and thus a
2128            // VisitedModule in this graph
2129            {
2130                let mut visited_reverse = Vec::new();
2131                child_graph.traverse_edges_reverse_dfs(
2132                    vec![b_module],
2133                    &mut (),
2134                    |_parent, child, _state_| {
2135                        visited_reverse.push(child);
2136                        Ok(GraphTraversalAction::Continue)
2137                    },
2138                    |_, _, _| Ok(()),
2139                )?;
2140                assert_eq!(
2141                    visited_reverse
2142                        .iter()
2143                        .map(|m| m.ident().to_string().owned())
2144                        .try_join()
2145                        .await?,
2146                    vec![rcstr!("[test]/b.js"), rcstr!("[test]/a.js")]
2147                );
2148            }
2149
2150            Ok(())
2151        })
2152        .await
2153        .unwrap();
2154    }
2155
2156    #[turbo_tasks::value(shared)]
2157    struct TestRepo {
2158        repo: FxHashMap<FileSystemPath, Vec<FileSystemPath>>,
2159    }
2160    #[turbo_tasks::value]
2161    struct MockModule {
2162        path: FileSystemPath,
2163        repo: ResolvedVc<TestRepo>,
2164    }
2165    #[turbo_tasks::value_impl]
2166    impl MockModule {
2167        #[turbo_tasks::function]
2168        fn new(path: FileSystemPath, repo: ResolvedVc<TestRepo>) -> Vc<Self> {
2169            Self { path, repo }.cell()
2170        }
2171    }
2172
2173    #[turbo_tasks::value_impl]
2174    impl Asset for MockModule {
2175        #[turbo_tasks::function]
2176        fn content(&self) -> Vc<AssetContent> {
2177            panic!("MockModule::content shouldn't be called")
2178        }
2179    }
2180
2181    #[turbo_tasks::value_impl]
2182    impl Module for MockModule {
2183        #[turbo_tasks::function]
2184        fn ident(&self) -> Vc<AssetIdent> {
2185            AssetIdent::from_path(self.path.clone())
2186        }
2187
2188        #[turbo_tasks::function]
2189        fn source(&self) -> Vc<crate::source::OptionSource> {
2190            Vc::cell(None)
2191        }
2192
2193        #[turbo_tasks::function]
2194        async fn references(&self) -> Result<Vc<ModuleReferences>> {
2195            let repo = self.repo.await?;
2196            let references = match repo.repo.get(&self.path) {
2197                Some(deps) => {
2198                    deps.iter()
2199                        .map(|p| {
2200                            Vc::upcast::<Box<dyn ModuleReference>>(
2201                                SingleChunkableModuleReference::new(
2202                                    Vc::upcast(MockModule::new(p.clone(), *self.repo)),
2203                                    rcstr!("normal-dep"),
2204                                    ExportUsage::all(),
2205                                ),
2206                            )
2207                            .to_resolved()
2208                        })
2209                        .try_join()
2210                        .await?
2211                }
2212                None => vec![],
2213            };
2214
2215            Ok(Vc::cell(references))
2216        }
2217        #[turbo_tasks::function]
2218        fn side_effects(self: Vc<Self>) -> Vc<ModuleSideEffects> {
2219            ModuleSideEffects::SideEffectful.cell()
2220        }
2221    }
2222
2223    /// Constructs a graph based on the provided dependency adjacency lists and calls the given test
2224    /// function.
2225    ///
2226    /// # Parameters
2227    /// - `entries`: A vector of entry module names (as `RcStr`). These are the starting points for
2228    ///   the graph.
2229    /// - `graph`: A map from module name (`RcStr`) to a vector of its dependency module names
2230    ///   (`RcStr`). Represents the adjacency list of the graph.
2231    /// - `test_fn`: A function that is called with:
2232    ///     - `ReadRef<SingleModuleGraph>`: The constructed module graph.
2233    ///     - `Vec<ResolvedVc<Box<dyn Module>>>`: The resolved entry modules.
2234    ///     - `FxHashMap<ResolvedVc<Box<dyn Module>>, RcStr>`: A mapping from module to its name for
2235    ///       easier analysis in tests.
2236    async fn run_graph_test(
2237        entries: Vec<RcStr>,
2238        graph: FxHashMap<RcStr, Vec<RcStr>>,
2239        test_fn: impl FnOnce(
2240            ModuleGraphRef,
2241            Vec<ResolvedVc<Box<dyn Module>>>,
2242            FxHashMap<ResolvedVc<Box<dyn Module>>, RcStr>,
2243        ) -> Result<()>
2244        + Send
2245        + 'static,
2246    ) {
2247        let tt = turbo_tasks::TurboTasks::new(TurboTasksBackend::new(
2248            BackendOptions::default(),
2249            noop_backing_storage(),
2250        ));
2251        tt.run_once(async move {
2252            let fs = VirtualFileSystem::new_with_name(rcstr!("test"));
2253            let root = fs.root().await?;
2254
2255            let repo = TestRepo {
2256                repo: graph
2257                    .iter()
2258                    .map(|(k, v)| {
2259                        (
2260                            root.join(k).unwrap(),
2261                            v.iter().map(|f| root.join(f).unwrap()).collect(),
2262                        )
2263                    })
2264                    .collect(),
2265            }
2266            .cell();
2267            let entry_modules = entries
2268                .iter()
2269                .map(|e| {
2270                    Vc::upcast::<Box<dyn Module>>(MockModule::new(root.join(e).unwrap(), repo))
2271                        .to_resolved()
2272                })
2273                .try_join()
2274                .await?;
2275            let graph = SingleModuleGraph::new_with_entries(
2276                GraphEntries::cell(GraphEntries(vec![ChunkGroupEntry::Entry(
2277                    entry_modules.clone(),
2278                )])),
2279                false,
2280                false,
2281            );
2282
2283            // Create a simple name mapping to make analyzing the visitors easier.
2284            // Technically they could always pull this name off of the
2285            // `module.ident().await?.path.path` themselves but you cannot `await` in visitors.
2286            let module_to_name = graph
2287                .await?
2288                .modules
2289                .keys()
2290                .map(|m| async move { Ok((*m, m.ident().await?.path.path.clone())) })
2291                .try_join()
2292                .await?
2293                .into_iter()
2294                .collect();
2295            test_fn(
2296                ModuleGraph::from_single_graph(graph).read_graphs().await?,
2297                entry_modules,
2298                module_to_name,
2299            )
2300        })
2301        .await
2302        .unwrap();
2303    }
2304}