about summary refs log tree commit diff
diff options
context:
space:
mode:
authorCamille GILLOT <gillot.camille@gmail.com>2025-07-02 08:36:22 +0000
committerCamille Gillot <gillot.camille@gmail.com>2025-09-07 16:24:46 +0000
commit9b8a719ae48db491a5f18d52fdbb802508bf75a5 (patch)
tree1bf9470fee0c7b0563924990a158e24a2e47995e
parent2ff92e83af6d646e05218374954c6ed2ebb67b3d (diff)
downloadrust-9b8a719ae48db491a5f18d52fdbb802508bf75a5.tar.gz
rust-9b8a719ae48db491a5f18d52fdbb802508bf75a5.zip
Reimplement DestinationPropagation according to live ranges.
-rw-r--r--compiler/rustc_index/src/interval.rs26
-rw-r--r--compiler/rustc_mir_dataflow/src/impls/liveness.rs37
-rw-r--r--compiler/rustc_mir_dataflow/src/impls/mod.rs3
-rw-r--r--compiler/rustc_mir_dataflow/src/points.rs80
-rw-r--r--compiler/rustc_mir_transform/src/dest_prop.rs843
-rw-r--r--tests/mir-opt/dest-prop/cycle.main.DestinationPropagation.panic-abort.diff16
-rw-r--r--tests/mir-opt/dest-prop/cycle.main.DestinationPropagation.panic-unwind.diff16
7 files changed, 449 insertions, 572 deletions
diff --git a/compiler/rustc_index/src/interval.rs b/compiler/rustc_index/src/interval.rs
index 4af5bfcaee6..69a7a69610e 100644
--- a/compiler/rustc_index/src/interval.rs
+++ b/compiler/rustc_index/src/interval.rs
@@ -219,6 +219,32 @@ impl<I: Idx> IntervalSet<I> {
         })
     }
 
+    pub fn disjoint(&self, other: &IntervalSet<I>) -> bool
+    where
+        I: Step,
+    {
+        let helper = move || {
+            let mut self_iter = self.iter_intervals();
+            let mut other_iter = other.iter_intervals();
+
+            let mut self_current = self_iter.next()?;
+            let mut other_current = other_iter.next()?;
+
+            loop {
+                if self_current.end <= other_current.start {
+                    self_current = self_iter.next()?;
+                    continue;
+                }
+                if other_current.end <= self_current.start {
+                    other_current = other_iter.next()?;
+                    continue;
+                }
+                return Some(false);
+            }
+        };
+        helper().unwrap_or(true)
+    }
+
     pub fn is_empty(&self) -> bool {
         self.map.is_empty()
     }
diff --git a/compiler/rustc_mir_dataflow/src/impls/liveness.rs b/compiler/rustc_mir_dataflow/src/impls/liveness.rs
index 6ec1b03a34e..5eba474a60c 100644
--- a/compiler/rustc_mir_dataflow/src/impls/liveness.rs
+++ b/compiler/rustc_mir_dataflow/src/impls/liveness.rs
@@ -92,7 +92,7 @@ impl<'tcx> Visitor<'tcx> for TransferFunction<'_> {
         }
 
         match DefUse::for_place(*place, context) {
-            Some(DefUse::Def) => {
+            DefUse::Def => {
                 if let PlaceContext::MutatingUse(
                     MutatingUseContext::Call | MutatingUseContext::AsmOutput,
                 ) = context
@@ -105,8 +105,8 @@ impl<'tcx> Visitor<'tcx> for TransferFunction<'_> {
                     self.0.kill(place.local);
                 }
             }
-            Some(DefUse::Use) => self.0.gen_(place.local),
-            None => {}
+            DefUse::Use => self.0.gen_(place.local),
+            DefUse::PartialWrite | DefUse::NonUse => {}
         }
 
         self.visit_projection(place.as_ref(), context, location);
@@ -131,23 +131,29 @@ impl<'tcx> Visitor<'tcx> for YieldResumeEffect<'_> {
 }
 
 #[derive(Eq, PartialEq, Clone)]
-enum DefUse {
+pub enum DefUse {
+    /// Full write to the local.
     Def,
+    /// Read of any part of the local.
     Use,
+    /// Partial write to the local.
+    PartialWrite,
+    /// Non-use, like debuginfo.
+    NonUse,
 }
 
 impl DefUse {
     fn apply(state: &mut DenseBitSet<Local>, place: Place<'_>, context: PlaceContext) {
         match DefUse::for_place(place, context) {
-            Some(DefUse::Def) => state.kill(place.local),
-            Some(DefUse::Use) => state.gen_(place.local),
-            None => {}
+            DefUse::Def => state.kill(place.local),
+            DefUse::Use => state.gen_(place.local),
+            DefUse::PartialWrite | DefUse::NonUse => {}
         }
     }
 
-    fn for_place(place: Place<'_>, context: PlaceContext) -> Option<DefUse> {
+    pub fn for_place(place: Place<'_>, context: PlaceContext) -> DefUse {
         match context {
-            PlaceContext::NonUse(_) => None,
+            PlaceContext::NonUse(_) => DefUse::NonUse,
 
             PlaceContext::MutatingUse(
                 MutatingUseContext::Call
@@ -156,21 +162,20 @@ impl DefUse {
                 | MutatingUseContext::Store
                 | MutatingUseContext::Deinit,
             ) => {
+                // Treat derefs as a use of the base local. `*p = 4` is not a def of `p` but a use.
                 if place.is_indirect() {
-                    // Treat derefs as a use of the base local. `*p = 4` is not a def of `p` but a
-                    // use.
-                    Some(DefUse::Use)
+                    DefUse::Use
                 } else if place.projection.is_empty() {
-                    Some(DefUse::Def)
+                    DefUse::Def
                 } else {
-                    None
+                    DefUse::PartialWrite
                 }
             }
 
             // Setting the discriminant is not a use because it does no reading, but it is also not
             // a def because it does not overwrite the whole place
             PlaceContext::MutatingUse(MutatingUseContext::SetDiscriminant) => {
-                place.is_indirect().then_some(DefUse::Use)
+                if place.is_indirect() { DefUse::Use } else { DefUse::PartialWrite }
             }
 
             // All other contexts are uses...
@@ -188,7 +193,7 @@ impl DefUse {
                 | NonMutatingUseContext::PlaceMention
                 | NonMutatingUseContext::FakeBorrow
                 | NonMutatingUseContext::SharedBorrow,
-            ) => Some(DefUse::Use),
+            ) => DefUse::Use,
 
             PlaceContext::MutatingUse(MutatingUseContext::Projection)
             | PlaceContext::NonMutatingUse(NonMutatingUseContext::Projection) => {
diff --git a/compiler/rustc_mir_dataflow/src/impls/mod.rs b/compiler/rustc_mir_dataflow/src/impls/mod.rs
index 3f29b819a6d..6d573e1c00e 100644
--- a/compiler/rustc_mir_dataflow/src/impls/mod.rs
+++ b/compiler/rustc_mir_dataflow/src/impls/mod.rs
@@ -9,7 +9,8 @@ pub use self::initialized::{
     MaybeUninitializedPlaces, MaybeUninitializedPlacesDomain,
 };
 pub use self::liveness::{
-    MaybeLiveLocals, MaybeTransitiveLiveLocals, TransferFunction as LivenessTransferFunction,
+    DefUse, MaybeLiveLocals, MaybeTransitiveLiveLocals,
+    TransferFunction as LivenessTransferFunction,
 };
 pub use self::storage_liveness::{
     MaybeRequiresStorage, MaybeStorageDead, MaybeStorageLive, always_storage_live_locals,
diff --git a/compiler/rustc_mir_dataflow/src/points.rs b/compiler/rustc_mir_dataflow/src/points.rs
index 55a373d4c51..e3d1e04a319 100644
--- a/compiler/rustc_mir_dataflow/src/points.rs
+++ b/compiler/rustc_mir_dataflow/src/points.rs
@@ -1,9 +1,5 @@
-use rustc_index::bit_set::DenseBitSet;
-use rustc_index::interval::SparseIntervalMatrix;
 use rustc_index::{Idx, IndexVec};
-use rustc_middle::mir::{self, BasicBlock, Body, Location};
-
-use crate::framework::{Analysis, Direction, Results, ResultsVisitor, visit_results};
+use rustc_middle::mir::{BasicBlock, Body, Location};
 
 /// Maps between a `Location` and a `PointIndex` (and vice versa).
 pub struct DenseLocationMap {
@@ -93,77 +89,3 @@ rustc_index::newtype_index! {
     #[debug_format = "PointIndex({})"]
     pub struct PointIndex {}
 }
-
-/// Add points depending on the result of the given dataflow analysis.
-pub fn save_as_intervals<'tcx, N, A>(
-    elements: &DenseLocationMap,
-    body: &mir::Body<'tcx>,
-    mut analysis: A,
-    results: Results<A::Domain>,
-) -> SparseIntervalMatrix<N, PointIndex>
-where
-    N: Idx,
-    A: Analysis<'tcx, Domain = DenseBitSet<N>>,
-{
-    let mut values = SparseIntervalMatrix::new(elements.num_points());
-    let reachable_blocks = mir::traversal::reachable_as_bitset(body);
-    if A::Direction::IS_BACKWARD {
-        // Iterate blocks in decreasing order, to visit locations in decreasing order. This
-        // allows to use the more efficient `prepend` method to interval sets.
-        let callback = |state: &DenseBitSet<N>, location| {
-            let point = elements.point_from_location(location);
-            // Use internal iterator manually as it is much more efficient.
-            state.iter().for_each(|node| values.prepend(node, point));
-        };
-        let mut visitor = Visitor { callback };
-        visit_results(
-            body,
-            // Note the `.rev()`.
-            body.basic_blocks.indices().filter(|&bb| reachable_blocks.contains(bb)).rev(),
-            &mut analysis,
-            &results,
-            &mut visitor,
-        );
-    } else {
-        // Iterate blocks in increasing order, to visit locations in increasing order. This
-        // allows to use the more efficient `append` method to interval sets.
-        let callback = |state: &DenseBitSet<N>, location| {
-            let point = elements.point_from_location(location);
-            // Use internal iterator manually as it is much more efficient.
-            state.iter().for_each(|node| values.append(node, point));
-        };
-        let mut visitor = Visitor { callback };
-        visit_results(body, reachable_blocks.iter(), &mut analysis, &results, &mut visitor);
-    }
-    values
-}
-
-struct Visitor<F> {
-    callback: F,
-}
-
-impl<'tcx, A, F> ResultsVisitor<'tcx, A> for Visitor<F>
-where
-    A: Analysis<'tcx>,
-    F: FnMut(&A::Domain, Location),
-{
-    fn visit_after_primary_statement_effect<'mir>(
-        &mut self,
-        _analysis: &mut A,
-        state: &A::Domain,
-        _statement: &'mir mir::Statement<'tcx>,
-        location: Location,
-    ) {
-        (self.callback)(state, location);
-    }
-
-    fn visit_after_primary_terminator_effect<'mir>(
-        &mut self,
-        _analysis: &mut A,
-        state: &A::Domain,
-        _terminator: &'mir mir::Terminator<'tcx>,
-        location: Location,
-    ) {
-        (self.callback)(state, location);
-    }
-}
diff --git a/compiler/rustc_mir_transform/src/dest_prop.rs b/compiler/rustc_mir_transform/src/dest_prop.rs
index cf7425251e8..da1d339b526 100644
--- a/compiler/rustc_mir_transform/src/dest_prop.rs
+++ b/compiler/rustc_mir_transform/src/dest_prop.rs
@@ -59,6 +59,12 @@
 //! The first two conditions are simple structural requirements on the `Assign` statements that can
 //! be trivially checked. The third requirement however is more difficult and costly to check.
 //!
+//! ## Current implementation
+//!
+//! The current implementation relies on live range computation to check for conflicts. We only
+//! allow to merge locals that have disjoint live ranges. The live range are defined with
+//! half-statement granularity, so as to make all writes be live for at least a half statement.
+//!
 //! ## Future Improvements
 //!
 //! There are a number of ways in which this pass could be improved in the future:
@@ -117,9 +123,8 @@
 //! - Layout optimizations for coroutines have been added to improve code generation for
 //!   async/await, which are very similar in spirit to what this optimization does.
 //!
-//! Also, rustc now has a simple NRVO pass (see `nrvo.rs`), which handles a subset of the cases that
-//! this destination propagation pass handles, proving that similar optimizations can be performed
-//! on MIR.
+//! [The next approach][attempt 4] computes a conflict matrix between locals by forbidding merging
+//! locals with competing writes or with one write while the other is live.
 //!
 //! ## Pre/Post Optimization
 //!
@@ -130,20 +135,18 @@
 //! [attempt 1]: https://github.com/rust-lang/rust/pull/47954
 //! [attempt 2]: https://github.com/rust-lang/rust/pull/71003
 //! [attempt 3]: https://github.com/rust-lang/rust/pull/72632
+//! [attempt 4]: https://github.com/rust-lang/rust/pull/96451
 
-use rustc_data_structures::fx::{FxIndexMap, IndexEntry, IndexOccupiedEntry};
+use rustc_data_structures::fx::FxIndexMap;
 use rustc_index::bit_set::DenseBitSet;
 use rustc_index::interval::SparseIntervalMatrix;
-use rustc_middle::bug;
-use rustc_middle::mir::visit::{MutVisitor, PlaceContext, Visitor};
-use rustc_middle::mir::{
-    Body, HasLocalDecls, InlineAsmOperand, Local, LocalKind, Location, MirDumper, Operand,
-    PassWhere, Place, Rvalue, Statement, StatementKind, TerminatorKind, traversal,
-};
+use rustc_index::{IndexSlice, IndexVec, newtype_index};
+use rustc_middle::mir::visit::{MutVisitor, NonMutatingUseContext, PlaceContext, Visitor};
+use rustc_middle::mir::*;
 use rustc_middle::ty::TyCtxt;
-use rustc_mir_dataflow::Analysis;
-use rustc_mir_dataflow::impls::MaybeLiveLocals;
-use rustc_mir_dataflow::points::{DenseLocationMap, PointIndex, save_as_intervals};
+use rustc_mir_dataflow::impls::DefUse;
+use rustc_mir_dataflow::points::{DenseLocationMap, PointIndex};
+use rustc_mir_dataflow::{Analysis, Backward, Results};
 use tracing::{debug, trace};
 
 pub(super) struct DestinationPropagation;
@@ -161,84 +164,62 @@ impl<'tcx> crate::MirPass<'tcx> for DestinationPropagation {
         sess.mir_opt_level() >= 3
     }
 
+    #[tracing::instrument(level = "trace", skip(self, tcx, body))]
     fn run_pass(&self, tcx: TyCtxt<'tcx>, body: &mut Body<'tcx>) {
         let def_id = body.source.def_id();
-        let mut candidates = Candidates::default();
-        let mut write_info = WriteInfo::default();
-        trace!(func = ?tcx.def_path_str(def_id));
+        trace!(?def_id);
 
         let borrowed = rustc_mir_dataflow::impls::borrowed_locals(body);
 
-        let live = MaybeLiveLocals.iterate_to_fixpoint(tcx, body, Some("MaybeLiveLocals-DestProp"));
+        let candidates = Candidates::find(body, &borrowed);
+        trace!(?candidates);
+        if candidates.c.is_empty() {
+            return;
+        }
+
+        let live =
+            MaybeTwoStepLiveLocals.iterate_to_fixpoint(tcx, body, Some("MaybeLiveLocals-DestProp"));
+
         let points = DenseLocationMap::new(body);
-        let mut live = save_as_intervals(&points, body, live.analysis, live.results);
-
-        // In order to avoid having to collect data for every single pair of locals in the body, we
-        // do not allow doing more than one merge for places that are derived from the same local at
-        // once. To avoid missed opportunities, we instead iterate to a fixed point - we'll refer to
-        // each of these iterations as a "round."
-        //
-        // Reaching a fixed point could in theory take up to `min(l, s)` rounds - however, we do not
-        // expect to see MIR like that. To verify this, a test was run against `[rust-lang/regex]` -
-        // the average MIR body saw 1.32 full iterations of this loop. The most that was hit were 30
-        // for a single function. Only 80/2801 (2.9%) of functions saw at least 5.
-        //
-        // [rust-lang/regex]:
-        //     https://github.com/rust-lang/regex/tree/b5372864e2df6a2f5e543a556a62197f50ca3650
-        let mut round_count = 0;
-        loop {
-            // PERF: Can we do something smarter than recalculating the candidates and liveness
-            // results?
-            candidates.reset_and_find(body, &borrowed);
-            trace!(?candidates);
-            dest_prop_mir_dump(tcx, body, &points, &live, round_count);
-
-            FilterInformation::filter_liveness(
-                &mut candidates,
-                &points,
-                &live,
-                &mut write_info,
-                body,
-            );
-
-            // Because we only filter once per round, it is unsound to use a local for more than
-            // one merge operation within a single round of optimizations. We store here which ones
-            // we have already used.
-            let mut merged_locals: DenseBitSet<Local> =
-                DenseBitSet::new_empty(body.local_decls.len());
-
-            // This is the set of merges we will apply this round. It is a subset of the candidates.
-            let mut merges = FxIndexMap::default();
-
-            for (src, candidates) in candidates.c.iter() {
-                if merged_locals.contains(*src) {
-                    continue;
-                }
-                let Some(dest) = candidates.iter().find(|dest| !merged_locals.contains(**dest))
-                else {
-                    continue;
-                };
-
-                // Replace `src` by `dest` everywhere.
-                merges.insert(*src, *dest);
-                merged_locals.insert(*src);
-                merged_locals.insert(*dest);
-
-                // Update liveness information based on the merge we just performed.
-                // Every location where `src` was live, `dest` will be live.
-                live.union_rows(*src, *dest);
-            }
-            trace!(merging = ?merges);
+        let mut relevant = RelevantLocals::compute(&candidates, body.local_decls.len());
+        let mut live = save_as_intervals(&points, body, &relevant.original, live.results);
 
-            if merges.is_empty() {
-                break;
-            }
-            round_count += 1;
+        dest_prop_mir_dump(tcx, body, &points, &live, &relevant);
+
+        let mut merged_locals = DenseBitSet::new_empty(body.local_decls.len());
+
+        for (src, candidates) in candidates.c.into_iter() {
+            trace!(?src, ?candidates);
+
+            let Some(src) = relevant.find(src) else { continue };
+            let Some(src_live_ranges) = &live.row(src) else { continue };
+            trace!(?src, ?src_live_ranges);
+
+            let dst = candidates.into_iter().find_map(|dst| {
+                let dst = relevant.find(dst)?;
+                let dst_live_ranges = &live.row(dst)?;
+                trace!(?dst, ?dst_live_ranges);
+
+                let disjoint = src_live_ranges.disjoint(dst_live_ranges);
+                disjoint.then_some(dst)
+            });
+            let Some(dst) = dst else { continue };
+
+            merged_locals.insert(relevant.original[src]);
+            merged_locals.insert(relevant.original[dst]);
 
-            apply_merges(body, tcx, merges, merged_locals);
+            relevant.union(src, dst);
+            live.union_rows(src, dst);
         }
+        trace!(?merged_locals);
 
-        trace!(round_count);
+        relevant.make_idempotent();
+
+        if merged_locals.is_empty() {
+            return;
+        }
+
+        apply_merges(body, tcx, relevant, merged_locals);
     }
 
     fn is_required(&self) -> bool {
@@ -246,30 +227,6 @@ impl<'tcx> crate::MirPass<'tcx> for DestinationPropagation {
     }
 }
 
-#[derive(Debug, Default)]
-struct Candidates {
-    /// The set of candidates we are considering in this optimization.
-    ///
-    /// We will always merge the key into at most one of its values.
-    ///
-    /// Whether a place ends up in the key or the value does not correspond to whether it appears as
-    /// the lhs or rhs of any assignment. As a matter of fact, the places in here might never appear
-    /// in an assignment at all. This happens because if we see an assignment like this:
-    ///
-    /// ```ignore (syntax-highlighting-only)
-    /// _1.0 = _2.0
-    /// ```
-    ///
-    /// We will still report that we would like to merge `_1` and `_2` in an attempt to allow us to
-    /// remove that assignment.
-    c: FxIndexMap<Local, Vec<Local>>,
-
-    /// A reverse index of the `c` set; if the `c` set contains `a => Place { local: b, proj }`,
-    /// then this contains `b => a`.
-    // PERF: Possibly these should be `SmallVec`s?
-    reverse: FxIndexMap<Local, Vec<Local>>,
-}
-
 //////////////////////////////////////////////////////////
 // Merging
 //
@@ -278,16 +235,16 @@ struct Candidates {
 fn apply_merges<'tcx>(
     body: &mut Body<'tcx>,
     tcx: TyCtxt<'tcx>,
-    merges: FxIndexMap<Local, Local>,
+    relevant: RelevantLocals,
     merged_locals: DenseBitSet<Local>,
 ) {
-    let mut merger = Merger { tcx, merges, merged_locals };
+    let mut merger = Merger { tcx, relevant, merged_locals };
     merger.visit_body_preserves_cfg(body);
 }
 
 struct Merger<'tcx> {
     tcx: TyCtxt<'tcx>,
-    merges: FxIndexMap<Local, Local>,
+    relevant: RelevantLocals,
     merged_locals: DenseBitSet<Local>,
 }
 
@@ -297,8 +254,8 @@ impl<'tcx> MutVisitor<'tcx> for Merger<'tcx> {
     }
 
     fn visit_local(&mut self, local: &mut Local, _: PlaceContext, _location: Location) {
-        if let Some(dest) = self.merges.get(local) {
-            *local = *dest;
+        if let Some(relevant) = self.relevant.find(*local) {
+            *local = self.relevant.original[relevant];
         }
     }
 
@@ -336,376 +293,116 @@ impl<'tcx> MutVisitor<'tcx> for Merger<'tcx> {
 }
 
 //////////////////////////////////////////////////////////
-// Liveness filtering
+// Relevant locals
 //
-// This section enforces bullet point 2
+// Small utility to reduce size of the conflict matrix by only considering locals that appear in
+// the candidates
 
-struct FilterInformation<'a, 'tcx> {
-    body: &'a Body<'tcx>,
-    points: &'a DenseLocationMap,
-    live: &'a SparseIntervalMatrix<Local, PointIndex>,
-    candidates: &'a mut Candidates,
-    write_info: &'a mut WriteInfo,
-    at: Location,
+newtype_index! {
+    /// Represent a subset of locals which appear in candidates.
+    struct RelevantLocal {}
 }
 
-// We first implement some utility functions which we will expose removing candidates according to
-// different needs. Throughout the liveness filtering, the `candidates` are only ever accessed
-// through these methods, and not directly.
-impl Candidates {
-    /// Collects the candidates for merging.
-    ///
-    /// This is responsible for enforcing the first and third bullet point.
-    fn reset_and_find<'tcx>(&mut self, body: &Body<'tcx>, borrowed: &DenseBitSet<Local>) {
-        self.c.clear();
-        self.reverse.clear();
-        let mut visitor = FindAssignments { body, candidates: &mut self.c, borrowed };
-        visitor.visit_body(body);
-        // Deduplicate candidates.
-        for (_, cands) in self.c.iter_mut() {
-            cands.sort();
-            cands.dedup();
-        }
-        // Generate the reverse map.
-        for (src, cands) in self.c.iter() {
-            for dest in cands.iter().copied() {
-                self.reverse.entry(dest).or_default().push(*src);
-            }
-        }
-    }
-
-    /// Just `Vec::retain`, but the condition is inverted and we add debugging output
-    fn vec_filter_candidates(
-        src: Local,
-        v: &mut Vec<Local>,
-        mut f: impl FnMut(Local) -> CandidateFilter,
-        at: Location,
-    ) {
-        v.retain(|dest| {
-            let remove = f(*dest);
-            if remove == CandidateFilter::Remove {
-                trace!("eliminating {:?} => {:?} due to conflict at {:?}", src, dest, at);
-            }
-            remove == CandidateFilter::Keep
-        });
-    }
-
-    /// `vec_filter_candidates` but for an `Entry`
-    fn entry_filter_candidates(
-        mut entry: IndexOccupiedEntry<'_, Local, Vec<Local>>,
-        p: Local,
-        f: impl FnMut(Local) -> CandidateFilter,
-        at: Location,
-    ) {
-        let candidates = entry.get_mut();
-        Self::vec_filter_candidates(p, candidates, f, at);
-        if candidates.len() == 0 {
-            // FIXME(#120456) - is `swap_remove` correct?
-            entry.swap_remove();
-        }
-    }
-
-    /// For all candidates `(p, q)` or `(q, p)` removes the candidate if `f(q)` says to do so
-    fn filter_candidates_by(
-        &mut self,
-        p: Local,
-        mut f: impl FnMut(Local) -> CandidateFilter,
-        at: Location,
-    ) {
-        // Cover the cases where `p` appears as a `src`
-        if let IndexEntry::Occupied(entry) = self.c.entry(p) {
-            Self::entry_filter_candidates(entry, p, &mut f, at);
-        }
-        // And the cases where `p` appears as a `dest`
-        let Some(srcs) = self.reverse.get_mut(&p) else {
-            return;
-        };
-        // We use `retain` here to remove the elements from the reverse set if we've removed the
-        // matching candidate in the forward set.
-        srcs.retain(|src| {
-            if f(*src) == CandidateFilter::Keep {
-                return true;
-            }
-            let IndexEntry::Occupied(entry) = self.c.entry(*src) else {
-                return false;
-            };
-            Self::entry_filter_candidates(
-                entry,
-                *src,
-                |dest| {
-                    if dest == p { CandidateFilter::Remove } else { CandidateFilter::Keep }
-                },
-                at,
-            );
-            false
-        });
-    }
+#[derive(Debug)]
+struct RelevantLocals {
+    original: IndexVec<RelevantLocal, Local>,
+    shrink: IndexVec<Local, Option<RelevantLocal>>,
+    renames: IndexVec<RelevantLocal, RelevantLocal>,
 }
 
-#[derive(Copy, Clone, PartialEq, Eq)]
-enum CandidateFilter {
-    Keep,
-    Remove,
-}
+impl RelevantLocals {
+    #[tracing::instrument(level = "trace", skip(candidates, num_locals), ret)]
+    fn compute(candidates: &Candidates, num_locals: usize) -> RelevantLocals {
+        let mut original = IndexVec::with_capacity(candidates.c.len());
+        let mut shrink = IndexVec::from_elem_n(None, num_locals);
 
-impl<'a, 'tcx> FilterInformation<'a, 'tcx> {
-    /// Filters the set of candidates to remove those that conflict.
-    ///
-    /// The steps we take are exactly those that are outlined at the top of the file. For each
-    /// statement/terminator, we collect the set of locals that are written to in that
-    /// statement/terminator, and then we remove all pairs of candidates that contain one such local
-    /// and another one that is live.
-    ///
-    /// We need to be careful about the ordering of operations within each statement/terminator
-    /// here. Many statements might write and read from more than one place, and we need to consider
-    /// them all. The strategy for doing this is as follows: We first gather all the places that are
-    /// written to within the statement/terminator via `WriteInfo`. Then, we use the liveness
-    /// analysis from *before* the statement/terminator (in the control flow sense) to eliminate
-    /// candidates - this is because we want to conservatively treat a pair of locals that is both
-    /// read and written in the statement/terminator to be conflicting, and the liveness analysis
-    /// before the statement/terminator will correctly report locals that are read in the
-    /// statement/terminator to be live. We are additionally conservative by treating all written to
-    /// locals as also being read from.
-    fn filter_liveness(
-        candidates: &mut Candidates,
-        points: &DenseLocationMap,
-        live: &SparseIntervalMatrix<Local, PointIndex>,
-        write_info: &mut WriteInfo,
-        body: &Body<'tcx>,
-    ) {
-        let mut this = FilterInformation {
-            body,
-            points,
-            live,
-            candidates,
-            // We don't actually store anything at this scope, we just keep things here to be able
-            // to reuse the allocation.
-            write_info,
-            // Doesn't matter what we put here, will be overwritten before being used
-            at: Location::START,
+        // Mark a local as relevant and record it into the maps.
+        let mut declare = |local| {
+            shrink.get_or_insert_with(local, || original.push(local));
         };
-        this.internal_filter_liveness();
-    }
-
-    fn internal_filter_liveness(&mut self) {
-        for (block, data) in traversal::preorder(self.body) {
-            self.at = Location { block, statement_index: data.statements.len() };
-            self.write_info.for_terminator(&data.terminator().kind);
-            self.apply_conflicts();
 
-            for (i, statement) in data.statements.iter().enumerate().rev() {
-                self.at = Location { block, statement_index: i };
-                self.write_info.for_statement(&statement.kind, self.body);
-                self.apply_conflicts();
+        for (&src, destinations) in candidates.c.iter() {
+            declare(src);
+            for &dest in destinations {
+                declare(dest)
             }
         }
-    }
 
-    fn apply_conflicts(&mut self) {
-        let writes = &self.write_info.writes;
-        for p in writes {
-            let other_skip = self.write_info.skip_pair.and_then(|(a, b)| {
-                if a == *p {
-                    Some(b)
-                } else if b == *p {
-                    Some(a)
-                } else {
-                    None
-                }
-            });
-            let at = self.points.point_from_location(self.at);
-            self.candidates.filter_candidates_by(
-                *p,
-                |q| {
-                    if Some(q) == other_skip {
-                        return CandidateFilter::Keep;
-                    }
-                    // It is possible that a local may be live for less than the
-                    // duration of a statement This happens in the case of function
-                    // calls or inline asm. Because of this, we also mark locals as
-                    // conflicting when both of them are written to in the same
-                    // statement.
-                    if self.live.contains(q, at) || writes.contains(&q) {
-                        CandidateFilter::Remove
-                    } else {
-                        CandidateFilter::Keep
-                    }
-                },
-                self.at,
-            );
-        }
+        let renames = IndexVec::from_fn_n(|l| l, original.len());
+        RelevantLocals { original, shrink, renames }
     }
-}
 
-/// Describes where a statement/terminator writes to
-#[derive(Default, Debug)]
-struct WriteInfo {
-    writes: Vec<Local>,
-    /// If this pair of locals is a candidate pair, completely skip processing it during this
-    /// statement. All other candidates are unaffected.
-    skip_pair: Option<(Local, Local)>,
-}
-
-impl WriteInfo {
-    fn for_statement<'tcx>(&mut self, statement: &StatementKind<'tcx>, body: &Body<'tcx>) {
-        self.reset();
-        match statement {
-            StatementKind::Assign(box (lhs, rhs)) => {
-                self.add_place(*lhs);
-                match rhs {
-                    Rvalue::Use(op) => {
-                        self.add_operand(op);
-                        self.consider_skipping_for_assign_use(*lhs, op, body);
-                    }
-                    Rvalue::Repeat(op, _) => {
-                        self.add_operand(op);
-                    }
-                    Rvalue::Cast(_, op, _)
-                    | Rvalue::UnaryOp(_, op)
-                    | Rvalue::ShallowInitBox(op, _) => {
-                        self.add_operand(op);
-                    }
-                    Rvalue::BinaryOp(_, ops) => {
-                        for op in [&ops.0, &ops.1] {
-                            self.add_operand(op);
-                        }
-                    }
-                    Rvalue::Aggregate(_, ops) => {
-                        for op in ops {
-                            self.add_operand(op);
-                        }
-                    }
-                    Rvalue::WrapUnsafeBinder(op, _) => {
-                        self.add_operand(op);
-                    }
-                    Rvalue::ThreadLocalRef(_)
-                    | Rvalue::NullaryOp(_, _)
-                    | Rvalue::Ref(_, _, _)
-                    | Rvalue::RawPtr(_, _)
-                    | Rvalue::Len(_)
-                    | Rvalue::Discriminant(_)
-                    | Rvalue::CopyForDeref(_) => {}
-                }
-            }
-            // Retags are technically also reads, but reporting them as a write suffices
-            StatementKind::SetDiscriminant { place, .. }
-            | StatementKind::Deinit(place)
-            | StatementKind::Retag(_, place) => {
-                self.add_place(**place);
-            }
-            StatementKind::Intrinsic(_)
-            | StatementKind::ConstEvalCounter
-            | StatementKind::Nop
-            | StatementKind::Coverage(_)
-            | StatementKind::StorageLive(_)
-            | StatementKind::StorageDead(_)
-            | StatementKind::BackwardIncompatibleDropHint { .. }
-            | StatementKind::PlaceMention(_) => {}
-            StatementKind::FakeRead(_) | StatementKind::AscribeUserType(_, _) => {
-                bug!("{:?} not found in this MIR phase", statement)
-            }
+    fn find(&self, src: Local) -> Option<RelevantLocal> {
+        let mut src = self.shrink[src]?;
+        while let s2 = self.renames[src]
+            && src != s2
+        {
+            src = s2
         }
+        Some(src)
     }
 
-    fn consider_skipping_for_assign_use<'tcx>(
-        &mut self,
-        lhs: Place<'tcx>,
-        rhs: &Operand<'tcx>,
-        body: &Body<'tcx>,
-    ) {
-        let Some(rhs) = rhs.place() else { return };
-        if let Some(pair) = places_to_candidate_pair(lhs, rhs, body) {
-            self.skip_pair = Some(pair);
-        }
+    fn union(&mut self, lhs: RelevantLocal, rhs: RelevantLocal) {
+        self.renames[lhs] = rhs;
     }
 
-    fn for_terminator<'tcx>(&mut self, terminator: &TerminatorKind<'tcx>) {
-        self.reset();
-        match terminator {
-            TerminatorKind::SwitchInt { discr: op, .. }
-            | TerminatorKind::Assert { cond: op, .. } => {
-                self.add_operand(op);
-            }
-            TerminatorKind::Call { destination, func, args, .. } => {
-                self.add_place(*destination);
-                self.add_operand(func);
-                for arg in args {
-                    self.add_operand(&arg.node);
-                }
-            }
-            TerminatorKind::TailCall { func, args, .. } => {
-                self.add_operand(func);
-                for arg in args {
-                    self.add_operand(&arg.node);
-                }
-            }
-            TerminatorKind::InlineAsm { operands, .. } => {
-                for asm_operand in operands {
-                    match asm_operand {
-                        InlineAsmOperand::In { value, .. } => {
-                            self.add_operand(value);
-                        }
-                        InlineAsmOperand::Out { place, .. } => {
-                            if let Some(place) = place {
-                                self.add_place(*place);
-                            }
-                        }
-                        // Note that the `late` field in `InOut` is about whether the registers used
-                        // for these things overlap, and is of absolutely no interest to us.
-                        InlineAsmOperand::InOut { in_value, out_place, .. } => {
-                            if let Some(place) = out_place {
-                                self.add_place(*place);
-                            }
-                            self.add_operand(in_value);
-                        }
-                        InlineAsmOperand::Const { .. }
-                        | InlineAsmOperand::SymFn { .. }
-                        | InlineAsmOperand::SymStatic { .. }
-                        | InlineAsmOperand::Label { .. } => {}
-                    }
-                }
-            }
-            TerminatorKind::Goto { .. }
-            | TerminatorKind::UnwindResume
-            | TerminatorKind::UnwindTerminate(_)
-            | TerminatorKind::Return
-            | TerminatorKind::Unreachable { .. } => (),
-            TerminatorKind::Drop { .. } => {
-                // `Drop`s create a `&mut` and so are not considered
-            }
-            TerminatorKind::Yield { .. }
-            | TerminatorKind::CoroutineDrop
-            | TerminatorKind::FalseEdge { .. }
-            | TerminatorKind::FalseUnwind { .. } => {
-                bug!("{:?} not found in this MIR phase", terminator)
+    fn make_idempotent(&mut self) {
+        for l in self.renames.indices() {
+            let mut h = self.renames[l];
+            while let h2 = self.renames[h]
+                && h != h2
+            {
+                h = h2
             }
+            self.renames[l] = h;
+            debug_assert_eq!(h, self.renames[h], "non-idempotent for {l:?}");
         }
     }
+}
 
-    fn add_place(&mut self, place: Place<'_>) {
-        self.writes.push(place.local);
-    }
+/////////////////////////////////////////////////////
+// Candidate accumulation
+
+#[derive(Debug, Default)]
+struct Candidates {
+    /// The set of candidates we are considering in this optimization.
+    ///
+    /// We will always merge the key into at most one of its values.
+    ///
+    /// Whether a place ends up in the key or the value does not correspond to whether it appears as
+    /// the lhs or rhs of any assignment. As a matter of fact, the places in here might never appear
+    /// in an assignment at all. This happens because if we see an assignment like this:
+    ///
+    /// ```ignore (syntax-highlighting-only)
+    /// _1.0 = _2.0
+    /// ```
+    ///
+    /// We will still report that we would like to merge `_1` and `_2` in an attempt to allow us to
+    /// remove that assignment.
+    c: FxIndexMap<Local, Vec<Local>>,
+}
+
+// We first implement some utility functions which we will expose removing candidates according to
+// different needs. Throughout the liveness filtering, the `candidates` are only ever accessed
+// through these methods, and not directly.
+impl Candidates {
+    /// Collects the candidates for merging.
+    ///
+    /// This is responsible for enforcing the first and third bullet point.
+    fn find(body: &Body<'_>, borrowed: &DenseBitSet<Local>) -> Candidates {
+        let mut visitor = FindAssignments { body, candidates: Default::default(), borrowed };
+        visitor.visit_body(body);
 
-    fn add_operand<'tcx>(&mut self, op: &Operand<'tcx>) {
-        match op {
-            // FIXME(JakobDegen): In a previous version, the `Move` case was incorrectly treated as
-            // being a read only. This was unsound, however we cannot add a regression test because
-            // it is not possible to set this off with current MIR. Once we have that ability, a
-            // regression test should be added.
-            Operand::Move(p) => self.add_place(*p),
-            Operand::Copy(_) | Operand::Constant(_) => (),
+        // Deduplicate candidates.
+        for (_, cands) in visitor.candidates.iter_mut() {
+            cands.sort();
+            cands.dedup();
         }
-    }
 
-    fn reset(&mut self) {
-        self.writes.clear();
-        self.skip_pair = None;
+        Candidates { c: visitor.candidates }
     }
 }
 
-/////////////////////////////////////////////////////
-// Candidate accumulation
-
 /// If the pair of places is being considered for merging, returns the candidate which would be
 /// merged in order to accomplish this.
 ///
@@ -743,7 +440,7 @@ fn places_to_candidate_pair<'tcx>(
 
 struct FindAssignments<'a, 'tcx> {
     body: &'a Body<'tcx>,
-    candidates: &'a mut FxIndexMap<Local, Vec<Local>>,
+    candidates: FxIndexMap<Local, Vec<Local>>,
     borrowed: &'a DenseBitSet<Local>,
 }
 
@@ -803,22 +500,252 @@ fn dest_prop_mir_dump<'tcx>(
     tcx: TyCtxt<'tcx>,
     body: &Body<'tcx>,
     points: &DenseLocationMap,
-    live: &SparseIntervalMatrix<Local, PointIndex>,
-    round: usize,
+    live: &SparseIntervalMatrix<RelevantLocal, TwoStepIndex>,
+    relevant: &RelevantLocals,
 ) {
-    let locals_live_at = |location| {
+    let locals_live_at = |location, effect| {
         let location = points.point_from_location(location);
-        live.rows().filter(|&r| live.contains(r, location)).collect::<Vec<_>>()
+        let location = TwoStepIndex::new(location, effect);
+        live.rows()
+            .filter(|&r| live.contains(r, location))
+            .map(|rl| relevant.original[rl])
+            .collect::<Vec<_>>()
     };
 
     if let Some(dumper) = MirDumper::new(tcx, "DestinationPropagation-dataflow", body) {
         let extra_data = &|pass_where, w: &mut dyn std::io::Write| {
             if let PassWhere::BeforeLocation(loc) = pass_where {
-                writeln!(w, "        // live: {:?}", locals_live_at(loc))?;
+                writeln!(w, "        // before: {:?}", locals_live_at(loc, Effect::Before))?;
+            }
+            if let PassWhere::AfterLocation(loc) = pass_where {
+                writeln!(w, "        // after: {:?}", locals_live_at(loc, Effect::After))?;
             }
             Ok(())
         };
 
-        dumper.set_disambiguator(&round).set_extra_data(extra_data).dump_mir(body)
+        dumper.set_extra_data(extra_data).dump_mir(body)
+    }
+}
+
+struct MaybeTwoStepLiveLocals;
+
+#[derive(Copy, Clone, Debug)]
+enum Effect {
+    Before,
+    After,
+}
+
+rustc_index::newtype_index! {
+    /// A `PointIndex` but with the lower bit encoding early/late inside the statement.
+    #[orderable]
+    #[debug_format = "TwoStepIndex({})"]
+    struct TwoStepIndex {}
+}
+
+impl TwoStepIndex {
+    fn new(point: PointIndex, effect: Effect) -> TwoStepIndex {
+        let effect = match effect {
+            Effect::Before => 0,
+            Effect::After => 1,
+        };
+        TwoStepIndex::from_u32(2 * point.as_u32() + (effect as u32))
+    }
+}
+
+struct VisitPlacesWith<F>(F);
+
+impl<'tcx, F> Visitor<'tcx> for VisitPlacesWith<F>
+where
+    F: FnMut(Place<'tcx>, PlaceContext),
+{
+    fn visit_place(&mut self, place: &Place<'tcx>, ctxt: PlaceContext, _: Location) {
+        (self.0)(*place, ctxt);
+        for proj in place.projection.iter() {
+            match proj {
+                ProjectionElem::Index(index) => (self.0)(
+                    index.into(),
+                    PlaceContext::NonMutatingUse(NonMutatingUseContext::Copy),
+                ),
+                ProjectionElem::Deref
+                | ProjectionElem::Field(..)
+                | ProjectionElem::Downcast(..)
+                | ProjectionElem::ConstantIndex { .. }
+                | ProjectionElem::Subslice { .. }
+                | ProjectionElem::OpaqueCast(..)
+                | ProjectionElem::UnwrapUnsafeBinder(..)
+                | ProjectionElem::Subtype(..) => {}
+            };
+        }
+    }
+}
+
+impl<'tcx> Analysis<'tcx> for MaybeTwoStepLiveLocals {
+    type Domain = DenseBitSet<Local>;
+    type Direction = Backward;
+
+    const NAME: &'static str = "transitive liveness";
+
+    fn bottom_value(&self, body: &Body<'tcx>) -> DenseBitSet<Local> {
+        // bottom = not live
+        DenseBitSet::new_empty(body.local_decls.len())
+    }
+
+    fn initialize_start_block(&self, _: &Body<'tcx>, _: &mut DenseBitSet<Local>) {
+        // No variables are live until we observe a use
+    }
+
+    // This happens between the previous statement and this one.
+    #[tracing::instrument(level = "trace", skip(self, statement))]
+    fn apply_primary_statement_effect(
+        &mut self,
+        state: &mut DenseBitSet<Local>,
+        statement: &Statement<'tcx>,
+        location: Location,
+    ) {
+        VisitPlacesWith(|place, ctxt| match DefUse::for_place(place, ctxt) {
+            DefUse::Def => {
+                state.remove(place.local);
+            }
+            DefUse::Use => {
+                state.insert(place.local);
+            }
+            DefUse::PartialWrite | DefUse::NonUse => {}
+        })
+        .visit_statement(statement, location);
+    }
+
+    // This happens between this statement and the next one.
+    #[tracing::instrument(level = "trace", skip(self, statement))]
+    fn apply_early_statement_effect(
+        &mut self,
+        state: &mut DenseBitSet<Local>,
+        statement: &Statement<'tcx>,
+        location: Location,
+    ) {
+        // We need to ensure we have a non-zero live range even for dead stores. This is done by
+        // marking all the writes locals as live in the second half of the statement.
+        VisitPlacesWith(|place: Place<'tcx>, ctxt| match DefUse::for_place(place, ctxt) {
+            DefUse::Def | DefUse::PartialWrite => {
+                state.insert(place.local);
+            }
+            // We already perform the reads in the first part of the statement. As statements are
+            // not splittable, we do not need to re-read the same values.
+            DefUse::Use | DefUse::NonUse => {}
+        })
+        .visit_statement(statement, location);
+    }
+
+    // We model terminator as a special case in this two-step analysis. Consider the terminator
+    // `destination = func(arg0...)`.
+    //
+    // -- state at (location, Effect::Before)
+    // read(arg0)...
+    // write(destination)
+    // -- state at (location, Effect::After)
+    // read(arg0)...
+
+    // This happens between the last statement and the terminator.
+    #[tracing::instrument(level = "trace", skip(self, terminator))]
+    fn apply_primary_terminator_effect<'mir>(
+        &mut self,
+        state: &mut DenseBitSet<Local>,
+        terminator: &'mir Terminator<'tcx>,
+        location: Location,
+    ) -> TerminatorEdges<'mir, 'tcx> {
+        // Consider that all writes in this terminator happen at the start of the execution of the
+        // terminator. For instance if we pass a return-pointer to a `Call` terminator.
+        VisitPlacesWith(|place: Place<'tcx>, ctxt| match DefUse::for_place(place, ctxt) {
+            DefUse::Def => {
+                state.remove(place.local);
+            }
+            DefUse::Use => {
+                state.insert(place.local);
+            }
+            DefUse::PartialWrite | DefUse::NonUse => {}
+        })
+        .visit_terminator(terminator, location);
+        terminator.edges()
+    }
+
+    // This happens between the terminator and the end of the block.
+    #[tracing::instrument(level = "trace", skip(self, terminator))]
+    fn apply_early_terminator_effect<'mir>(
+        &mut self,
+        state: &mut DenseBitSet<Local>,
+        terminator: &'mir Terminator<'tcx>,
+        location: Location,
+    ) {
+        // Consider that all reads in this terminator happen at the end of the execution of the
+        // terminator, even after it may have written to the destination local. For instance if we
+        // pass arguments as pointers to a `Call` terminator.
+        VisitPlacesWith(|place: Place<'tcx>, ctxt| match DefUse::for_place(place, ctxt) {
+            DefUse::Def | DefUse::Use | DefUse::PartialWrite => {
+                state.insert(place.local);
+            }
+            DefUse::NonUse => {}
+        })
+        .visit_terminator(terminator, location);
     }
 }
+
+/// Add points depending on the result of the given dataflow analysis.
+fn save_as_intervals<'tcx>(
+    elements: &DenseLocationMap,
+    body: &Body<'tcx>,
+    relevant: &IndexSlice<RelevantLocal, Local>,
+    results: Results<DenseBitSet<Local>>,
+) -> SparseIntervalMatrix<RelevantLocal, TwoStepIndex> {
+    let mut values = SparseIntervalMatrix::new(2 * elements.num_points());
+    let mut state = MaybeTwoStepLiveLocals.bottom_value(body);
+    let reachable_blocks = traversal::reachable_as_bitset(body);
+
+    let two_step_loc = |location, effect| {
+        let point = elements.point_from_location(location);
+        TwoStepIndex::new(point, effect)
+    };
+    let mut prepend_at = |state: &mut DenseBitSet<Local>, twostep| {
+        for (relevant, &original) in relevant.iter_enumerated() {
+            if state.contains(original) {
+                values.prepend(relevant, twostep);
+            }
+        }
+    };
+
+    // Iterate blocks in decreasing order, to visit locations in decreasing order. This
+    // allows to use the more efficient `prepend` method to interval sets.
+    for block in body.basic_blocks.indices().rev() {
+        if !reachable_blocks.contains(block) {
+            continue;
+        }
+
+        state.clone_from(&results[block]);
+
+        let block_data = &body.basic_blocks[block];
+        let loc = Location { block, statement_index: block_data.statements.len() };
+
+        let term = block_data.terminator();
+        let mut twostep = two_step_loc(loc, Effect::After);
+        MaybeTwoStepLiveLocals.apply_early_terminator_effect(&mut state, term, loc);
+        prepend_at(&mut state, twostep);
+
+        twostep = TwoStepIndex::from_u32(twostep.as_u32() - 1);
+        debug_assert_eq!(twostep, two_step_loc(loc, Effect::Before));
+        MaybeTwoStepLiveLocals.apply_primary_terminator_effect(&mut state, term, loc);
+        prepend_at(&mut state, twostep);
+
+        for (statement_index, stmt) in block_data.statements.iter().enumerate().rev() {
+            let loc = Location { block, statement_index };
+            twostep = TwoStepIndex::from_u32(twostep.as_u32() - 1);
+            debug_assert_eq!(twostep, two_step_loc(loc, Effect::After));
+            MaybeTwoStepLiveLocals.apply_early_statement_effect(&mut state, stmt, loc);
+            prepend_at(&mut state, twostep);
+
+            twostep = TwoStepIndex::from_u32(twostep.as_u32() - 1);
+            debug_assert_eq!(twostep, two_step_loc(loc, Effect::Before));
+            MaybeTwoStepLiveLocals.apply_primary_statement_effect(&mut state, stmt, loc);
+            prepend_at(&mut state, twostep);
+        }
+    }
+
+    values
+}
diff --git a/tests/mir-opt/dest-prop/cycle.main.DestinationPropagation.panic-abort.diff b/tests/mir-opt/dest-prop/cycle.main.DestinationPropagation.panic-abort.diff
index 5d8aaedae37..f9b8881ae3b 100644
--- a/tests/mir-opt/dest-prop/cycle.main.DestinationPropagation.panic-abort.diff
+++ b/tests/mir-opt/dest-prop/cycle.main.DestinationPropagation.panic-abort.diff
@@ -9,15 +9,15 @@
       let mut _6: i32;
       scope 1 {
 -         debug x => _1;
-+         debug x => _6;
++         debug x => _4;
           let _2: i32;
           scope 2 {
 -             debug y => _2;
-+             debug y => _6;
++             debug y => _4;
               let _3: i32;
               scope 3 {
 -                 debug z => _3;
-+                 debug z => _6;
++                 debug z => _4;
               }
           }
       }
@@ -26,7 +26,7 @@
 -         StorageLive(_1);
 -         _1 = val() -> [return: bb1, unwind unreachable];
 +         nop;
-+         _6 = val() -> [return: bb1, unwind unreachable];
++         _4 = val() -> [return: bb1, unwind unreachable];
       }
   
       bb1: {
@@ -47,16 +47,14 @@
 +         nop;
 +         nop;
           StorageLive(_5);
--         StorageLive(_6);
+          StorageLive(_6);
 -         _6 = copy _1;
-+         nop;
-+         nop;
++         _6 = copy _4;
           _5 = std::mem::drop::<i32>(move _6) -> [return: bb2, unwind unreachable];
       }
   
       bb2: {
--         StorageDead(_6);
-+         nop;
+          StorageDead(_6);
           StorageDead(_5);
           _0 = const ();
 -         StorageDead(_3);
diff --git a/tests/mir-opt/dest-prop/cycle.main.DestinationPropagation.panic-unwind.diff b/tests/mir-opt/dest-prop/cycle.main.DestinationPropagation.panic-unwind.diff
index 05c9bcc1d73..013e40ff462 100644
--- a/tests/mir-opt/dest-prop/cycle.main.DestinationPropagation.panic-unwind.diff
+++ b/tests/mir-opt/dest-prop/cycle.main.DestinationPropagation.panic-unwind.diff
@@ -9,15 +9,15 @@
       let mut _6: i32;
       scope 1 {
 -         debug x => _1;
-+         debug x => _6;
++         debug x => _4;
           let _2: i32;
           scope 2 {
 -             debug y => _2;
-+             debug y => _6;
++             debug y => _4;
               let _3: i32;
               scope 3 {
 -                 debug z => _3;
-+                 debug z => _6;
++                 debug z => _4;
               }
           }
       }
@@ -26,7 +26,7 @@
 -         StorageLive(_1);
 -         _1 = val() -> [return: bb1, unwind continue];
 +         nop;
-+         _6 = val() -> [return: bb1, unwind continue];
++         _4 = val() -> [return: bb1, unwind continue];
       }
   
       bb1: {
@@ -47,16 +47,14 @@
 +         nop;
 +         nop;
           StorageLive(_5);
--         StorageLive(_6);
+          StorageLive(_6);
 -         _6 = copy _1;
-+         nop;
-+         nop;
++         _6 = copy _4;
           _5 = std::mem::drop::<i32>(move _6) -> [return: bb2, unwind continue];
       }
   
       bb2: {
--         StorageDead(_6);
-+         nop;
+          StorageDead(_6);
           StorageDead(_5);
           _0 = const ();
 -         StorageDead(_3);